aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorChris Lu <chrislusf@users.noreply.github.com>2025-07-11 10:19:27 -0700
committerGitHub <noreply@github.com>2025-07-11 10:19:27 -0700
commit51543bbb872161b7fa1c6ae30998981d4c55bc89 (patch)
treed7b7d646575bc534423d8ad8fe444765829b7c58
parenta9e1f006739d397087ba8e7c632de223be40707d (diff)
downloadseaweedfs-51543bbb872161b7fa1c6ae30998981d4c55bc89.tar.xz
seaweedfs-51543bbb872161b7fa1c6ae30998981d4c55bc89.zip
Admin UI: Add message queue to admin UI (#6958)
* add a menu item "Message Queue" * add a menu item "Message Queue" * move the "brokers" link under it. * add "topics", "subscribers". Add pages for them. * refactor * show topic details * admin display publisher and subscriber info * remove publisher and subscribers from the topic row pull down * collecting more stats from publishers and subscribers * fix layout * fix publisher name * add local listeners for mq broker and agent * render consumer group offsets * remove subscribers from left menu * topic with retention * support editing topic retention * show retention when listing topics * create bucket * Update s3_buckets_templ.go * embed the static assets into the binary fix https://github.com/seaweedfs/seaweedfs/issues/6964
-rw-r--r--.gitignore5
-rw-r--r--test/mq/Makefile228
-rw-r--r--test/mq/README.md244
-rw-r--r--test/mq/consumer/main.go192
-rw-r--r--test/mq/producer/main.go172
-rw-r--r--weed/admin/dash/admin_data.go83
-rw-r--r--weed/admin/dash/admin_server.go293
-rw-r--r--weed/admin/dash/bucket_management.go91
-rw-r--r--weed/admin/dash/mq_management.go615
-rw-r--r--weed/admin/dash/topic_retention.go296
-rw-r--r--weed/admin/dash/types.go144
-rw-r--r--weed/admin/handlers/admin_handlers.go31
-rw-r--r--weed/admin/handlers/cluster_handlers.go27
-rw-r--r--weed/admin/handlers/mq_handlers.go238
-rw-r--r--weed/admin/static_embed.go14
-rw-r--r--weed/admin/view/app/admin.templ12
-rw-r--r--weed/admin/view/app/admin_templ.go165
-rw-r--r--weed/admin/view/app/cluster_brokers.templ144
-rw-r--r--weed/admin/view/app/cluster_brokers_templ.go168
-rw-r--r--weed/admin/view/app/s3_buckets.templ272
-rw-r--r--weed/admin/view/app/s3_buckets_templ.go138
-rw-r--r--weed/admin/view/app/subscribers.templ151
-rw-r--r--weed/admin/view/app/subscribers_templ.go246
-rw-r--r--weed/admin/view/app/topic_details.templ677
-rw-r--r--weed/admin/view/app/topic_details_templ.go949
-rw-r--r--weed/admin/view/app/topics.templ511
-rw-r--r--weed/admin/view/app/topics_templ.go230
-rw-r--r--weed/admin/view/layout/layout.templ61
-rw-r--r--weed/admin/view/layout/layout_templ.go161
-rw-r--r--weed/command/admin.go11
-rw-r--r--weed/command/mq_agent.go20
-rw-r--r--weed/command/mq_broker.go19
-rw-r--r--weed/filer_client/filer_client_accessor.go15
-rw-r--r--weed/mq/broker/broker_grpc_configure.go1
-rw-r--r--weed/mq/broker/broker_grpc_lookup.go260
-rw-r--r--weed/mq/broker/broker_grpc_pub.go23
-rw-r--r--weed/mq/broker/broker_grpc_sub.go14
-rw-r--r--weed/mq/topic/local_partition_publishers.go67
-rw-r--r--weed/mq/topic/local_partition_subscribers.go74
-rw-r--r--weed/mq/topic/topic.go34
-rw-r--r--weed/pb/mq_broker.proto64
-rw-r--r--weed/pb/mq_pb/mq_broker.pb.go1218
-rw-r--r--weed/pb/mq_pb/mq_broker_grpc.pb.go114
-rw-r--r--weed/pb/worker_pb/worker.pb.go944
44 files changed, 8288 insertions, 1148 deletions
diff --git a/.gitignore b/.gitignore
index 363310f81..473527d24 100644
--- a/.gitignore
+++ b/.gitignore
@@ -95,3 +95,8 @@ docker/weed_sub
docker/weed_pub
weed/mq/schema/example.parquet
docker/agent_sub_record
+test/mq/bin/consumer
+test/mq/bin/producer
+test/producer
+bin/weed
+weed_binary
diff --git a/test/mq/Makefile b/test/mq/Makefile
new file mode 100644
index 000000000..da5ebd1bd
--- /dev/null
+++ b/test/mq/Makefile
@@ -0,0 +1,228 @@
+# SeaweedFS Message Queue Test Makefile
+
+# Build configuration
+GO_BUILD_CMD=go build -o bin/$(1) $(2)
+GO_RUN_CMD=go run $(1) $(2)
+
+# Default values
+AGENT_ADDR?=localhost:16777
+TOPIC_NAMESPACE?=test
+TOPIC_NAME?=test-topic
+PARTITION_COUNT?=4
+MESSAGE_COUNT?=100
+CONSUMER_GROUP?=test-consumer-group
+CONSUMER_INSTANCE?=test-consumer-1
+
+# Create bin directory
+$(shell mkdir -p bin)
+
+.PHONY: all build clean producer consumer test help
+
+all: build
+
+# Build targets
+build: build-producer build-consumer
+
+build-producer:
+ @echo "Building producer..."
+ $(call GO_BUILD_CMD,producer,./producer)
+
+build-consumer:
+ @echo "Building consumer..."
+ $(call GO_BUILD_CMD,consumer,./consumer)
+
+# Run targets
+producer: build-producer
+ @echo "Starting producer..."
+ ./bin/producer \
+ -agent=$(AGENT_ADDR) \
+ -namespace=$(TOPIC_NAMESPACE) \
+ -topic=$(TOPIC_NAME) \
+ -partitions=$(PARTITION_COUNT) \
+ -messages=$(MESSAGE_COUNT) \
+ -publisher=test-producer \
+ -size=1024 \
+ -interval=100ms
+
+consumer: build-consumer
+ @echo "Starting consumer..."
+ ./bin/consumer \
+ -agent=$(AGENT_ADDR) \
+ -namespace=$(TOPIC_NAMESPACE) \
+ -topic=$(TOPIC_NAME) \
+ -group=$(CONSUMER_GROUP) \
+ -instance=$(CONSUMER_INSTANCE) \
+ -max-partitions=10 \
+ -window-size=100 \
+ -offset=latest \
+ -show-messages=true \
+ -log-progress=true
+
+# Run producer directly with go run
+run-producer:
+ @echo "Running producer directly..."
+ $(call GO_RUN_CMD,./producer, \
+ -agent=$(AGENT_ADDR) \
+ -namespace=$(TOPIC_NAMESPACE) \
+ -topic=$(TOPIC_NAME) \
+ -partitions=$(PARTITION_COUNT) \
+ -messages=$(MESSAGE_COUNT) \
+ -publisher=test-producer \
+ -size=1024 \
+ -interval=100ms)
+
+# Run consumer directly with go run
+run-consumer:
+ @echo "Running consumer directly..."
+ $(call GO_RUN_CMD,./consumer, \
+ -agent=$(AGENT_ADDR) \
+ -namespace=$(TOPIC_NAMESPACE) \
+ -topic=$(TOPIC_NAME) \
+ -group=$(CONSUMER_GROUP) \
+ -instance=$(CONSUMER_INSTANCE) \
+ -max-partitions=10 \
+ -window-size=100 \
+ -offset=latest \
+ -show-messages=true \
+ -log-progress=true)
+
+# Test scenarios
+test: test-basic
+
+test-basic: build
+ @echo "Running basic producer/consumer test..."
+ @echo "1. Starting consumer in background..."
+ ./bin/consumer \
+ -agent=$(AGENT_ADDR) \
+ -namespace=$(TOPIC_NAMESPACE) \
+ -topic=$(TOPIC_NAME) \
+ -group=$(CONSUMER_GROUP) \
+ -instance=$(CONSUMER_INSTANCE) \
+ -offset=earliest \
+ -show-messages=false \
+ -log-progress=true & \
+ CONSUMER_PID=$$!; \
+ echo "Consumer PID: $$CONSUMER_PID"; \
+ sleep 2; \
+ echo "2. Starting producer..."; \
+ ./bin/producer \
+ -agent=$(AGENT_ADDR) \
+ -namespace=$(TOPIC_NAMESPACE) \
+ -topic=$(TOPIC_NAME) \
+ -partitions=$(PARTITION_COUNT) \
+ -messages=$(MESSAGE_COUNT) \
+ -publisher=test-producer \
+ -size=1024 \
+ -interval=50ms; \
+ echo "3. Waiting for consumer to process messages..."; \
+ sleep 5; \
+ echo "4. Stopping consumer..."; \
+ kill $$CONSUMER_PID || true; \
+ echo "Test completed!"
+
+test-performance: build
+ @echo "Running performance test..."
+ @echo "1. Starting consumer in background..."
+ ./bin/consumer \
+ -agent=$(AGENT_ADDR) \
+ -namespace=$(TOPIC_NAMESPACE) \
+ -topic=perf-test \
+ -group=perf-consumer-group \
+ -instance=perf-consumer-1 \
+ -offset=earliest \
+ -show-messages=false \
+ -log-progress=true & \
+ CONSUMER_PID=$$!; \
+ echo "Consumer PID: $$CONSUMER_PID"; \
+ sleep 2; \
+ echo "2. Starting high-throughput producer..."; \
+ ./bin/producer \
+ -agent=$(AGENT_ADDR) \
+ -namespace=$(TOPIC_NAMESPACE) \
+ -topic=perf-test \
+ -partitions=8 \
+ -messages=1000 \
+ -publisher=perf-producer \
+ -size=512 \
+ -interval=10ms; \
+ echo "3. Waiting for consumer to process messages..."; \
+ sleep 10; \
+ echo "4. Stopping consumer..."; \
+ kill $$CONSUMER_PID || true; \
+ echo "Performance test completed!"
+
+test-multiple-consumers: build
+ @echo "Running multiple consumers test..."
+ @echo "1. Starting multiple consumers in background..."
+ ./bin/consumer \
+ -agent=$(AGENT_ADDR) \
+ -namespace=$(TOPIC_NAMESPACE) \
+ -topic=multi-test \
+ -group=multi-consumer-group \
+ -instance=consumer-1 \
+ -offset=earliest \
+ -show-messages=false \
+ -log-progress=true & \
+ CONSUMER1_PID=$$!; \
+ ./bin/consumer \
+ -agent=$(AGENT_ADDR) \
+ -namespace=$(TOPIC_NAMESPACE) \
+ -topic=multi-test \
+ -group=multi-consumer-group \
+ -instance=consumer-2 \
+ -offset=earliest \
+ -show-messages=false \
+ -log-progress=true & \
+ CONSUMER2_PID=$$!; \
+ echo "Consumer PIDs: $$CONSUMER1_PID, $$CONSUMER2_PID"; \
+ sleep 2; \
+ echo "2. Starting producer..."; \
+ ./bin/producer \
+ -agent=$(AGENT_ADDR) \
+ -namespace=$(TOPIC_NAMESPACE) \
+ -topic=multi-test \
+ -partitions=8 \
+ -messages=200 \
+ -publisher=multi-producer \
+ -size=256 \
+ -interval=50ms; \
+ echo "3. Waiting for consumers to process messages..."; \
+ sleep 10; \
+ echo "4. Stopping consumers..."; \
+ kill $$CONSUMER1_PID $$CONSUMER2_PID || true; \
+ echo "Multiple consumers test completed!"
+
+# Clean up
+clean:
+ @echo "Cleaning up..."
+ rm -rf bin/
+ go clean -cache
+
+# Help
+help:
+ @echo "SeaweedFS Message Queue Test Makefile"
+ @echo ""
+ @echo "Usage:"
+ @echo " make build - Build producer and consumer binaries"
+ @echo " make producer - Run producer (builds first)"
+ @echo " make consumer - Run consumer (builds first)"
+ @echo " make run-producer - Run producer directly with go run"
+ @echo " make run-consumer - Run consumer directly with go run"
+ @echo " make test - Run basic producer/consumer test"
+ @echo " make test-performance - Run performance test"
+ @echo " make test-multiple-consumers - Run multiple consumers test"
+ @echo " make clean - Clean up build artifacts"
+ @echo ""
+ @echo "Configuration (set via environment variables):"
+ @echo " AGENT_ADDR=10.21.152.113:16777 - MQ agent address"
+ @echo " TOPIC_NAMESPACE=test - Topic namespace"
+ @echo " TOPIC_NAME=test-topic - Topic name"
+ @echo " PARTITION_COUNT=4 - Number of partitions"
+ @echo " MESSAGE_COUNT=100 - Number of messages to produce"
+ @echo " CONSUMER_GROUP=test-consumer-group - Consumer group name"
+ @echo " CONSUMER_INSTANCE=test-consumer-1 - Consumer instance ID"
+ @echo ""
+ @echo "Examples:"
+ @echo " make producer MESSAGE_COUNT=1000 PARTITION_COUNT=8"
+ @echo " make consumer CONSUMER_GROUP=my-group"
+ @echo " make test AGENT_ADDR=10.21.152.113:16777 MESSAGE_COUNT=500" \ No newline at end of file
diff --git a/test/mq/README.md b/test/mq/README.md
new file mode 100644
index 000000000..7fa7e39eb
--- /dev/null
+++ b/test/mq/README.md
@@ -0,0 +1,244 @@
+# SeaweedFS Message Queue Test Suite
+
+This directory contains test programs for SeaweedFS Message Queue (MQ) functionality, including message producers and consumers.
+
+## Prerequisites
+
+1. **SeaweedFS with MQ Broker and Agent**: You need a running SeaweedFS instance with MQ broker and agent enabled
+2. **Go**: Go 1.19 or later required for building the test programs
+
+## Quick Start
+
+### 1. Start SeaweedFS with MQ Broker and Agent
+
+```bash
+# Start SeaweedFS server with MQ broker and agent
+weed server -mq.broker -mq.agent -filer -volume
+
+# Or start components separately
+weed master
+weed volume -mserver=localhost:9333
+weed filer -master=localhost:9333
+weed mq.broker -filer=localhost:8888
+weed mq.agent -brokers=localhost:17777
+```
+
+### 2. Build Test Programs
+
+```bash
+# Build both producer and consumer
+make build
+
+# Or build individually
+make build-producer
+make build-consumer
+```
+
+### 3. Run Basic Test
+
+```bash
+# Run a basic producer/consumer test
+make test
+
+# Or run producer and consumer manually
+make consumer & # Start consumer in background
+make producer # Start producer
+```
+
+## Test Programs
+
+### Producer (`producer/main.go`)
+
+Generates structured messages and publishes them to a SeaweedMQ topic via the MQ agent.
+
+**Usage:**
+```bash
+./bin/producer [options]
+```
+
+**Options:**
+- `-agent`: MQ agent address (default: localhost:16777)
+- `-namespace`: Topic namespace (default: test)
+- `-topic`: Topic name (default: test-topic)
+- `-partitions`: Number of partitions (default: 4)
+- `-messages`: Number of messages to produce (default: 100)
+- `-publisher`: Publisher name (default: test-producer)
+- `-size`: Message size in bytes (default: 1024)
+- `-interval`: Interval between messages (default: 100ms)
+
+**Example:**
+```bash
+./bin/producer -agent=localhost:16777 -namespace=test -topic=my-topic -messages=1000 -interval=50ms
+```
+
+### Consumer (`consumer/main.go`)
+
+Consumes structured messages from a SeaweedMQ topic via the MQ agent.
+
+**Usage:**
+```bash
+./bin/consumer [options]
+```
+
+**Options:**
+- `-agent`: MQ agent address (default: localhost:16777)
+- `-namespace`: Topic namespace (default: test)
+- `-topic`: Topic name (default: test-topic)
+- `-group`: Consumer group name (default: test-consumer-group)
+- `-instance`: Consumer group instance ID (default: test-consumer-1)
+- `-max-partitions`: Maximum number of partitions to consume (default: 10)
+- `-window-size`: Sliding window size for concurrent processing (default: 100)
+- `-offset`: Offset type: earliest, latest, timestamp (default: latest)
+- `-offset-ts`: Offset timestamp in nanoseconds (for timestamp offset type)
+- `-filter`: Message filter (default: empty)
+- `-show-messages`: Show consumed messages (default: true)
+- `-log-progress`: Log progress every 10 messages (default: true)
+
+**Example:**
+```bash
+./bin/consumer -agent=localhost:16777 -namespace=test -topic=my-topic -group=my-group -offset=earliest
+```
+
+## Makefile Commands
+
+### Building
+- `make build`: Build both producer and consumer binaries
+- `make build-producer`: Build producer only
+- `make build-consumer`: Build consumer only
+
+### Running
+- `make producer`: Build and run producer
+- `make consumer`: Build and run consumer
+- `make run-producer`: Run producer directly with go run
+- `make run-consumer`: Run consumer directly with go run
+
+### Testing
+- `make test`: Run basic producer/consumer test
+- `make test-performance`: Run performance test (1000 messages, 8 partitions)
+- `make test-multiple-consumers`: Run test with multiple consumers
+
+### Cleanup
+- `make clean`: Remove build artifacts
+
+### Help
+- `make help`: Show detailed help
+
+## Configuration
+
+Configure tests using environment variables:
+
+```bash
+export AGENT_ADDR=localhost:16777
+export TOPIC_NAMESPACE=test
+export TOPIC_NAME=test-topic
+export PARTITION_COUNT=4
+export MESSAGE_COUNT=100
+export CONSUMER_GROUP=test-consumer-group
+export CONSUMER_INSTANCE=test-consumer-1
+```
+
+## Example Usage Scenarios
+
+### 1. Basic Producer/Consumer Test
+
+```bash
+# Terminal 1: Start consumer
+make consumer
+
+# Terminal 2: Run producer
+make producer MESSAGE_COUNT=50
+```
+
+### 2. Performance Testing
+
+```bash
+# Test with high throughput
+make test-performance
+```
+
+### 3. Multiple Consumer Groups
+
+```bash
+# Terminal 1: Consumer group 1
+make consumer CONSUMER_GROUP=group1
+
+# Terminal 2: Consumer group 2
+make consumer CONSUMER_GROUP=group2
+
+# Terminal 3: Producer
+make producer MESSAGE_COUNT=200
+```
+
+### 4. Different Offset Types
+
+```bash
+# Consume from earliest
+make consumer OFFSET=earliest
+
+# Consume from latest
+make consumer OFFSET=latest
+
+# Consume from timestamp
+make consumer OFFSET=timestamp OFFSET_TS=1699000000000000000
+```
+
+## Troubleshooting
+
+### Common Issues
+
+1. **Connection Refused**: Make sure SeaweedFS MQ agent is running on the specified address
+2. **Agent Not Found**: Ensure both MQ broker and agent are running (agent requires broker)
+3. **Topic Not Found**: The producer will create the topic automatically on first publish
+4. **Consumer Not Receiving Messages**: Check if consumer group offset is correct (try `earliest`)
+5. **Build Failures**: Ensure you're running from the SeaweedFS root directory
+
+### Debug Mode
+
+Enable verbose logging:
+```bash
+# Run with debug logging
+GLOG_v=4 make producer
+GLOG_v=4 make consumer
+```
+
+### Check Broker and Agent Status
+
+```bash
+# Check if broker is running
+curl http://localhost:9333/cluster/brokers
+
+# Check if agent is running (if running as server)
+curl http://localhost:9333/cluster/agents
+
+# Or use weed shell
+weed shell -master=localhost:9333
+> mq.broker.list
+```
+
+## Architecture
+
+The test setup demonstrates:
+
+1. **Agent-Based Architecture**: Uses MQ agent as intermediary between clients and brokers
+2. **Structured Messages**: Messages use schema-based RecordValue format instead of raw bytes
+3. **Topic Management**: Creating and configuring topics with multiple partitions
+4. **Message Production**: Publishing structured messages with keys for partitioning
+5. **Message Consumption**: Consuming structured messages with consumer groups and offset management
+6. **Load Balancing**: Multiple consumers in same group share partition assignments
+7. **Fault Tolerance**: Graceful handling of agent and broker failures and reconnections
+
+## Files
+
+- `producer/main.go`: Message producer implementation
+- `consumer/main.go`: Message consumer implementation
+- `Makefile`: Build and test automation
+- `README.md`: This documentation
+- `bin/`: Built binaries (created during build)
+
+## Next Steps
+
+1. Modify the producer to send structured data using `RecordType`
+2. Implement message filtering in the consumer
+3. Add metrics collection and monitoring
+4. Test with multiple broker instances
+5. Implement schema evolution testing \ No newline at end of file
diff --git a/test/mq/consumer/main.go b/test/mq/consumer/main.go
new file mode 100644
index 000000000..e842b9721
--- /dev/null
+++ b/test/mq/consumer/main.go
@@ -0,0 +1,192 @@
+package main
+
+import (
+ "flag"
+ "fmt"
+ "log"
+ "os"
+ "os/signal"
+ "sync"
+ "syscall"
+ "time"
+
+ "github.com/seaweedfs/seaweedfs/weed/mq/client/agent_client"
+ "github.com/seaweedfs/seaweedfs/weed/mq/topic"
+ "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
+)
+
+var (
+ agentAddr = flag.String("agent", "localhost:16777", "MQ agent address")
+ topicNamespace = flag.String("namespace", "test", "topic namespace")
+ topicName = flag.String("topic", "test-topic", "topic name")
+ consumerGroup = flag.String("group", "test-consumer-group", "consumer group name")
+ consumerGroupInstanceId = flag.String("instance", "test-consumer-1", "consumer group instance id")
+ maxPartitions = flag.Int("max-partitions", 10, "maximum number of partitions to consume")
+ slidingWindowSize = flag.Int("window-size", 100, "sliding window size for concurrent processing")
+ offsetType = flag.String("offset", "latest", "offset type: earliest, latest, timestamp")
+ offsetTsNs = flag.Int64("offset-ts", 0, "offset timestamp in nanoseconds (for timestamp offset type)")
+ showMessages = flag.Bool("show-messages", true, "show consumed messages")
+ logProgress = flag.Bool("log-progress", true, "log progress every 10 messages")
+ filter = flag.String("filter", "", "message filter")
+)
+
+func main() {
+ flag.Parse()
+
+ fmt.Printf("Starting message consumer:\n")
+ fmt.Printf(" Agent: %s\n", *agentAddr)
+ fmt.Printf(" Topic: %s.%s\n", *topicNamespace, *topicName)
+ fmt.Printf(" Consumer Group: %s\n", *consumerGroup)
+ fmt.Printf(" Consumer Instance: %s\n", *consumerGroupInstanceId)
+ fmt.Printf(" Max Partitions: %d\n", *maxPartitions)
+ fmt.Printf(" Sliding Window Size: %d\n", *slidingWindowSize)
+ fmt.Printf(" Offset Type: %s\n", *offsetType)
+ fmt.Printf(" Filter: %s\n", *filter)
+
+ // Create topic
+ topicObj := topic.NewTopic(*topicNamespace, *topicName)
+
+ // Determine offset type
+ var pbOffsetType schema_pb.OffsetType
+ switch *offsetType {
+ case "earliest":
+ pbOffsetType = schema_pb.OffsetType_RESET_TO_EARLIEST
+ case "latest":
+ pbOffsetType = schema_pb.OffsetType_RESET_TO_LATEST
+ case "timestamp":
+ pbOffsetType = schema_pb.OffsetType_EXACT_TS_NS
+ default:
+ pbOffsetType = schema_pb.OffsetType_RESET_TO_LATEST
+ }
+
+ // Create subscribe option
+ option := &agent_client.SubscribeOption{
+ ConsumerGroup: *consumerGroup,
+ ConsumerGroupInstanceId: *consumerGroupInstanceId,
+ Topic: topicObj,
+ OffsetType: pbOffsetType,
+ OffsetTsNs: *offsetTsNs,
+ Filter: *filter,
+ MaxSubscribedPartitions: int32(*maxPartitions),
+ SlidingWindowSize: int32(*slidingWindowSize),
+ }
+
+ // Create subscribe session
+ session, err := agent_client.NewSubscribeSession(*agentAddr, option)
+ if err != nil {
+ log.Fatalf("Failed to create subscribe session: %v", err)
+ }
+ defer session.CloseSession()
+
+ // Statistics
+ var messageCount int64
+ var mu sync.Mutex
+ startTime := time.Now()
+
+ // Handle graceful shutdown
+ sigChan := make(chan os.Signal, 1)
+ signal.Notify(sigChan, syscall.SIGINT, syscall.SIGTERM)
+
+ // Channel to signal completion
+ done := make(chan error, 1)
+
+ // Start consuming messages
+ fmt.Printf("\nStarting to consume messages...\n")
+ go func() {
+ err := session.SubscribeMessageRecord(
+ // onEachMessageFn
+ func(key []byte, record *schema_pb.RecordValue) {
+ mu.Lock()
+ messageCount++
+ currentCount := messageCount
+ mu.Unlock()
+
+ if *showMessages {
+ fmt.Printf("Received message: key=%s\n", string(key))
+ printRecordValue(record)
+ }
+
+ if *logProgress && currentCount%10 == 0 {
+ elapsed := time.Since(startTime)
+ rate := float64(currentCount) / elapsed.Seconds()
+ fmt.Printf("Consumed %d messages (%.2f msg/sec)\n", currentCount, rate)
+ }
+ },
+ // onCompletionFn
+ func() {
+ fmt.Printf("Subscription completed\n")
+ done <- nil
+ },
+ )
+ if err != nil {
+ done <- err
+ }
+ }()
+
+ // Wait for signal or completion
+ select {
+ case <-sigChan:
+ fmt.Printf("\nReceived shutdown signal, stopping consumer...\n")
+ case err := <-done:
+ if err != nil {
+ log.Printf("Subscription error: %v", err)
+ }
+ }
+
+ // Print final statistics
+ mu.Lock()
+ finalCount := messageCount
+ mu.Unlock()
+
+ duration := time.Since(startTime)
+ fmt.Printf("Consumed %d messages in %v\n", finalCount, duration)
+ if duration.Seconds() > 0 {
+ fmt.Printf("Average throughput: %.2f messages/sec\n", float64(finalCount)/duration.Seconds())
+ }
+}
+
+func printRecordValue(record *schema_pb.RecordValue) {
+ if record == nil || record.Fields == nil {
+ fmt.Printf(" (empty record)\n")
+ return
+ }
+
+ for fieldName, value := range record.Fields {
+ fmt.Printf(" %s: %s\n", fieldName, formatValue(value))
+ }
+}
+
+func formatValue(value *schema_pb.Value) string {
+ if value == nil {
+ return "(nil)"
+ }
+
+ switch kind := value.Kind.(type) {
+ case *schema_pb.Value_BoolValue:
+ return fmt.Sprintf("%t", kind.BoolValue)
+ case *schema_pb.Value_Int32Value:
+ return fmt.Sprintf("%d", kind.Int32Value)
+ case *schema_pb.Value_Int64Value:
+ return fmt.Sprintf("%d", kind.Int64Value)
+ case *schema_pb.Value_FloatValue:
+ return fmt.Sprintf("%f", kind.FloatValue)
+ case *schema_pb.Value_DoubleValue:
+ return fmt.Sprintf("%f", kind.DoubleValue)
+ case *schema_pb.Value_BytesValue:
+ if len(kind.BytesValue) > 50 {
+ return fmt.Sprintf("bytes[%d] %x...", len(kind.BytesValue), kind.BytesValue[:50])
+ }
+ return fmt.Sprintf("bytes[%d] %x", len(kind.BytesValue), kind.BytesValue)
+ case *schema_pb.Value_StringValue:
+ if len(kind.StringValue) > 100 {
+ return fmt.Sprintf("\"%s...\"", kind.StringValue[:100])
+ }
+ return fmt.Sprintf("\"%s\"", kind.StringValue)
+ case *schema_pb.Value_ListValue:
+ return fmt.Sprintf("list[%d items]", len(kind.ListValue.Values))
+ case *schema_pb.Value_RecordValue:
+ return fmt.Sprintf("record[%d fields]", len(kind.RecordValue.Fields))
+ default:
+ return "(unknown)"
+ }
+}
diff --git a/test/mq/producer/main.go b/test/mq/producer/main.go
new file mode 100644
index 000000000..31fba4ca7
--- /dev/null
+++ b/test/mq/producer/main.go
@@ -0,0 +1,172 @@
+package main
+
+import (
+ "flag"
+ "fmt"
+ "log"
+ "time"
+
+ "github.com/seaweedfs/seaweedfs/weed/mq/client/agent_client"
+ "github.com/seaweedfs/seaweedfs/weed/mq/schema"
+ "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
+)
+
+var (
+ agentAddr = flag.String("agent", "localhost:16777", "MQ agent address")
+ topicNamespace = flag.String("namespace", "test", "topic namespace")
+ topicName = flag.String("topic", "test-topic", "topic name")
+ partitionCount = flag.Int("partitions", 4, "number of partitions")
+ messageCount = flag.Int("messages", 100, "number of messages to produce")
+ publisherName = flag.String("publisher", "test-producer", "publisher name")
+ messageSize = flag.Int("size", 1024, "message size in bytes")
+ interval = flag.Duration("interval", 100*time.Millisecond, "interval between messages")
+)
+
+// TestMessage represents the structure of messages we'll be sending
+type TestMessage struct {
+ ID int64 `json:"id"`
+ Message string `json:"message"`
+ Payload []byte `json:"payload"`
+ Timestamp int64 `json:"timestamp"`
+}
+
+func main() {
+ flag.Parse()
+
+ fmt.Printf("Starting message producer:\n")
+ fmt.Printf(" Agent: %s\n", *agentAddr)
+ fmt.Printf(" Topic: %s.%s\n", *topicNamespace, *topicName)
+ fmt.Printf(" Partitions: %d\n", *partitionCount)
+ fmt.Printf(" Messages: %d\n", *messageCount)
+ fmt.Printf(" Publisher: %s\n", *publisherName)
+ fmt.Printf(" Message Size: %d bytes\n", *messageSize)
+ fmt.Printf(" Interval: %v\n", *interval)
+
+ // Create an instance of the message struct to generate schema from
+ messageInstance := TestMessage{}
+
+ // Automatically generate RecordType from the struct
+ recordType := schema.StructToSchema(messageInstance)
+ if recordType == nil {
+ log.Fatalf("Failed to generate schema from struct")
+ }
+
+ fmt.Printf("\nGenerated schema with %d fields:\n", len(recordType.Fields))
+ for _, field := range recordType.Fields {
+ fmt.Printf(" - %s: %s\n", field.Name, getTypeString(field.Type))
+ }
+
+ topicSchema := schema.NewSchema(*topicNamespace, *topicName, recordType)
+
+ // Create publish session
+ session, err := agent_client.NewPublishSession(*agentAddr, topicSchema, *partitionCount, *publisherName)
+ if err != nil {
+ log.Fatalf("Failed to create publish session: %v", err)
+ }
+ defer session.CloseSession()
+
+ // Create message payload
+ payload := make([]byte, *messageSize)
+ for i := range payload {
+ payload[i] = byte(i % 256)
+ }
+
+ // Start producing messages
+ fmt.Printf("\nStarting to produce messages...\n")
+ startTime := time.Now()
+
+ for i := 0; i < *messageCount; i++ {
+ key := fmt.Sprintf("key-%d", i)
+
+ // Create a message struct
+ message := TestMessage{
+ ID: int64(i),
+ Message: fmt.Sprintf("This is message number %d", i),
+ Payload: payload[:min(100, len(payload))], // First 100 bytes
+ Timestamp: time.Now().UnixNano(),
+ }
+
+ // Convert struct to RecordValue
+ record := structToRecordValue(message)
+
+ err := session.PublishMessageRecord([]byte(key), record)
+ if err != nil {
+ log.Printf("Failed to publish message %d: %v", i, err)
+ continue
+ }
+
+ if (i+1)%10 == 0 {
+ fmt.Printf("Published %d messages\n", i+1)
+ }
+
+ if *interval > 0 {
+ time.Sleep(*interval)
+ }
+ }
+
+ duration := time.Since(startTime)
+ fmt.Printf("\nCompleted producing %d messages in %v\n", *messageCount, duration)
+ fmt.Printf("Throughput: %.2f messages/sec\n", float64(*messageCount)/duration.Seconds())
+}
+
+// Helper function to convert struct to RecordValue
+func structToRecordValue(msg TestMessage) *schema_pb.RecordValue {
+ return &schema_pb.RecordValue{
+ Fields: map[string]*schema_pb.Value{
+ "ID": {
+ Kind: &schema_pb.Value_Int64Value{
+ Int64Value: msg.ID,
+ },
+ },
+ "Message": {
+ Kind: &schema_pb.Value_StringValue{
+ StringValue: msg.Message,
+ },
+ },
+ "Payload": {
+ Kind: &schema_pb.Value_BytesValue{
+ BytesValue: msg.Payload,
+ },
+ },
+ "Timestamp": {
+ Kind: &schema_pb.Value_Int64Value{
+ Int64Value: msg.Timestamp,
+ },
+ },
+ },
+ }
+}
+
+func getTypeString(t *schema_pb.Type) string {
+ switch kind := t.Kind.(type) {
+ case *schema_pb.Type_ScalarType:
+ switch kind.ScalarType {
+ case schema_pb.ScalarType_BOOL:
+ return "bool"
+ case schema_pb.ScalarType_INT32:
+ return "int32"
+ case schema_pb.ScalarType_INT64:
+ return "int64"
+ case schema_pb.ScalarType_FLOAT:
+ return "float"
+ case schema_pb.ScalarType_DOUBLE:
+ return "double"
+ case schema_pb.ScalarType_BYTES:
+ return "bytes"
+ case schema_pb.ScalarType_STRING:
+ return "string"
+ }
+ case *schema_pb.Type_ListType:
+ return fmt.Sprintf("list<%s>", getTypeString(kind.ListType.ElementType))
+ case *schema_pb.Type_RecordType:
+ return "record"
+ }
+ return "unknown"
+}
+
+func min(a, b int) int {
+ if a < b {
+ return a
+ }
+ return b
+}
diff --git a/weed/admin/dash/admin_data.go b/weed/admin/dash/admin_data.go
index ef94169c2..dde6cfdbb 100644
--- a/weed/admin/dash/admin_data.go
+++ b/weed/admin/dash/admin_data.go
@@ -12,15 +12,16 @@ import (
)
type AdminData struct {
- Username string `json:"username"`
- TotalVolumes int `json:"total_volumes"`
- TotalFiles int64 `json:"total_files"`
- TotalSize int64 `json:"total_size"`
- MasterNodes []MasterNode `json:"master_nodes"`
- VolumeServers []VolumeServer `json:"volume_servers"`
- FilerNodes []FilerNode `json:"filer_nodes"`
- DataCenters []DataCenter `json:"datacenters"`
- LastUpdated time.Time `json:"last_updated"`
+ Username string `json:"username"`
+ TotalVolumes int `json:"total_volumes"`
+ TotalFiles int64 `json:"total_files"`
+ TotalSize int64 `json:"total_size"`
+ MasterNodes []MasterNode `json:"master_nodes"`
+ VolumeServers []VolumeServer `json:"volume_servers"`
+ FilerNodes []FilerNode `json:"filer_nodes"`
+ MessageBrokers []MessageBrokerNode `json:"message_brokers"`
+ DataCenters []DataCenter `json:"datacenters"`
+ LastUpdated time.Time `json:"last_updated"`
}
// Object Store Users management structures
@@ -76,6 +77,13 @@ type FilerNode struct {
LastUpdated time.Time `json:"last_updated"`
}
+type MessageBrokerNode struct {
+ Address string `json:"address"`
+ DataCenter string `json:"datacenter"`
+ Rack string `json:"rack"`
+ LastUpdated time.Time `json:"last_updated"`
+}
+
// GetAdminData retrieves admin data as a struct (for reuse by both JSON and HTML handlers)
func (s *AdminServer) GetAdminData(username string) (AdminData, error) {
if username == "" {
@@ -95,17 +103,21 @@ func (s *AdminServer) GetAdminData(username string) (AdminData, error) {
// Get filer nodes status
filerNodes := s.getFilerNodesStatus()
+ // Get message broker nodes status
+ messageBrokers := s.getMessageBrokerNodesStatus()
+
// Prepare admin data
adminData := AdminData{
- Username: username,
- TotalVolumes: topology.TotalVolumes,
- TotalFiles: topology.TotalFiles,
- TotalSize: topology.TotalSize,
- MasterNodes: masterNodes,
- VolumeServers: topology.VolumeServers,
- FilerNodes: filerNodes,
- DataCenters: topology.DataCenters,
- LastUpdated: topology.UpdatedAt,
+ Username: username,
+ TotalVolumes: topology.TotalVolumes,
+ TotalFiles: topology.TotalFiles,
+ TotalSize: topology.TotalSize,
+ MasterNodes: masterNodes,
+ VolumeServers: topology.VolumeServers,
+ FilerNodes: filerNodes,
+ MessageBrokers: messageBrokers,
+ DataCenters: topology.DataCenters,
+ LastUpdated: topology.UpdatedAt,
}
return adminData, nil
@@ -200,3 +212,38 @@ func (s *AdminServer) getFilerNodesStatus() []FilerNode {
return filerNodes
}
+
+// getMessageBrokerNodesStatus checks status of all message broker nodes using master's ListClusterNodes
+func (s *AdminServer) getMessageBrokerNodesStatus() []MessageBrokerNode {
+ var messageBrokers []MessageBrokerNode
+
+ // Get message broker nodes from master using ListClusterNodes
+ err := s.WithMasterClient(func(client master_pb.SeaweedClient) error {
+ resp, err := client.ListClusterNodes(context.Background(), &master_pb.ListClusterNodesRequest{
+ ClientType: cluster.BrokerType,
+ })
+ if err != nil {
+ return err
+ }
+
+ // Process each message broker node
+ for _, node := range resp.ClusterNodes {
+ messageBrokers = append(messageBrokers, MessageBrokerNode{
+ Address: node.Address,
+ DataCenter: node.DataCenter,
+ Rack: node.Rack,
+ LastUpdated: time.Now(),
+ })
+ }
+
+ return nil
+ })
+
+ if err != nil {
+ glog.Errorf("Failed to get message broker nodes from master %s: %v", s.masterAddress, err)
+ // Return empty list if we can't get broker info from master
+ return []MessageBrokerNode{}
+ }
+
+ return messageBrokers
+}
diff --git a/weed/admin/dash/admin_server.go b/weed/admin/dash/admin_server.go
index c8da2bbb7..95bff6deb 100644
--- a/weed/admin/dash/admin_server.go
+++ b/weed/admin/dash/admin_server.go
@@ -5,6 +5,7 @@ import (
"context"
"fmt"
"net/http"
+ "strconv"
"time"
"github.com/gin-gonic/gin"
@@ -16,6 +17,8 @@ import (
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/iam_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
+ "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
+ "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
"github.com/seaweedfs/seaweedfs/weed/security"
"github.com/seaweedfs/seaweedfs/weed/util"
"google.golang.org/grpc"
@@ -44,6 +47,9 @@ type AdminServer struct {
// Maintenance system
maintenanceManager *maintenance.MaintenanceManager
+ // Topic retention purger
+ topicRetentionPurger *TopicRetentionPurger
+
// Worker gRPC server
workerGrpcServer *WorkerGrpcServer
}
@@ -61,6 +67,9 @@ func NewAdminServer(masterAddress string, templateFS http.FileSystem, dataDir st
configPersistence: NewConfigPersistence(dataDir),
}
+ // Initialize topic retention purger
+ server.topicRetentionPurger = NewTopicRetentionPurger(server)
+
// Initialize credential manager with defaults
credentialManager, err := credential.NewCredentialManagerWithDefaults("")
if err != nil {
@@ -257,14 +266,41 @@ func (s *AdminServer) GetS3Buckets() ([]S3Bucket, error) {
quotaEnabled = false
}
+ // Get versioning and object lock information from extended attributes
+ versioningEnabled := false
+ objectLockEnabled := false
+ objectLockMode := ""
+ var objectLockDuration int32 = 0
+
+ if resp.Entry.Extended != nil {
+ if versioningBytes, exists := resp.Entry.Extended["s3.versioning"]; exists {
+ versioningEnabled = string(versioningBytes) == "Enabled"
+ }
+ if objectLockBytes, exists := resp.Entry.Extended["s3.objectlock"]; exists {
+ objectLockEnabled = string(objectLockBytes) == "Enabled"
+ }
+ if objectLockModeBytes, exists := resp.Entry.Extended["s3.objectlock.mode"]; exists {
+ objectLockMode = string(objectLockModeBytes)
+ }
+ if objectLockDurationBytes, exists := resp.Entry.Extended["s3.objectlock.duration"]; exists {
+ if duration, err := strconv.ParseInt(string(objectLockDurationBytes), 10, 32); err == nil {
+ objectLockDuration = int32(duration)
+ }
+ }
+ }
+
bucket := S3Bucket{
- Name: bucketName,
- CreatedAt: time.Unix(resp.Entry.Attributes.Crtime, 0),
- Size: size,
- ObjectCount: objectCount,
- LastModified: time.Unix(resp.Entry.Attributes.Mtime, 0),
- Quota: quota,
- QuotaEnabled: quotaEnabled,
+ Name: bucketName,
+ CreatedAt: time.Unix(resp.Entry.Attributes.Crtime, 0),
+ Size: size,
+ ObjectCount: objectCount,
+ LastModified: time.Unix(resp.Entry.Attributes.Mtime, 0),
+ Quota: quota,
+ QuotaEnabled: quotaEnabled,
+ VersioningEnabled: versioningEnabled,
+ ObjectLockEnabled: objectLockEnabled,
+ ObjectLockMode: objectLockMode,
+ ObjectLockDuration: objectLockDuration,
}
buckets = append(buckets, bucket)
}
@@ -305,6 +341,45 @@ func (s *AdminServer) GetBucketDetails(bucketName string) (*BucketDetails, error
details.Bucket.CreatedAt = time.Unix(bucketResp.Entry.Attributes.Crtime, 0)
details.Bucket.LastModified = time.Unix(bucketResp.Entry.Attributes.Mtime, 0)
+ // Get quota information from entry
+ quota := bucketResp.Entry.Quota
+ quotaEnabled := quota > 0
+ if quota < 0 {
+ // Negative quota means disabled
+ quota = -quota
+ quotaEnabled = false
+ }
+ details.Bucket.Quota = quota
+ details.Bucket.QuotaEnabled = quotaEnabled
+
+ // Get versioning and object lock information from extended attributes
+ versioningEnabled := false
+ objectLockEnabled := false
+ objectLockMode := ""
+ var objectLockDuration int32 = 0
+
+ if bucketResp.Entry.Extended != nil {
+ if versioningBytes, exists := bucketResp.Entry.Extended["s3.versioning"]; exists {
+ versioningEnabled = string(versioningBytes) == "Enabled"
+ }
+ if objectLockBytes, exists := bucketResp.Entry.Extended["s3.objectlock"]; exists {
+ objectLockEnabled = string(objectLockBytes) == "Enabled"
+ }
+ if objectLockModeBytes, exists := bucketResp.Entry.Extended["s3.objectlock.mode"]; exists {
+ objectLockMode = string(objectLockModeBytes)
+ }
+ if objectLockDurationBytes, exists := bucketResp.Entry.Extended["s3.objectlock.duration"]; exists {
+ if duration, err := strconv.ParseInt(string(objectLockDurationBytes), 10, 32); err == nil {
+ objectLockDuration = int32(duration)
+ }
+ }
+ }
+
+ details.Bucket.VersioningEnabled = versioningEnabled
+ details.Bucket.ObjectLockEnabled = objectLockEnabled
+ details.Bucket.ObjectLockMode = objectLockMode
+ details.Bucket.ObjectLockDuration = objectLockDuration
+
// List objects in bucket (recursively)
return s.listBucketObjects(client, bucketPath, "", details)
})
@@ -598,6 +673,48 @@ func (s *AdminServer) GetClusterFilers() (*ClusterFilersData, error) {
}, nil
}
+// GetClusterBrokers retrieves cluster message brokers data
+func (s *AdminServer) GetClusterBrokers() (*ClusterBrokersData, error) {
+ var brokers []MessageBrokerInfo
+
+ // Get broker information from master using ListClusterNodes
+ err := s.WithMasterClient(func(client master_pb.SeaweedClient) error {
+ resp, err := client.ListClusterNodes(context.Background(), &master_pb.ListClusterNodesRequest{
+ ClientType: cluster.BrokerType,
+ })
+ if err != nil {
+ return err
+ }
+
+ // Process each broker node
+ for _, node := range resp.ClusterNodes {
+ createdAt := time.Unix(0, node.CreatedAtNs)
+
+ brokerInfo := MessageBrokerInfo{
+ Address: node.Address,
+ DataCenter: node.DataCenter,
+ Rack: node.Rack,
+ Version: node.Version,
+ CreatedAt: createdAt,
+ }
+
+ brokers = append(brokers, brokerInfo)
+ }
+
+ return nil
+ })
+
+ if err != nil {
+ return nil, fmt.Errorf("failed to get broker nodes from master: %v", err)
+ }
+
+ return &ClusterBrokersData{
+ Brokers: brokers,
+ TotalBrokers: len(brokers),
+ LastUpdated: time.Now(),
+ }, nil
+}
+
// GetAllFilers method moved to client_management.go
// GetVolumeDetails method moved to volume_management.go
@@ -1054,6 +1171,17 @@ func (as *AdminServer) triggerMaintenanceScan() error {
return as.maintenanceManager.TriggerScan()
}
+// TriggerTopicRetentionPurgeAPI triggers topic retention purge via HTTP API
+func (as *AdminServer) TriggerTopicRetentionPurgeAPI(c *gin.Context) {
+ err := as.TriggerTopicRetentionPurge()
+ if err != nil {
+ c.JSON(http.StatusInternalServerError, gin.H{"error": err.Error()})
+ return
+ }
+
+ c.JSON(http.StatusOK, gin.H{"message": "Topic retention purge triggered successfully"})
+}
+
// GetConfigInfo returns information about the admin configuration
func (as *AdminServer) GetConfigInfo(c *gin.Context) {
configInfo := as.configPersistence.GetConfigInfo()
@@ -1184,6 +1312,157 @@ func (s *AdminServer) StopMaintenanceManager() {
}
}
+// TriggerTopicRetentionPurge triggers topic data purging based on retention policies
+func (s *AdminServer) TriggerTopicRetentionPurge() error {
+ if s.topicRetentionPurger == nil {
+ return fmt.Errorf("topic retention purger not initialized")
+ }
+
+ glog.V(0).Infof("Triggering topic retention purge")
+ return s.topicRetentionPurger.PurgeExpiredTopicData()
+}
+
+// GetTopicRetentionPurger returns the topic retention purger
+func (s *AdminServer) GetTopicRetentionPurger() *TopicRetentionPurger {
+ return s.topicRetentionPurger
+}
+
+// CreateTopicWithRetention creates a new topic with optional retention configuration
+func (s *AdminServer) CreateTopicWithRetention(namespace, name string, partitionCount int32, retentionEnabled bool, retentionSeconds int64) error {
+ // Find broker leader to create the topic
+ brokerLeader, err := s.findBrokerLeader()
+ if err != nil {
+ return fmt.Errorf("failed to find broker leader: %v", err)
+ }
+
+ // Create retention configuration
+ var retention *mq_pb.TopicRetention
+ if retentionEnabled {
+ retention = &mq_pb.TopicRetention{
+ Enabled: true,
+ RetentionSeconds: retentionSeconds,
+ }
+ } else {
+ retention = &mq_pb.TopicRetention{
+ Enabled: false,
+ RetentionSeconds: 0,
+ }
+ }
+
+ // Create the topic via broker
+ err = s.withBrokerClient(brokerLeader, func(client mq_pb.SeaweedMessagingClient) error {
+ ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
+ defer cancel()
+
+ _, err := client.ConfigureTopic(ctx, &mq_pb.ConfigureTopicRequest{
+ Topic: &schema_pb.Topic{
+ Namespace: namespace,
+ Name: name,
+ },
+ PartitionCount: partitionCount,
+ Retention: retention,
+ })
+ return err
+ })
+
+ if err != nil {
+ return fmt.Errorf("failed to create topic: %v", err)
+ }
+
+ glog.V(0).Infof("Created topic %s.%s with %d partitions (retention: enabled=%v, seconds=%d)",
+ namespace, name, partitionCount, retentionEnabled, retentionSeconds)
+ return nil
+}
+
+// UpdateTopicRetention updates the retention configuration for an existing topic
+func (s *AdminServer) UpdateTopicRetention(namespace, name string, enabled bool, retentionSeconds int64) error {
+ // Get broker information from master
+ var brokerAddress string
+ err := s.WithMasterClient(func(client master_pb.SeaweedClient) error {
+ resp, err := client.ListClusterNodes(context.Background(), &master_pb.ListClusterNodesRequest{
+ ClientType: cluster.BrokerType,
+ })
+ if err != nil {
+ return err
+ }
+
+ // Find the first available broker
+ for _, node := range resp.ClusterNodes {
+ brokerAddress = node.Address
+ break
+ }
+
+ return nil
+ })
+
+ if err != nil {
+ return fmt.Errorf("failed to get broker nodes from master: %v", err)
+ }
+
+ if brokerAddress == "" {
+ return fmt.Errorf("no active brokers found")
+ }
+
+ // Create gRPC connection
+ conn, err := grpc.Dial(brokerAddress, s.grpcDialOption)
+ if err != nil {
+ return fmt.Errorf("failed to connect to broker: %v", err)
+ }
+ defer conn.Close()
+
+ client := mq_pb.NewSeaweedMessagingClient(conn)
+
+ // First, get the current topic configuration to preserve existing settings
+ ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
+ defer cancel()
+
+ currentConfig, err := client.GetTopicConfiguration(ctx, &mq_pb.GetTopicConfigurationRequest{
+ Topic: &schema_pb.Topic{
+ Namespace: namespace,
+ Name: name,
+ },
+ })
+ if err != nil {
+ return fmt.Errorf("failed to get current topic configuration: %v", err)
+ }
+
+ // Create the topic configuration request, preserving all existing settings
+ configRequest := &mq_pb.ConfigureTopicRequest{
+ Topic: &schema_pb.Topic{
+ Namespace: namespace,
+ Name: name,
+ },
+ // Preserve existing partition count - this is critical!
+ PartitionCount: currentConfig.PartitionCount,
+ // Preserve existing record type if it exists
+ RecordType: currentConfig.RecordType,
+ }
+
+ // Update only the retention configuration
+ if enabled {
+ configRequest.Retention = &mq_pb.TopicRetention{
+ RetentionSeconds: retentionSeconds,
+ Enabled: true,
+ }
+ } else {
+ // Set retention to disabled
+ configRequest.Retention = &mq_pb.TopicRetention{
+ RetentionSeconds: 0,
+ Enabled: false,
+ }
+ }
+
+ // Send the configuration request with preserved settings
+ _, err = client.ConfigureTopic(ctx, configRequest)
+ if err != nil {
+ return fmt.Errorf("failed to update topic retention: %v", err)
+ }
+
+ glog.V(0).Infof("Updated topic %s.%s retention (enabled: %v, seconds: %d) while preserving %d partitions",
+ namespace, name, enabled, retentionSeconds, currentConfig.PartitionCount)
+ return nil
+}
+
// Shutdown gracefully shuts down the admin server
func (s *AdminServer) Shutdown() {
glog.V(1).Infof("Shutting down admin server...")
diff --git a/weed/admin/dash/bucket_management.go b/weed/admin/dash/bucket_management.go
index 7fbd74a1e..09a8449a5 100644
--- a/weed/admin/dash/bucket_management.go
+++ b/weed/admin/dash/bucket_management.go
@@ -22,11 +22,15 @@ type S3BucketsData struct {
}
type CreateBucketRequest struct {
- Name string `json:"name" binding:"required"`
- Region string `json:"region"`
- QuotaSize int64 `json:"quota_size"` // Quota size in bytes
- QuotaUnit string `json:"quota_unit"` // Unit: MB, GB, TB
- QuotaEnabled bool `json:"quota_enabled"` // Whether quota is enabled
+ Name string `json:"name" binding:"required"`
+ Region string `json:"region"`
+ QuotaSize int64 `json:"quota_size"` // Quota size in bytes
+ QuotaUnit string `json:"quota_unit"` // Unit: MB, GB, TB
+ QuotaEnabled bool `json:"quota_enabled"` // Whether quota is enabled
+ VersioningEnabled bool `json:"versioning_enabled"` // Whether versioning is enabled
+ ObjectLockEnabled bool `json:"object_lock_enabled"` // Whether object lock is enabled
+ ObjectLockMode string `json:"object_lock_mode"` // Object lock mode: "GOVERNANCE" or "COMPLIANCE"
+ ObjectLockDuration int32 `json:"object_lock_duration"` // Default retention duration in days
}
// S3 Bucket Management Handlers
@@ -89,21 +93,43 @@ func (s *AdminServer) CreateBucket(c *gin.Context) {
return
}
+ // Validate object lock settings
+ if req.ObjectLockEnabled {
+ // Object lock requires versioning to be enabled
+ req.VersioningEnabled = true
+
+ // Validate object lock mode
+ if req.ObjectLockMode != "GOVERNANCE" && req.ObjectLockMode != "COMPLIANCE" {
+ c.JSON(http.StatusBadRequest, gin.H{"error": "Object lock mode must be either GOVERNANCE or COMPLIANCE"})
+ return
+ }
+
+ // Validate retention duration
+ if req.ObjectLockDuration <= 0 {
+ c.JSON(http.StatusBadRequest, gin.H{"error": "Object lock duration must be greater than 0 days"})
+ return
+ }
+ }
+
// Convert quota to bytes
quotaBytes := convertQuotaToBytes(req.QuotaSize, req.QuotaUnit)
- err := s.CreateS3BucketWithQuota(req.Name, quotaBytes, req.QuotaEnabled)
+ err := s.CreateS3BucketWithObjectLock(req.Name, quotaBytes, req.QuotaEnabled, req.VersioningEnabled, req.ObjectLockEnabled, req.ObjectLockMode, req.ObjectLockDuration)
if err != nil {
c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to create bucket: " + err.Error()})
return
}
c.JSON(http.StatusCreated, gin.H{
- "message": "Bucket created successfully",
- "bucket": req.Name,
- "quota_size": req.QuotaSize,
- "quota_unit": req.QuotaUnit,
- "quota_enabled": req.QuotaEnabled,
+ "message": "Bucket created successfully",
+ "bucket": req.Name,
+ "quota_size": req.QuotaSize,
+ "quota_unit": req.QuotaUnit,
+ "quota_enabled": req.QuotaEnabled,
+ "versioning_enabled": req.VersioningEnabled,
+ "object_lock_enabled": req.ObjectLockEnabled,
+ "object_lock_mode": req.ObjectLockMode,
+ "object_lock_duration": req.ObjectLockDuration,
})
}
@@ -258,6 +284,11 @@ func (s *AdminServer) SetBucketQuota(bucketName string, quotaBytes int64, quotaE
// CreateS3BucketWithQuota creates a new S3 bucket with quota settings
func (s *AdminServer) CreateS3BucketWithQuota(bucketName string, quotaBytes int64, quotaEnabled bool) error {
+ return s.CreateS3BucketWithObjectLock(bucketName, quotaBytes, quotaEnabled, false, false, "", 0)
+}
+
+// CreateS3BucketWithObjectLock creates a new S3 bucket with quota, versioning, and object lock settings
+func (s *AdminServer) CreateS3BucketWithObjectLock(bucketName string, quotaBytes int64, quotaEnabled, versioningEnabled, objectLockEnabled bool, objectLockMode string, objectLockDuration int32) error {
return s.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
// First ensure /buckets directory exists
_, err := client.CreateEntry(context.Background(), &filer_pb.CreateEntryRequest{
@@ -299,21 +330,41 @@ func (s *AdminServer) CreateS3BucketWithQuota(bucketName string, quotaBytes int6
quota = 0
}
+ // Prepare bucket attributes with versioning and object lock metadata
+ attributes := &filer_pb.FuseAttributes{
+ FileMode: uint32(0755 | os.ModeDir), // Directory mode
+ Uid: filer_pb.OS_UID,
+ Gid: filer_pb.OS_GID,
+ Crtime: time.Now().Unix(),
+ Mtime: time.Now().Unix(),
+ TtlSec: 0,
+ }
+
+ // Create extended attributes map for versioning and object lock
+ extended := make(map[string][]byte)
+ if versioningEnabled {
+ extended["s3.versioning"] = []byte("Enabled")
+ } else {
+ extended["s3.versioning"] = []byte("Suspended")
+ }
+
+ if objectLockEnabled {
+ extended["s3.objectlock"] = []byte("Enabled")
+ extended["s3.objectlock.mode"] = []byte(objectLockMode)
+ extended["s3.objectlock.duration"] = []byte(fmt.Sprintf("%d", objectLockDuration))
+ } else {
+ extended["s3.objectlock"] = []byte("Disabled")
+ }
+
// Create bucket directory under /buckets
_, err = client.CreateEntry(context.Background(), &filer_pb.CreateEntryRequest{
Directory: "/buckets",
Entry: &filer_pb.Entry{
Name: bucketName,
IsDirectory: true,
- Attributes: &filer_pb.FuseAttributes{
- FileMode: uint32(0755 | os.ModeDir), // Directory mode
- Uid: filer_pb.OS_UID,
- Gid: filer_pb.OS_GID,
- Crtime: time.Now().Unix(),
- Mtime: time.Now().Unix(),
- TtlSec: 0,
- },
- Quota: quota,
+ Attributes: attributes,
+ Extended: extended,
+ Quota: quota,
},
})
if err != nil {
diff --git a/weed/admin/dash/mq_management.go b/weed/admin/dash/mq_management.go
new file mode 100644
index 000000000..d47547f6b
--- /dev/null
+++ b/weed/admin/dash/mq_management.go
@@ -0,0 +1,615 @@
+package dash
+
+import (
+ "context"
+ "fmt"
+ "io"
+ "path/filepath"
+ "strings"
+ "time"
+
+ "github.com/seaweedfs/seaweedfs/weed/cluster"
+ "github.com/seaweedfs/seaweedfs/weed/filer"
+ "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/mq/topic"
+ "github.com/seaweedfs/seaweedfs/weed/pb"
+ "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
+ "github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
+ "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
+ "github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
+ "github.com/seaweedfs/seaweedfs/weed/util"
+)
+
+// GetTopics retrieves message queue topics data
+func (s *AdminServer) GetTopics() (*TopicsData, error) {
+ var topics []TopicInfo
+
+ // Find broker leader and get topics
+ brokerLeader, err := s.findBrokerLeader()
+ if err != nil {
+ // If no broker leader found, return empty data
+ return &TopicsData{
+ Topics: topics,
+ TotalTopics: len(topics),
+ LastUpdated: time.Now(),
+ }, nil
+ }
+
+ // Connect to broker leader and list topics
+ err = s.withBrokerClient(brokerLeader, func(client mq_pb.SeaweedMessagingClient) error {
+ ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
+ defer cancel()
+
+ resp, err := client.ListTopics(ctx, &mq_pb.ListTopicsRequest{})
+ if err != nil {
+ return err
+ }
+
+ // Convert protobuf topics to TopicInfo - only include available data
+ for _, pbTopic := range resp.Topics {
+ topicInfo := TopicInfo{
+ Name: fmt.Sprintf("%s.%s", pbTopic.Namespace, pbTopic.Name),
+ Partitions: 0, // Will be populated by LookupTopicBrokers call
+ Retention: TopicRetentionInfo{
+ Enabled: false,
+ DisplayValue: 0,
+ DisplayUnit: "days",
+ },
+ }
+
+ // Get topic configuration to get partition count and retention info
+ lookupResp, err := client.LookupTopicBrokers(ctx, &mq_pb.LookupTopicBrokersRequest{
+ Topic: pbTopic,
+ })
+ if err == nil {
+ topicInfo.Partitions = len(lookupResp.BrokerPartitionAssignments)
+ }
+
+ // Get topic configuration for retention information
+ configResp, err := client.GetTopicConfiguration(ctx, &mq_pb.GetTopicConfigurationRequest{
+ Topic: pbTopic,
+ })
+ if err == nil && configResp.Retention != nil {
+ topicInfo.Retention = convertTopicRetention(configResp.Retention)
+ }
+
+ topics = append(topics, topicInfo)
+ }
+
+ return nil
+ })
+
+ if err != nil {
+ // If connection fails, return empty data
+ return &TopicsData{
+ Topics: topics,
+ TotalTopics: len(topics),
+ LastUpdated: time.Now(),
+ }, nil
+ }
+
+ return &TopicsData{
+ Topics: topics,
+ TotalTopics: len(topics),
+ LastUpdated: time.Now(),
+ // Don't include TotalMessages and TotalSize as they're not available
+ }, nil
+}
+
+// GetSubscribers retrieves message queue subscribers data
+func (s *AdminServer) GetSubscribers() (*SubscribersData, error) {
+ var subscribers []SubscriberInfo
+
+ // Find broker leader and get subscriber info from broker stats
+ brokerLeader, err := s.findBrokerLeader()
+ if err != nil {
+ // If no broker leader found, return empty data
+ return &SubscribersData{
+ Subscribers: subscribers,
+ TotalSubscribers: len(subscribers),
+ ActiveSubscribers: 0,
+ LastUpdated: time.Now(),
+ }, nil
+ }
+
+ // Connect to broker leader and get subscriber information
+ // Note: SeaweedMQ doesn't have a direct API to list all subscribers
+ // We would need to collect this information from broker statistics
+ // For now, return empty data structure as subscriber info is not
+ // directly available through the current MQ API
+ err = s.withBrokerClient(brokerLeader, func(client mq_pb.SeaweedMessagingClient) error {
+ // TODO: Implement subscriber data collection from broker statistics
+ // This would require access to broker internal statistics about
+ // active subscribers, consumer groups, etc.
+ return nil
+ })
+
+ if err != nil {
+ // If connection fails, return empty data
+ return &SubscribersData{
+ Subscribers: subscribers,
+ TotalSubscribers: len(subscribers),
+ ActiveSubscribers: 0,
+ LastUpdated: time.Now(),
+ }, nil
+ }
+
+ activeCount := 0
+ for _, sub := range subscribers {
+ if sub.Status == "active" {
+ activeCount++
+ }
+ }
+
+ return &SubscribersData{
+ Subscribers: subscribers,
+ TotalSubscribers: len(subscribers),
+ ActiveSubscribers: activeCount,
+ LastUpdated: time.Now(),
+ }, nil
+}
+
+// GetTopicDetails retrieves detailed information about a specific topic
+func (s *AdminServer) GetTopicDetails(namespace, topicName string) (*TopicDetailsData, error) {
+ // Find broker leader
+ brokerLeader, err := s.findBrokerLeader()
+ if err != nil {
+ return nil, fmt.Errorf("failed to find broker leader: %v", err)
+ }
+
+ var topicDetails *TopicDetailsData
+
+ // Connect to broker leader and get topic configuration
+ err = s.withBrokerClient(brokerLeader, func(client mq_pb.SeaweedMessagingClient) error {
+ ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
+ defer cancel()
+
+ // Get topic configuration using the new API
+ configResp, err := client.GetTopicConfiguration(ctx, &mq_pb.GetTopicConfigurationRequest{
+ Topic: &schema_pb.Topic{
+ Namespace: namespace,
+ Name: topicName,
+ },
+ })
+ if err != nil {
+ return fmt.Errorf("failed to get topic configuration: %v", err)
+ }
+
+ // Initialize topic details
+ topicDetails = &TopicDetailsData{
+ TopicName: fmt.Sprintf("%s.%s", namespace, topicName),
+ Namespace: namespace,
+ Name: topicName,
+ Partitions: []PartitionInfo{},
+ Schema: []SchemaFieldInfo{},
+ Publishers: []PublisherInfo{},
+ Subscribers: []TopicSubscriberInfo{},
+ ConsumerGroupOffsets: []ConsumerGroupOffsetInfo{},
+ Retention: convertTopicRetention(configResp.Retention),
+ CreatedAt: time.Unix(0, configResp.CreatedAtNs),
+ LastUpdated: time.Unix(0, configResp.LastUpdatedNs),
+ }
+
+ // Set current time if timestamps are not available
+ if configResp.CreatedAtNs == 0 {
+ topicDetails.CreatedAt = time.Now()
+ }
+ if configResp.LastUpdatedNs == 0 {
+ topicDetails.LastUpdated = time.Now()
+ }
+
+ // Process partitions
+ for _, assignment := range configResp.BrokerPartitionAssignments {
+ if assignment.Partition != nil {
+ partitionInfo := PartitionInfo{
+ ID: assignment.Partition.RangeStart,
+ LeaderBroker: assignment.LeaderBroker,
+ FollowerBroker: assignment.FollowerBroker,
+ MessageCount: 0, // Will be enhanced later with actual stats
+ TotalSize: 0, // Will be enhanced later with actual stats
+ LastDataTime: time.Time{}, // Will be enhanced later
+ CreatedAt: time.Now(),
+ }
+ topicDetails.Partitions = append(topicDetails.Partitions, partitionInfo)
+ }
+ }
+
+ // Process schema from RecordType
+ if configResp.RecordType != nil {
+ topicDetails.Schema = convertRecordTypeToSchemaFields(configResp.RecordType)
+ }
+
+ // Get publishers information
+ publishersResp, err := client.GetTopicPublishers(ctx, &mq_pb.GetTopicPublishersRequest{
+ Topic: &schema_pb.Topic{
+ Namespace: namespace,
+ Name: topicName,
+ },
+ })
+ if err != nil {
+ // Log error but don't fail the entire request
+ glog.V(0).Infof("failed to get topic publishers for %s.%s: %v", namespace, topicName, err)
+ } else {
+ glog.V(1).Infof("got %d publishers for topic %s.%s", len(publishersResp.Publishers), namespace, topicName)
+ topicDetails.Publishers = convertTopicPublishers(publishersResp.Publishers)
+ }
+
+ // Get subscribers information
+ subscribersResp, err := client.GetTopicSubscribers(ctx, &mq_pb.GetTopicSubscribersRequest{
+ Topic: &schema_pb.Topic{
+ Namespace: namespace,
+ Name: topicName,
+ },
+ })
+ if err != nil {
+ // Log error but don't fail the entire request
+ glog.V(0).Infof("failed to get topic subscribers for %s.%s: %v", namespace, topicName, err)
+ } else {
+ glog.V(1).Infof("got %d subscribers for topic %s.%s", len(subscribersResp.Subscribers), namespace, topicName)
+ topicDetails.Subscribers = convertTopicSubscribers(subscribersResp.Subscribers)
+ }
+
+ return nil
+ })
+
+ if err != nil {
+ return nil, err
+ }
+
+ // Get consumer group offsets from the filer
+ offsets, err := s.GetConsumerGroupOffsets(namespace, topicName)
+ if err != nil {
+ // Log error but don't fail the entire request
+ glog.V(0).Infof("failed to get consumer group offsets for %s.%s: %v", namespace, topicName, err)
+ } else {
+ glog.V(1).Infof("got %d consumer group offsets for topic %s.%s", len(offsets), namespace, topicName)
+ topicDetails.ConsumerGroupOffsets = offsets
+ }
+
+ return topicDetails, nil
+}
+
+// GetConsumerGroupOffsets retrieves consumer group offsets for a topic from the filer
+func (s *AdminServer) GetConsumerGroupOffsets(namespace, topicName string) ([]ConsumerGroupOffsetInfo, error) {
+ var offsets []ConsumerGroupOffsetInfo
+
+ err := s.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
+ // Get the topic directory: /topics/namespace/topicName
+ topicObj := topic.NewTopic(namespace, topicName)
+ topicDir := topicObj.Dir()
+
+ // List all version directories under the topic directory (e.g., v2025-07-10-05-44-34)
+ versionStream, err := client.ListEntries(context.Background(), &filer_pb.ListEntriesRequest{
+ Directory: topicDir,
+ Prefix: "",
+ StartFromFileName: "",
+ InclusiveStartFrom: false,
+ Limit: 1000,
+ })
+ if err != nil {
+ return fmt.Errorf("failed to list topic directory %s: %v", topicDir, err)
+ }
+
+ // Process each version directory
+ for {
+ versionResp, err := versionStream.Recv()
+ if err != nil {
+ if err == io.EOF {
+ break
+ }
+ return fmt.Errorf("failed to receive version entries: %v", err)
+ }
+
+ // Only process directories that are versions (start with "v")
+ if versionResp.Entry.IsDirectory && strings.HasPrefix(versionResp.Entry.Name, "v") {
+ versionDir := filepath.Join(topicDir, versionResp.Entry.Name)
+
+ // List all partition directories under the version directory (e.g., 0315-0630)
+ partitionStream, err := client.ListEntries(context.Background(), &filer_pb.ListEntriesRequest{
+ Directory: versionDir,
+ Prefix: "",
+ StartFromFileName: "",
+ InclusiveStartFrom: false,
+ Limit: 1000,
+ })
+ if err != nil {
+ glog.Warningf("Failed to list version directory %s: %v", versionDir, err)
+ continue
+ }
+
+ // Process each partition directory
+ for {
+ partitionResp, err := partitionStream.Recv()
+ if err != nil {
+ if err == io.EOF {
+ break
+ }
+ glog.Warningf("Failed to receive partition entries: %v", err)
+ break
+ }
+
+ // Only process directories that are partitions (format: NNNN-NNNN)
+ if partitionResp.Entry.IsDirectory {
+ // Parse partition range to get partition start ID (e.g., "0315-0630" -> 315)
+ var partitionStart, partitionStop int32
+ if n, err := fmt.Sscanf(partitionResp.Entry.Name, "%04d-%04d", &partitionStart, &partitionStop); n != 2 || err != nil {
+ // Skip directories that don't match the partition format
+ continue
+ }
+
+ partitionDir := filepath.Join(versionDir, partitionResp.Entry.Name)
+
+ // List all .offset files in this partition directory
+ offsetStream, err := client.ListEntries(context.Background(), &filer_pb.ListEntriesRequest{
+ Directory: partitionDir,
+ Prefix: "",
+ StartFromFileName: "",
+ InclusiveStartFrom: false,
+ Limit: 1000,
+ })
+ if err != nil {
+ glog.Warningf("Failed to list partition directory %s: %v", partitionDir, err)
+ continue
+ }
+
+ // Process each offset file
+ for {
+ offsetResp, err := offsetStream.Recv()
+ if err != nil {
+ if err == io.EOF {
+ break
+ }
+ glog.Warningf("Failed to receive offset entries: %v", err)
+ break
+ }
+
+ // Only process .offset files
+ if !offsetResp.Entry.IsDirectory && strings.HasSuffix(offsetResp.Entry.Name, ".offset") {
+ consumerGroup := strings.TrimSuffix(offsetResp.Entry.Name, ".offset")
+
+ // Read the offset value from the file
+ offsetData, err := filer.ReadInsideFiler(client, partitionDir, offsetResp.Entry.Name)
+ if err != nil {
+ glog.Warningf("Failed to read offset file %s: %v", offsetResp.Entry.Name, err)
+ continue
+ }
+
+ if len(offsetData) == 8 {
+ offset := int64(util.BytesToUint64(offsetData))
+
+ // Get the file modification time
+ lastUpdated := time.Unix(offsetResp.Entry.Attributes.Mtime, 0)
+
+ offsets = append(offsets, ConsumerGroupOffsetInfo{
+ ConsumerGroup: consumerGroup,
+ PartitionID: partitionStart, // Use partition start as the ID
+ Offset: offset,
+ LastUpdated: lastUpdated,
+ })
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+
+ return nil
+ })
+
+ if err != nil {
+ return nil, fmt.Errorf("failed to get consumer group offsets: %v", err)
+ }
+
+ return offsets, nil
+}
+
+// convertRecordTypeToSchemaFields converts a protobuf RecordType to SchemaFieldInfo slice
+func convertRecordTypeToSchemaFields(recordType *schema_pb.RecordType) []SchemaFieldInfo {
+ var schemaFields []SchemaFieldInfo
+
+ if recordType == nil || recordType.Fields == nil {
+ return schemaFields
+ }
+
+ for _, field := range recordType.Fields {
+ schemaField := SchemaFieldInfo{
+ Name: field.Name,
+ Type: getFieldTypeString(field.Type),
+ Required: field.IsRequired,
+ }
+ schemaFields = append(schemaFields, schemaField)
+ }
+
+ return schemaFields
+}
+
+// getFieldTypeString converts a protobuf Type to a human-readable string
+func getFieldTypeString(fieldType *schema_pb.Type) string {
+ if fieldType == nil {
+ return "unknown"
+ }
+
+ switch kind := fieldType.Kind.(type) {
+ case *schema_pb.Type_ScalarType:
+ return getScalarTypeString(kind.ScalarType)
+ case *schema_pb.Type_RecordType:
+ return "record"
+ case *schema_pb.Type_ListType:
+ elementType := getFieldTypeString(kind.ListType.ElementType)
+ return fmt.Sprintf("list<%s>", elementType)
+ default:
+ return "unknown"
+ }
+}
+
+// getScalarTypeString converts a protobuf ScalarType to a string
+func getScalarTypeString(scalarType schema_pb.ScalarType) string {
+ switch scalarType {
+ case schema_pb.ScalarType_BOOL:
+ return "bool"
+ case schema_pb.ScalarType_INT32:
+ return "int32"
+ case schema_pb.ScalarType_INT64:
+ return "int64"
+ case schema_pb.ScalarType_FLOAT:
+ return "float"
+ case schema_pb.ScalarType_DOUBLE:
+ return "double"
+ case schema_pb.ScalarType_BYTES:
+ return "bytes"
+ case schema_pb.ScalarType_STRING:
+ return "string"
+ default:
+ return "unknown"
+ }
+}
+
+// convertTopicPublishers converts protobuf TopicPublisher slice to PublisherInfo slice
+func convertTopicPublishers(publishers []*mq_pb.TopicPublisher) []PublisherInfo {
+ publisherInfos := make([]PublisherInfo, 0, len(publishers))
+
+ for _, publisher := range publishers {
+ publisherInfo := PublisherInfo{
+ PublisherName: publisher.PublisherName,
+ ClientID: publisher.ClientId,
+ PartitionID: publisher.Partition.RangeStart,
+ Broker: publisher.Broker,
+ IsActive: publisher.IsActive,
+ LastPublishedOffset: publisher.LastPublishedOffset,
+ LastAckedOffset: publisher.LastAckedOffset,
+ }
+
+ // Convert timestamps
+ if publisher.ConnectTimeNs > 0 {
+ publisherInfo.ConnectTime = time.Unix(0, publisher.ConnectTimeNs)
+ }
+ if publisher.LastSeenTimeNs > 0 {
+ publisherInfo.LastSeenTime = time.Unix(0, publisher.LastSeenTimeNs)
+ }
+
+ publisherInfos = append(publisherInfos, publisherInfo)
+ }
+
+ return publisherInfos
+}
+
+// convertTopicSubscribers converts protobuf TopicSubscriber slice to TopicSubscriberInfo slice
+func convertTopicSubscribers(subscribers []*mq_pb.TopicSubscriber) []TopicSubscriberInfo {
+ subscriberInfos := make([]TopicSubscriberInfo, 0, len(subscribers))
+
+ for _, subscriber := range subscribers {
+ subscriberInfo := TopicSubscriberInfo{
+ ConsumerGroup: subscriber.ConsumerGroup,
+ ConsumerID: subscriber.ConsumerId,
+ ClientID: subscriber.ClientId,
+ PartitionID: subscriber.Partition.RangeStart,
+ Broker: subscriber.Broker,
+ IsActive: subscriber.IsActive,
+ CurrentOffset: subscriber.CurrentOffset,
+ LastReceivedOffset: subscriber.LastReceivedOffset,
+ }
+
+ // Convert timestamps
+ if subscriber.ConnectTimeNs > 0 {
+ subscriberInfo.ConnectTime = time.Unix(0, subscriber.ConnectTimeNs)
+ }
+ if subscriber.LastSeenTimeNs > 0 {
+ subscriberInfo.LastSeenTime = time.Unix(0, subscriber.LastSeenTimeNs)
+ }
+
+ subscriberInfos = append(subscriberInfos, subscriberInfo)
+ }
+
+ return subscriberInfos
+}
+
+// findBrokerLeader finds the current broker leader
+func (s *AdminServer) findBrokerLeader() (string, error) {
+ // First, try to find any broker from the cluster
+ var brokers []string
+ err := s.WithMasterClient(func(client master_pb.SeaweedClient) error {
+ resp, err := client.ListClusterNodes(context.Background(), &master_pb.ListClusterNodesRequest{
+ ClientType: cluster.BrokerType,
+ })
+ if err != nil {
+ return err
+ }
+
+ for _, node := range resp.ClusterNodes {
+ brokers = append(brokers, node.Address)
+ }
+
+ return nil
+ })
+
+ if err != nil {
+ return "", fmt.Errorf("failed to list brokers: %v", err)
+ }
+
+ if len(brokers) == 0 {
+ return "", fmt.Errorf("no brokers found in cluster")
+ }
+
+ // Try each broker to find the leader
+ for _, brokerAddr := range brokers {
+ err := s.withBrokerClient(brokerAddr, func(client mq_pb.SeaweedMessagingClient) error {
+ ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second)
+ defer cancel()
+
+ // Try to find broker leader
+ _, err := client.FindBrokerLeader(ctx, &mq_pb.FindBrokerLeaderRequest{
+ FilerGroup: "",
+ })
+ if err == nil {
+ return nil // This broker is the leader
+ }
+ return err
+ })
+ if err == nil {
+ return brokerAddr, nil
+ }
+ }
+
+ return "", fmt.Errorf("no broker leader found")
+}
+
+// withBrokerClient connects to a message queue broker and executes a function
+func (s *AdminServer) withBrokerClient(brokerAddress string, fn func(client mq_pb.SeaweedMessagingClient) error) error {
+ return pb.WithBrokerGrpcClient(false, brokerAddress, s.grpcDialOption, fn)
+}
+
+// convertTopicRetention converts protobuf retention to TopicRetentionInfo
+func convertTopicRetention(retention *mq_pb.TopicRetention) TopicRetentionInfo {
+ if retention == nil || !retention.Enabled {
+ return TopicRetentionInfo{
+ Enabled: false,
+ RetentionSeconds: 0,
+ DisplayValue: 0,
+ DisplayUnit: "days",
+ }
+ }
+
+ // Convert seconds to human-readable format
+ seconds := retention.RetentionSeconds
+ var displayValue int32
+ var displayUnit string
+
+ if seconds >= 86400 { // >= 1 day
+ displayValue = int32(seconds / 86400)
+ displayUnit = "days"
+ } else if seconds >= 3600 { // >= 1 hour
+ displayValue = int32(seconds / 3600)
+ displayUnit = "hours"
+ } else {
+ displayValue = int32(seconds)
+ displayUnit = "seconds"
+ }
+
+ return TopicRetentionInfo{
+ Enabled: retention.Enabled,
+ RetentionSeconds: seconds,
+ DisplayValue: displayValue,
+ DisplayUnit: displayUnit,
+ }
+}
diff --git a/weed/admin/dash/topic_retention.go b/weed/admin/dash/topic_retention.go
new file mode 100644
index 000000000..06a9e9ad6
--- /dev/null
+++ b/weed/admin/dash/topic_retention.go
@@ -0,0 +1,296 @@
+package dash
+
+import (
+ "context"
+ "fmt"
+ "io"
+ "path/filepath"
+ "sort"
+ "strings"
+ "time"
+
+ "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/mq/topic"
+ "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
+ "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
+)
+
+// TopicRetentionPurger handles topic data purging based on retention policies
+type TopicRetentionPurger struct {
+ adminServer *AdminServer
+}
+
+// NewTopicRetentionPurger creates a new topic retention purger
+func NewTopicRetentionPurger(adminServer *AdminServer) *TopicRetentionPurger {
+ return &TopicRetentionPurger{
+ adminServer: adminServer,
+ }
+}
+
+// PurgeExpiredTopicData purges expired topic data based on retention policies
+func (p *TopicRetentionPurger) PurgeExpiredTopicData() error {
+ glog.V(1).Infof("Starting topic data purge based on retention policies")
+
+ // Get all topics with retention enabled
+ topics, err := p.getTopicsWithRetention()
+ if err != nil {
+ return fmt.Errorf("failed to get topics with retention: %v", err)
+ }
+
+ glog.V(1).Infof("Found %d topics with retention enabled", len(topics))
+
+ // Process each topic
+ for _, topicRetention := range topics {
+ err := p.purgeTopicData(topicRetention)
+ if err != nil {
+ glog.Errorf("Failed to purge data for topic %s: %v", topicRetention.TopicName, err)
+ continue
+ }
+ }
+
+ glog.V(1).Infof("Completed topic data purge")
+ return nil
+}
+
+// TopicRetentionConfig represents a topic with its retention configuration
+type TopicRetentionConfig struct {
+ TopicName string
+ Namespace string
+ Name string
+ RetentionSeconds int64
+}
+
+// getTopicsWithRetention retrieves all topics that have retention enabled
+func (p *TopicRetentionPurger) getTopicsWithRetention() ([]TopicRetentionConfig, error) {
+ var topicsWithRetention []TopicRetentionConfig
+
+ // Find broker leader to get topics
+ brokerLeader, err := p.adminServer.findBrokerLeader()
+ if err != nil {
+ return nil, fmt.Errorf("failed to find broker leader: %v", err)
+ }
+
+ // Get all topics from the broker
+ err = p.adminServer.withBrokerClient(brokerLeader, func(client mq_pb.SeaweedMessagingClient) error {
+ ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
+ defer cancel()
+
+ resp, err := client.ListTopics(ctx, &mq_pb.ListTopicsRequest{})
+ if err != nil {
+ return err
+ }
+
+ // Check each topic for retention configuration
+ for _, pbTopic := range resp.Topics {
+ configResp, err := client.GetTopicConfiguration(ctx, &mq_pb.GetTopicConfigurationRequest{
+ Topic: pbTopic,
+ })
+ if err != nil {
+ glog.Warningf("Failed to get configuration for topic %s.%s: %v", pbTopic.Namespace, pbTopic.Name, err)
+ continue
+ }
+
+ // Check if retention is enabled
+ if configResp.Retention != nil && configResp.Retention.Enabled && configResp.Retention.RetentionSeconds > 0 {
+ topicRetention := TopicRetentionConfig{
+ TopicName: fmt.Sprintf("%s.%s", pbTopic.Namespace, pbTopic.Name),
+ Namespace: pbTopic.Namespace,
+ Name: pbTopic.Name,
+ RetentionSeconds: configResp.Retention.RetentionSeconds,
+ }
+ topicsWithRetention = append(topicsWithRetention, topicRetention)
+ }
+ }
+
+ return nil
+ })
+
+ if err != nil {
+ return nil, err
+ }
+
+ return topicsWithRetention, nil
+}
+
+// purgeTopicData purges expired data for a specific topic
+func (p *TopicRetentionPurger) purgeTopicData(topicRetention TopicRetentionConfig) error {
+ glog.V(1).Infof("Purging expired data for topic %s with retention %d seconds", topicRetention.TopicName, topicRetention.RetentionSeconds)
+
+ // Calculate cutoff time
+ cutoffTime := time.Now().Add(-time.Duration(topicRetention.RetentionSeconds) * time.Second)
+
+ // Get topic directory
+ topicObj := topic.NewTopic(topicRetention.Namespace, topicRetention.Name)
+ topicDir := topicObj.Dir()
+
+ var purgedDirs []string
+
+ err := p.adminServer.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
+ // List all version directories under the topic directory
+ versionStream, err := client.ListEntries(context.Background(), &filer_pb.ListEntriesRequest{
+ Directory: topicDir,
+ Prefix: "",
+ StartFromFileName: "",
+ InclusiveStartFrom: false,
+ Limit: 1000,
+ })
+ if err != nil {
+ return fmt.Errorf("failed to list topic directory %s: %v", topicDir, err)
+ }
+
+ var versionDirs []VersionDirInfo
+
+ // Collect all version directories
+ for {
+ versionResp, err := versionStream.Recv()
+ if err != nil {
+ if err == io.EOF {
+ break
+ }
+ return fmt.Errorf("failed to receive version entries: %v", err)
+ }
+
+ // Only process directories that are versions (start with "v")
+ if versionResp.Entry.IsDirectory && strings.HasPrefix(versionResp.Entry.Name, "v") {
+ versionTime, err := p.parseVersionTime(versionResp.Entry.Name)
+ if err != nil {
+ glog.Warningf("Failed to parse version time from %s: %v", versionResp.Entry.Name, err)
+ continue
+ }
+
+ versionDirs = append(versionDirs, VersionDirInfo{
+ Name: versionResp.Entry.Name,
+ VersionTime: versionTime,
+ ModTime: time.Unix(versionResp.Entry.Attributes.Mtime, 0),
+ })
+ }
+ }
+
+ // Sort version directories by time (oldest first)
+ sort.Slice(versionDirs, func(i, j int) bool {
+ return versionDirs[i].VersionTime.Before(versionDirs[j].VersionTime)
+ })
+
+ // Keep at least the most recent version directory, even if it's expired
+ if len(versionDirs) <= 1 {
+ glog.V(1).Infof("Topic %s has %d version directories, keeping all", topicRetention.TopicName, len(versionDirs))
+ return nil
+ }
+
+ // Purge expired directories (keep the most recent one)
+ for i := 0; i < len(versionDirs)-1; i++ {
+ versionDir := versionDirs[i]
+
+ // Check if this version directory is expired
+ if versionDir.VersionTime.Before(cutoffTime) {
+ dirPath := filepath.Join(topicDir, versionDir.Name)
+
+ // Delete the entire version directory
+ err := p.deleteDirectoryRecursively(client, dirPath)
+ if err != nil {
+ glog.Errorf("Failed to delete expired directory %s: %v", dirPath, err)
+ } else {
+ purgedDirs = append(purgedDirs, dirPath)
+ glog.V(1).Infof("Purged expired directory: %s (created: %s)", dirPath, versionDir.VersionTime.Format("2006-01-02 15:04:05"))
+ }
+ }
+ }
+
+ return nil
+ })
+
+ if err != nil {
+ return err
+ }
+
+ if len(purgedDirs) > 0 {
+ glog.V(0).Infof("Purged %d expired directories for topic %s", len(purgedDirs), topicRetention.TopicName)
+ }
+
+ return nil
+}
+
+// VersionDirInfo represents a version directory with its timestamp
+type VersionDirInfo struct {
+ Name string
+ VersionTime time.Time
+ ModTime time.Time
+}
+
+// parseVersionTime parses the version directory name to extract the timestamp
+// Version format: v2025-01-10-05-44-34
+func (p *TopicRetentionPurger) parseVersionTime(versionName string) (time.Time, error) {
+ // Remove the 'v' prefix
+ if !strings.HasPrefix(versionName, "v") {
+ return time.Time{}, fmt.Errorf("invalid version format: %s", versionName)
+ }
+
+ timeStr := versionName[1:] // Remove 'v'
+
+ // Parse the time format: 2025-01-10-05-44-34
+ versionTime, err := time.Parse("2006-01-02-15-04-05", timeStr)
+ if err != nil {
+ return time.Time{}, fmt.Errorf("failed to parse version time %s: %v", timeStr, err)
+ }
+
+ return versionTime, nil
+}
+
+// deleteDirectoryRecursively deletes a directory and all its contents
+func (p *TopicRetentionPurger) deleteDirectoryRecursively(client filer_pb.SeaweedFilerClient, dirPath string) error {
+ // List all entries in the directory
+ stream, err := client.ListEntries(context.Background(), &filer_pb.ListEntriesRequest{
+ Directory: dirPath,
+ Prefix: "",
+ StartFromFileName: "",
+ InclusiveStartFrom: false,
+ Limit: 1000,
+ })
+ if err != nil {
+ return fmt.Errorf("failed to list directory %s: %v", dirPath, err)
+ }
+
+ // Delete all entries
+ for {
+ resp, err := stream.Recv()
+ if err != nil {
+ if err == io.EOF {
+ break
+ }
+ return fmt.Errorf("failed to receive entries: %v", err)
+ }
+
+ entryPath := filepath.Join(dirPath, resp.Entry.Name)
+
+ if resp.Entry.IsDirectory {
+ // Recursively delete subdirectory
+ err = p.deleteDirectoryRecursively(client, entryPath)
+ if err != nil {
+ return fmt.Errorf("failed to delete subdirectory %s: %v", entryPath, err)
+ }
+ } else {
+ // Delete file
+ _, err = client.DeleteEntry(context.Background(), &filer_pb.DeleteEntryRequest{
+ Directory: dirPath,
+ Name: resp.Entry.Name,
+ })
+ if err != nil {
+ return fmt.Errorf("failed to delete file %s: %v", entryPath, err)
+ }
+ }
+ }
+
+ // Delete the directory itself
+ parentDir := filepath.Dir(dirPath)
+ dirName := filepath.Base(dirPath)
+
+ _, err = client.DeleteEntry(context.Background(), &filer_pb.DeleteEntryRequest{
+ Directory: parentDir,
+ Name: dirName,
+ })
+ if err != nil {
+ return fmt.Errorf("failed to delete directory %s: %v", dirPath, err)
+ }
+
+ return nil
+}
diff --git a/weed/admin/dash/types.go b/weed/admin/dash/types.go
index 07157d9dc..60f499229 100644
--- a/weed/admin/dash/types.go
+++ b/weed/admin/dash/types.go
@@ -48,13 +48,17 @@ type VolumeServer struct {
// S3 Bucket management structures
type S3Bucket struct {
- Name string `json:"name"`
- CreatedAt time.Time `json:"created_at"`
- Size int64 `json:"size"`
- ObjectCount int64 `json:"object_count"`
- LastModified time.Time `json:"last_modified"`
- Quota int64 `json:"quota"` // Quota in bytes, 0 means no quota
- QuotaEnabled bool `json:"quota_enabled"` // Whether quota is enabled
+ Name string `json:"name"`
+ CreatedAt time.Time `json:"created_at"`
+ Size int64 `json:"size"`
+ ObjectCount int64 `json:"object_count"`
+ LastModified time.Time `json:"last_modified"`
+ Quota int64 `json:"quota"` // Quota in bytes, 0 means no quota
+ QuotaEnabled bool `json:"quota_enabled"` // Whether quota is enabled
+ VersioningEnabled bool `json:"versioning_enabled"` // Whether versioning is enabled
+ ObjectLockEnabled bool `json:"object_lock_enabled"` // Whether object lock is enabled
+ ObjectLockMode string `json:"object_lock_mode"` // Object lock mode: "GOVERNANCE" or "COMPLIANCE"
+ ObjectLockDuration int32 `json:"object_lock_duration"` // Default retention duration in days
}
type S3Object struct {
@@ -189,6 +193,132 @@ type ClusterFilersData struct {
LastUpdated time.Time `json:"last_updated"`
}
+type MessageBrokerInfo struct {
+ Address string `json:"address"`
+ DataCenter string `json:"datacenter"`
+ Rack string `json:"rack"`
+ Version string `json:"version"`
+ CreatedAt time.Time `json:"created_at"`
+}
+
+type ClusterBrokersData struct {
+ Username string `json:"username"`
+ Brokers []MessageBrokerInfo `json:"brokers"`
+ TotalBrokers int `json:"total_brokers"`
+ LastUpdated time.Time `json:"last_updated"`
+}
+
+type TopicInfo struct {
+ Name string `json:"name"`
+ Partitions int `json:"partitions"`
+ Subscribers int `json:"subscribers"`
+ MessageCount int64 `json:"message_count"`
+ TotalSize int64 `json:"total_size"`
+ LastMessage time.Time `json:"last_message"`
+ CreatedAt time.Time `json:"created_at"`
+ Retention TopicRetentionInfo `json:"retention"`
+}
+
+type TopicsData struct {
+ Username string `json:"username"`
+ Topics []TopicInfo `json:"topics"`
+ TotalTopics int `json:"total_topics"`
+ TotalMessages int64 `json:"total_messages"`
+ TotalSize int64 `json:"total_size"`
+ LastUpdated time.Time `json:"last_updated"`
+}
+
+type SubscriberInfo struct {
+ Name string `json:"name"`
+ Topic string `json:"topic"`
+ ConsumerGroup string `json:"consumer_group"`
+ Status string `json:"status"`
+ LastSeen time.Time `json:"last_seen"`
+ MessageCount int64 `json:"message_count"`
+ CreatedAt time.Time `json:"created_at"`
+}
+
+type SubscribersData struct {
+ Username string `json:"username"`
+ Subscribers []SubscriberInfo `json:"subscribers"`
+ TotalSubscribers int `json:"total_subscribers"`
+ ActiveSubscribers int `json:"active_subscribers"`
+ LastUpdated time.Time `json:"last_updated"`
+}
+
+// Topic Details structures
+type PartitionInfo struct {
+ ID int32 `json:"id"`
+ LeaderBroker string `json:"leader_broker"`
+ FollowerBroker string `json:"follower_broker"`
+ MessageCount int64 `json:"message_count"`
+ TotalSize int64 `json:"total_size"`
+ LastDataTime time.Time `json:"last_data_time"`
+ CreatedAt time.Time `json:"created_at"`
+}
+
+type SchemaFieldInfo struct {
+ Name string `json:"name"`
+ Type string `json:"type"`
+ Required bool `json:"required"`
+}
+
+type PublisherInfo struct {
+ PublisherName string `json:"publisher_name"`
+ ClientID string `json:"client_id"`
+ PartitionID int32 `json:"partition_id"`
+ Broker string `json:"broker"`
+ ConnectTime time.Time `json:"connect_time"`
+ LastSeenTime time.Time `json:"last_seen_time"`
+ IsActive bool `json:"is_active"`
+ LastPublishedOffset int64 `json:"last_published_offset"`
+ LastAckedOffset int64 `json:"last_acked_offset"`
+}
+
+type TopicSubscriberInfo struct {
+ ConsumerGroup string `json:"consumer_group"`
+ ConsumerID string `json:"consumer_id"`
+ ClientID string `json:"client_id"`
+ PartitionID int32 `json:"partition_id"`
+ Broker string `json:"broker"`
+ ConnectTime time.Time `json:"connect_time"`
+ LastSeenTime time.Time `json:"last_seen_time"`
+ IsActive bool `json:"is_active"`
+ CurrentOffset int64 `json:"current_offset"` // last acknowledged offset
+ LastReceivedOffset int64 `json:"last_received_offset"` // last received offset
+}
+
+type ConsumerGroupOffsetInfo struct {
+ ConsumerGroup string `json:"consumer_group"`
+ PartitionID int32 `json:"partition_id"`
+ Offset int64 `json:"offset"`
+ LastUpdated time.Time `json:"last_updated"`
+}
+
+type TopicRetentionInfo struct {
+ Enabled bool `json:"enabled"`
+ RetentionSeconds int64 `json:"retention_seconds"`
+ DisplayValue int32 `json:"display_value"` // for UI rendering
+ DisplayUnit string `json:"display_unit"` // for UI rendering
+}
+
+type TopicDetailsData struct {
+ Username string `json:"username"`
+ TopicName string `json:"topic_name"`
+ Namespace string `json:"namespace"`
+ Name string `json:"name"`
+ Partitions []PartitionInfo `json:"partitions"`
+ Schema []SchemaFieldInfo `json:"schema"`
+ Publishers []PublisherInfo `json:"publishers"`
+ Subscribers []TopicSubscriberInfo `json:"subscribers"`
+ ConsumerGroupOffsets []ConsumerGroupOffsetInfo `json:"consumer_group_offsets"`
+ Retention TopicRetentionInfo `json:"retention"`
+ MessageCount int64 `json:"message_count"`
+ TotalSize int64 `json:"total_size"`
+ CreatedAt time.Time `json:"created_at"`
+ LastUpdated time.Time `json:"last_updated"`
+}
+
// Volume server management structures
type ClusterVolumeServersData struct {
Username string `json:"username"`
diff --git a/weed/admin/handlers/admin_handlers.go b/weed/admin/handlers/admin_handlers.go
index 03d156d08..dc7905bc1 100644
--- a/weed/admin/handlers/admin_handlers.go
+++ b/weed/admin/handlers/admin_handlers.go
@@ -18,6 +18,7 @@ type AdminHandlers struct {
fileBrowserHandlers *FileBrowserHandlers
userHandlers *UserHandlers
maintenanceHandlers *MaintenanceHandlers
+ mqHandlers *MessageQueueHandlers
}
// NewAdminHandlers creates a new instance of AdminHandlers
@@ -27,6 +28,7 @@ func NewAdminHandlers(adminServer *dash.AdminServer) *AdminHandlers {
fileBrowserHandlers := NewFileBrowserHandlers(adminServer)
userHandlers := NewUserHandlers(adminServer)
maintenanceHandlers := NewMaintenanceHandlers(adminServer)
+ mqHandlers := NewMessageQueueHandlers(adminServer)
return &AdminHandlers{
adminServer: adminServer,
authHandlers: authHandlers,
@@ -34,6 +36,7 @@ func NewAdminHandlers(adminServer *dash.AdminServer) *AdminHandlers {
fileBrowserHandlers: fileBrowserHandlers,
userHandlers: userHandlers,
maintenanceHandlers: maintenanceHandlers,
+ mqHandlers: mqHandlers,
}
}
@@ -72,6 +75,11 @@ func (h *AdminHandlers) SetupRoutes(r *gin.Engine, authRequired bool, username,
protected.GET("/cluster/volumes/:id/:server", h.clusterHandlers.ShowVolumeDetails)
protected.GET("/cluster/collections", h.clusterHandlers.ShowClusterCollections)
+ // Message Queue management routes
+ protected.GET("/mq/brokers", h.mqHandlers.ShowBrokers)
+ protected.GET("/mq/topics", h.mqHandlers.ShowTopics)
+ protected.GET("/mq/topics/:namespace/:topic", h.mqHandlers.ShowTopicDetails)
+
// Maintenance system routes
protected.GET("/maintenance", h.maintenanceHandlers.ShowMaintenanceQueue)
protected.GET("/maintenance/workers", h.maintenanceHandlers.ShowMaintenanceWorkers)
@@ -144,6 +152,15 @@ func (h *AdminHandlers) SetupRoutes(r *gin.Engine, authRequired bool, username,
maintenanceApi.GET("/config", h.adminServer.GetMaintenanceConfigAPI)
maintenanceApi.PUT("/config", h.adminServer.UpdateMaintenanceConfigAPI)
}
+
+ // Message Queue API routes
+ mqApi := api.Group("/mq")
+ {
+ mqApi.GET("/topics/:namespace/:topic", h.mqHandlers.GetTopicDetailsAPI)
+ mqApi.POST("/topics/create", h.mqHandlers.CreateTopicAPI)
+ mqApi.POST("/topics/retention/update", h.mqHandlers.UpdateTopicRetentionAPI)
+ mqApi.POST("/retention/purge", h.adminServer.TriggerTopicRetentionPurgeAPI)
+ }
}
} else {
// No authentication required - all routes are public
@@ -166,6 +183,11 @@ func (h *AdminHandlers) SetupRoutes(r *gin.Engine, authRequired bool, username,
r.GET("/cluster/volumes/:id/:server", h.clusterHandlers.ShowVolumeDetails)
r.GET("/cluster/collections", h.clusterHandlers.ShowClusterCollections)
+ // Message Queue management routes
+ r.GET("/mq/brokers", h.mqHandlers.ShowBrokers)
+ r.GET("/mq/topics", h.mqHandlers.ShowTopics)
+ r.GET("/mq/topics/:namespace/:topic", h.mqHandlers.ShowTopicDetails)
+
// Maintenance system routes
r.GET("/maintenance", h.maintenanceHandlers.ShowMaintenanceQueue)
r.GET("/maintenance/workers", h.maintenanceHandlers.ShowMaintenanceWorkers)
@@ -238,6 +260,15 @@ func (h *AdminHandlers) SetupRoutes(r *gin.Engine, authRequired bool, username,
maintenanceApi.GET("/config", h.adminServer.GetMaintenanceConfigAPI)
maintenanceApi.PUT("/config", h.adminServer.UpdateMaintenanceConfigAPI)
}
+
+ // Message Queue API routes
+ mqApi := api.Group("/mq")
+ {
+ mqApi.GET("/topics/:namespace/:topic", h.mqHandlers.GetTopicDetailsAPI)
+ mqApi.POST("/topics/create", h.mqHandlers.CreateTopicAPI)
+ mqApi.POST("/topics/retention/update", h.mqHandlers.UpdateTopicRetentionAPI)
+ mqApi.POST("/retention/purge", h.adminServer.TriggerTopicRetentionPurgeAPI)
+ }
}
}
}
diff --git a/weed/admin/handlers/cluster_handlers.go b/weed/admin/handlers/cluster_handlers.go
index d8378e690..03f7e88a0 100644
--- a/weed/admin/handlers/cluster_handlers.go
+++ b/weed/admin/handlers/cluster_handlers.go
@@ -215,6 +215,33 @@ func (h *ClusterHandlers) ShowClusterFilers(c *gin.Context) {
}
}
+// ShowClusterBrokers renders the cluster message brokers page
+func (h *ClusterHandlers) ShowClusterBrokers(c *gin.Context) {
+ // Get cluster brokers data
+ brokersData, err := h.adminServer.GetClusterBrokers()
+ if err != nil {
+ c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to get cluster brokers: " + err.Error()})
+ return
+ }
+
+ // Set username
+ username := c.GetString("username")
+ if username == "" {
+ username = "admin"
+ }
+ brokersData.Username = username
+
+ // Render HTML template
+ c.Header("Content-Type", "text/html")
+ brokersComponent := app.ClusterBrokers(*brokersData)
+ layoutComponent := layout.Layout(c, brokersComponent)
+ err = layoutComponent.Render(c.Request.Context(), c.Writer)
+ if err != nil {
+ c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to render template: " + err.Error()})
+ return
+ }
+}
+
// GetClusterTopology returns the cluster topology as JSON
func (h *ClusterHandlers) GetClusterTopology(c *gin.Context) {
topology, err := h.adminServer.GetClusterTopology()
diff --git a/weed/admin/handlers/mq_handlers.go b/weed/admin/handlers/mq_handlers.go
new file mode 100644
index 000000000..8508998e6
--- /dev/null
+++ b/weed/admin/handlers/mq_handlers.go
@@ -0,0 +1,238 @@
+package handlers
+
+import (
+ "fmt"
+ "net/http"
+
+ "github.com/gin-gonic/gin"
+ "github.com/seaweedfs/seaweedfs/weed/admin/dash"
+ "github.com/seaweedfs/seaweedfs/weed/admin/view/app"
+ "github.com/seaweedfs/seaweedfs/weed/admin/view/layout"
+)
+
+// MessageQueueHandlers contains all the HTTP handlers for message queue management
+type MessageQueueHandlers struct {
+ adminServer *dash.AdminServer
+}
+
+// NewMessageQueueHandlers creates a new instance of MessageQueueHandlers
+func NewMessageQueueHandlers(adminServer *dash.AdminServer) *MessageQueueHandlers {
+ return &MessageQueueHandlers{
+ adminServer: adminServer,
+ }
+}
+
+// ShowBrokers renders the message queue brokers page
+func (h *MessageQueueHandlers) ShowBrokers(c *gin.Context) {
+ // Get cluster brokers data
+ brokersData, err := h.adminServer.GetClusterBrokers()
+ if err != nil {
+ c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to get cluster brokers: " + err.Error()})
+ return
+ }
+
+ // Set username
+ username := c.GetString("username")
+ if username == "" {
+ username = "admin"
+ }
+ brokersData.Username = username
+
+ // Render HTML template
+ c.Header("Content-Type", "text/html")
+ brokersComponent := app.ClusterBrokers(*brokersData)
+ layoutComponent := layout.Layout(c, brokersComponent)
+ err = layoutComponent.Render(c.Request.Context(), c.Writer)
+ if err != nil {
+ c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to render template: " + err.Error()})
+ return
+ }
+}
+
+// ShowTopics renders the message queue topics page
+func (h *MessageQueueHandlers) ShowTopics(c *gin.Context) {
+ // Get topics data
+ topicsData, err := h.adminServer.GetTopics()
+ if err != nil {
+ c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to get topics: " + err.Error()})
+ return
+ }
+
+ // Set username
+ username := c.GetString("username")
+ if username == "" {
+ username = "admin"
+ }
+ topicsData.Username = username
+
+ // Render HTML template
+ c.Header("Content-Type", "text/html")
+ topicsComponent := app.Topics(*topicsData)
+ layoutComponent := layout.Layout(c, topicsComponent)
+ err = layoutComponent.Render(c.Request.Context(), c.Writer)
+ if err != nil {
+ c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to render template: " + err.Error()})
+ return
+ }
+}
+
+// ShowSubscribers renders the message queue subscribers page
+func (h *MessageQueueHandlers) ShowSubscribers(c *gin.Context) {
+ // Get subscribers data
+ subscribersData, err := h.adminServer.GetSubscribers()
+ if err != nil {
+ c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to get subscribers: " + err.Error()})
+ return
+ }
+
+ // Set username
+ username := c.GetString("username")
+ if username == "" {
+ username = "admin"
+ }
+ subscribersData.Username = username
+
+ // Render HTML template
+ c.Header("Content-Type", "text/html")
+ subscribersComponent := app.Subscribers(*subscribersData)
+ layoutComponent := layout.Layout(c, subscribersComponent)
+ err = layoutComponent.Render(c.Request.Context(), c.Writer)
+ if err != nil {
+ c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to render template: " + err.Error()})
+ return
+ }
+}
+
+// ShowTopicDetails renders the topic details page
+func (h *MessageQueueHandlers) ShowTopicDetails(c *gin.Context) {
+ // Get topic parameters from URL
+ namespace := c.Param("namespace")
+ topicName := c.Param("topic")
+
+ if namespace == "" || topicName == "" {
+ c.JSON(http.StatusBadRequest, gin.H{"error": "Missing namespace or topic name"})
+ return
+ }
+
+ // Get topic details data
+ topicDetailsData, err := h.adminServer.GetTopicDetails(namespace, topicName)
+ if err != nil {
+ c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to get topic details: " + err.Error()})
+ return
+ }
+
+ // Set username
+ username := c.GetString("username")
+ if username == "" {
+ username = "admin"
+ }
+ topicDetailsData.Username = username
+
+ // Render HTML template
+ c.Header("Content-Type", "text/html")
+ topicDetailsComponent := app.TopicDetails(*topicDetailsData)
+ layoutComponent := layout.Layout(c, topicDetailsComponent)
+ err = layoutComponent.Render(c.Request.Context(), c.Writer)
+ if err != nil {
+ c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to render template: " + err.Error()})
+ return
+ }
+}
+
+// GetTopicDetailsAPI returns topic details as JSON for AJAX calls
+func (h *MessageQueueHandlers) GetTopicDetailsAPI(c *gin.Context) {
+ // Get topic parameters from URL
+ namespace := c.Param("namespace")
+ topicName := c.Param("topic")
+
+ if namespace == "" || topicName == "" {
+ c.JSON(http.StatusBadRequest, gin.H{"error": "Missing namespace or topic name"})
+ return
+ }
+
+ // Get topic details data
+ topicDetailsData, err := h.adminServer.GetTopicDetails(namespace, topicName)
+ if err != nil {
+ c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to get topic details: " + err.Error()})
+ return
+ }
+
+ // Return JSON data
+ c.JSON(http.StatusOK, topicDetailsData)
+}
+
+// CreateTopicAPI creates a new topic with retention configuration
+func (h *MessageQueueHandlers) CreateTopicAPI(c *gin.Context) {
+ var req struct {
+ Namespace string `json:"namespace" binding:"required"`
+ Name string `json:"name" binding:"required"`
+ PartitionCount int32 `json:"partition_count" binding:"required"`
+ Retention struct {
+ Enabled bool `json:"enabled"`
+ RetentionSeconds int64 `json:"retention_seconds"`
+ } `json:"retention"`
+ }
+
+ if err := c.ShouldBindJSON(&req); err != nil {
+ c.JSON(http.StatusBadRequest, gin.H{"error": "Invalid request: " + err.Error()})
+ return
+ }
+
+ // Validate inputs
+ if req.PartitionCount < 1 || req.PartitionCount > 100 {
+ c.JSON(http.StatusBadRequest, gin.H{"error": "Partition count must be between 1 and 100"})
+ return
+ }
+
+ if req.Retention.Enabled && req.Retention.RetentionSeconds <= 0 {
+ c.JSON(http.StatusBadRequest, gin.H{"error": "Retention seconds must be positive when retention is enabled"})
+ return
+ }
+
+ // Create the topic via admin server
+ err := h.adminServer.CreateTopicWithRetention(req.Namespace, req.Name, req.PartitionCount, req.Retention.Enabled, req.Retention.RetentionSeconds)
+ if err != nil {
+ c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to create topic: " + err.Error()})
+ return
+ }
+
+ c.JSON(http.StatusOK, gin.H{
+ "message": "Topic created successfully",
+ "topic": fmt.Sprintf("%s.%s", req.Namespace, req.Name),
+ })
+}
+
+type UpdateTopicRetentionRequest struct {
+ Namespace string `json:"namespace"`
+ Name string `json:"name"`
+ Retention struct {
+ Enabled bool `json:"enabled"`
+ RetentionSeconds int64 `json:"retention_seconds"`
+ } `json:"retention"`
+}
+
+func (h *MessageQueueHandlers) UpdateTopicRetentionAPI(c *gin.Context) {
+ var request UpdateTopicRetentionRequest
+ if err := c.ShouldBindJSON(&request); err != nil {
+ c.JSON(http.StatusBadRequest, gin.H{"error": err.Error()})
+ return
+ }
+
+ // Validate required fields
+ if request.Namespace == "" || request.Name == "" {
+ c.JSON(http.StatusBadRequest, gin.H{"error": "namespace and name are required"})
+ return
+ }
+
+ // Update the topic retention
+ err := h.adminServer.UpdateTopicRetention(request.Namespace, request.Name, request.Retention.Enabled, request.Retention.RetentionSeconds)
+ if err != nil {
+ c.JSON(http.StatusInternalServerError, gin.H{"error": err.Error()})
+ return
+ }
+
+ c.JSON(http.StatusOK, gin.H{
+ "message": "Topic retention updated successfully",
+ "topic": request.Namespace + "." + request.Name,
+ })
+}
diff --git a/weed/admin/static_embed.go b/weed/admin/static_embed.go
new file mode 100644
index 000000000..1910afd86
--- /dev/null
+++ b/weed/admin/static_embed.go
@@ -0,0 +1,14 @@
+package admin
+
+import (
+ "embed"
+ "io/fs"
+)
+
+//go:embed static/*
+var StaticFS embed.FS
+
+// GetStaticFS returns the embedded static filesystem
+func GetStaticFS() (fs.FS, error) {
+ return fs.Sub(StaticFS, "static")
+}
diff --git a/weed/admin/view/app/admin.templ b/weed/admin/view/app/admin.templ
index 1234583cd..c79b36042 100644
--- a/weed/admin/view/app/admin.templ
+++ b/weed/admin/view/app/admin.templ
@@ -140,7 +140,7 @@ templ Admin(data dash.AdminData) {
</div>
</div>
</div>
- <div class="col-4">
+ <div class="col-3">
<div class="card bg-light">
<div class="card-body">
<h5>{fmt.Sprintf("%d", len(data.VolumeServers))}</h5>
@@ -148,7 +148,7 @@ templ Admin(data dash.AdminData) {
</div>
</div>
</div>
- <div class="col-4">
+ <div class="col-3">
<div class="card bg-light">
<div class="card-body">
<h5>{fmt.Sprintf("%d", len(data.FilerNodes))}</h5>
@@ -156,6 +156,14 @@ templ Admin(data dash.AdminData) {
</div>
</div>
</div>
+ <div class="col-3">
+ <div class="card bg-light">
+ <div class="card-body">
+ <h5>{fmt.Sprintf("%d", len(data.MessageBrokers))}</h5>
+ <small class="text-muted">Message Brokers</small>
+ </div>
+ </div>
+ </div>
</div>
</div>
</div>
diff --git a/weed/admin/view/app/admin_templ.go b/weed/admin/view/app/admin_templ.go
index 9a25a465f..7320edd47 100644
--- a/weed/admin/view/app/admin_templ.go
+++ b/weed/admin/view/app/admin_templ.go
@@ -124,7 +124,7 @@ func Admin(data dash.AdminData) templ.Component {
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 11, "</h5><small class=\"text-muted\">Masters</small></div></div></div><div class=\"col-4\"><div class=\"card bg-light\"><div class=\"card-body\"><h5>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 11, "</h5><small class=\"text-muted\">Masters</small></div></div></div><div class=\"col-3\"><div class=\"card bg-light\"><div class=\"card-body\"><h5>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
@@ -137,7 +137,7 @@ func Admin(data dash.AdminData) templ.Component {
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 12, "</h5><small class=\"text-muted\">Volume Servers</small></div></div></div><div class=\"col-4\"><div class=\"card bg-light\"><div class=\"card-body\"><h5>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 12, "</h5><small class=\"text-muted\">Volume Servers</small></div></div></div><div class=\"col-3\"><div class=\"card bg-light\"><div class=\"card-body\"><h5>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
@@ -150,226 +150,239 @@ func Admin(data dash.AdminData) templ.Component {
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, "</h5><small class=\"text-muted\">Filers</small></div></div></div></div></div></div></div></div><!-- Volume Servers --><div class=\"row\"><div class=\"col-12\"><div class=\"card shadow mb-4\"><div class=\"card-header py-3 d-flex flex-row align-items-center justify-content-between\"><h6 class=\"m-0 font-weight-bold text-primary\"><i class=\"fas fa-database me-2\"></i>Volume Servers</h6><div class=\"dropdown no-arrow\"><a class=\"dropdown-toggle\" href=\"#\" role=\"button\" data-bs-toggle=\"dropdown\"><i class=\"fas fa-ellipsis-v fa-sm fa-fw text-gray-400\"></i></a><div class=\"dropdown-menu dropdown-menu-right shadow animated--fade-in\"><div class=\"dropdown-header\">Actions:</div><a class=\"dropdown-item\" href=\"/volumes\">View Details</a> <a class=\"dropdown-item\" href=\"/cluster\">Topology View</a></div></div></div><div class=\"card-body\"><div class=\"table-responsive\"><table class=\"table table-hover\" width=\"100%\" cellspacing=\"0\"><thead><tr><th>ID</th><th>Address</th><th>Data Center</th><th>Rack</th><th>Volumes</th><th>Capacity</th></tr></thead> <tbody>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, "</h5><small class=\"text-muted\">Filers</small></div></div></div><div class=\"col-3\"><div class=\"card bg-light\"><div class=\"card-body\"><h5>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var9 string
+ templ_7745c5c3_Var9, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", len(data.MessageBrokers)))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 162, Col: 88}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var9))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 14, "</h5><small class=\"text-muted\">Message Brokers</small></div></div></div></div></div></div></div></div><!-- Volume Servers --><div class=\"row\"><div class=\"col-12\"><div class=\"card shadow mb-4\"><div class=\"card-header py-3 d-flex flex-row align-items-center justify-content-between\"><h6 class=\"m-0 font-weight-bold text-primary\"><i class=\"fas fa-database me-2\"></i>Volume Servers</h6><div class=\"dropdown no-arrow\"><a class=\"dropdown-toggle\" href=\"#\" role=\"button\" data-bs-toggle=\"dropdown\"><i class=\"fas fa-ellipsis-v fa-sm fa-fw text-gray-400\"></i></a><div class=\"dropdown-menu dropdown-menu-right shadow animated--fade-in\"><div class=\"dropdown-header\">Actions:</div><a class=\"dropdown-item\" href=\"/volumes\">View Details</a> <a class=\"dropdown-item\" href=\"/cluster\">Topology View</a></div></div></div><div class=\"card-body\"><div class=\"table-responsive\"><table class=\"table table-hover\" width=\"100%\" cellspacing=\"0\"><thead><tr><th>ID</th><th>Address</th><th>Data Center</th><th>Rack</th><th>Volumes</th><th>Capacity</th></tr></thead> <tbody>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
for _, vs := range data.VolumeServers {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 14, "<tr><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "<tr><td>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var9 string
- templ_7745c5c3_Var9, templ_7745c5c3_Err = templ.JoinStringErrs(vs.ID)
+ var templ_7745c5c3_Var10 string
+ templ_7745c5c3_Var10, templ_7745c5c3_Err = templ.JoinStringErrs(vs.ID)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 200, Col: 54}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 208, Col: 54}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var9))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var10))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "</td><td><a href=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, "</td><td><a href=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var10 templ.SafeURL = templ.SafeURL(fmt.Sprintf("http://%s/ui/index.html", vs.PublicURL))
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(string(templ_7745c5c3_Var10)))
+ var templ_7745c5c3_Var11 templ.SafeURL = templ.SafeURL(fmt.Sprintf("http://%s/ui/index.html", vs.PublicURL))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(string(templ_7745c5c3_Var11)))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, "\" target=\"_blank\">")
- if templ_7745c5c3_Err != nil {
- return templ_7745c5c3_Err
- }
- var templ_7745c5c3_Var11 string
- templ_7745c5c3_Var11, templ_7745c5c3_Err = templ.JoinStringErrs(vs.Address)
- if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 203, Col: 63}
- }
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var11))
- if templ_7745c5c3_Err != nil {
- return templ_7745c5c3_Err
- }
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, " <i class=\"fas fa-external-link-alt ms-1 text-muted\"></i></a></td><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, "\" target=\"_blank\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var12 string
- templ_7745c5c3_Var12, templ_7745c5c3_Err = templ.JoinStringErrs(vs.DataCenter)
+ templ_7745c5c3_Var12, templ_7745c5c3_Err = templ.JoinStringErrs(vs.Address)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 207, Col: 62}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 211, Col: 63}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var12))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 18, "</td><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 18, " <i class=\"fas fa-external-link-alt ms-1 text-muted\"></i></a></td><td>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var13 string
- templ_7745c5c3_Var13, templ_7745c5c3_Err = templ.JoinStringErrs(vs.Rack)
+ templ_7745c5c3_Var13, templ_7745c5c3_Err = templ.JoinStringErrs(vs.DataCenter)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 208, Col: 56}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 215, Col: 62}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var13))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, "</td><td><div class=\"progress\" style=\"height: 20px;\"><div class=\"progress-bar\" role=\"progressbar\" style=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, "</td><td>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var14 string
- templ_7745c5c3_Var14, templ_7745c5c3_Err = templruntime.SanitizeStyleAttributeValues(fmt.Sprintf("width: %d%%", calculatePercent(vs.Volumes, vs.MaxVolumes)))
+ templ_7745c5c3_Var14, templ_7745c5c3_Err = templ.JoinStringErrs(vs.Rack)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 212, Col: 135}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 216, Col: 56}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var14))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, "\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, "</td><td><div class=\"progress\" style=\"height: 20px;\"><div class=\"progress-bar\" role=\"progressbar\" style=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var15 string
- templ_7745c5c3_Var15, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d/%d", vs.Volumes, vs.MaxVolumes))
+ templ_7745c5c3_Var15, templ_7745c5c3_Err = templruntime.SanitizeStyleAttributeValues(fmt.Sprintf("width: %d%%", calculatePercent(vs.Volumes, vs.MaxVolumes)))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 213, Col: 104}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 220, Col: 135}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var15))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, "</div></div></td><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, "\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var16 string
- templ_7745c5c3_Var16, templ_7745c5c3_Err = templ.JoinStringErrs(formatBytes(vs.DiskUsage))
+ templ_7745c5c3_Var16, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d/%d", vs.Volumes, vs.MaxVolumes))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 217, Col: 74}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 221, Col: 104}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var16))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 22, " / ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 22, "</div></div></td><td>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var17 string
- templ_7745c5c3_Var17, templ_7745c5c3_Err = templ.JoinStringErrs(formatBytes(vs.DiskCapacity))
+ templ_7745c5c3_Var17, templ_7745c5c3_Err = templ.JoinStringErrs(formatBytes(vs.DiskUsage))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 217, Col: 107}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 225, Col: 74}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var17))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 23, "</td></tr>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 23, " / ")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var18 string
+ templ_7745c5c3_Var18, templ_7745c5c3_Err = templ.JoinStringErrs(formatBytes(vs.DiskCapacity))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 225, Col: 107}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var18))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, "</td></tr>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
if len(data.VolumeServers) == 0 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, "<tr><td colspan=\"6\" class=\"text-center text-muted py-4\"><i class=\"fas fa-info-circle me-2\"></i> No volume servers found</td></tr>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 25, "<tr><td colspan=\"6\" class=\"text-center text-muted py-4\"><i class=\"fas fa-info-circle me-2\"></i> No volume servers found</td></tr>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 25, "</tbody></table></div></div></div></div></div><!-- Filer Nodes --><div class=\"row mb-4\"><div class=\"col-12\"><div class=\"card shadow mb-4\"><div class=\"card-header py-3 d-flex flex-row align-items-center justify-content-between\"><h6 class=\"m-0 font-weight-bold text-primary\"><i class=\"fas fa-folder me-2\"></i>Filer Nodes</h6><div class=\"dropdown no-arrow\"><a class=\"dropdown-toggle\" href=\"#\" role=\"button\" data-bs-toggle=\"dropdown\"><i class=\"fas fa-ellipsis-v fa-sm fa-fw text-gray-400\"></i></a><div class=\"dropdown-menu dropdown-menu-right shadow animated--fade-in\"><div class=\"dropdown-header\">Actions:</div><a class=\"dropdown-item\" href=\"/filer\">File Browser</a> <a class=\"dropdown-item\" href=\"/cluster\">Topology View</a></div></div></div><div class=\"card-body\"><div class=\"table-responsive\"><table class=\"table table-hover\" width=\"100%\" cellspacing=\"0\"><thead><tr><th>Address</th><th>Data Center</th><th>Rack</th><th>Last Updated</th></tr></thead> <tbody>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, "</tbody></table></div></div></div></div></div><!-- Filer Nodes --><div class=\"row mb-4\"><div class=\"col-12\"><div class=\"card shadow mb-4\"><div class=\"card-header py-3 d-flex flex-row align-items-center justify-content-between\"><h6 class=\"m-0 font-weight-bold text-primary\"><i class=\"fas fa-folder me-2\"></i>Filer Nodes</h6><div class=\"dropdown no-arrow\"><a class=\"dropdown-toggle\" href=\"#\" role=\"button\" data-bs-toggle=\"dropdown\"><i class=\"fas fa-ellipsis-v fa-sm fa-fw text-gray-400\"></i></a><div class=\"dropdown-menu dropdown-menu-right shadow animated--fade-in\"><div class=\"dropdown-header\">Actions:</div><a class=\"dropdown-item\" href=\"/filer\">File Browser</a> <a class=\"dropdown-item\" href=\"/cluster\">Topology View</a></div></div></div><div class=\"card-body\"><div class=\"table-responsive\"><table class=\"table table-hover\" width=\"100%\" cellspacing=\"0\"><thead><tr><th>Address</th><th>Data Center</th><th>Rack</th><th>Last Updated</th></tr></thead> <tbody>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
for _, filer := range data.FilerNodes {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, "<tr><td><a href=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, "<tr><td><a href=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var18 templ.SafeURL = templ.SafeURL(fmt.Sprintf("http://%s", filer.Address))
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(string(templ_7745c5c3_Var18)))
+ var templ_7745c5c3_Var19 templ.SafeURL = templ.SafeURL(fmt.Sprintf("http://%s", filer.Address))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(string(templ_7745c5c3_Var19)))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, "\" target=\"_blank\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, "\" target=\"_blank\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var19 string
- templ_7745c5c3_Var19, templ_7745c5c3_Err = templ.JoinStringErrs(filer.Address)
+ var templ_7745c5c3_Var20 string
+ templ_7745c5c3_Var20, templ_7745c5c3_Err = templ.JoinStringErrs(filer.Address)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 271, Col: 66}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 279, Col: 66}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var19))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var20))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, " <i class=\"fas fa-external-link-alt ms-1 text-muted\"></i></a></td><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, " <i class=\"fas fa-external-link-alt ms-1 text-muted\"></i></a></td><td>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var20 string
- templ_7745c5c3_Var20, templ_7745c5c3_Err = templ.JoinStringErrs(filer.DataCenter)
+ var templ_7745c5c3_Var21 string
+ templ_7745c5c3_Var21, templ_7745c5c3_Err = templ.JoinStringErrs(filer.DataCenter)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 275, Col: 65}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 283, Col: 65}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var20))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var21))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, "</td><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 30, "</td><td>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var21 string
- templ_7745c5c3_Var21, templ_7745c5c3_Err = templ.JoinStringErrs(filer.Rack)
+ var templ_7745c5c3_Var22 string
+ templ_7745c5c3_Var22, templ_7745c5c3_Err = templ.JoinStringErrs(filer.Rack)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 276, Col: 59}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 284, Col: 59}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var21))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var22))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 30, "</td><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 31, "</td><td>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var22 string
- templ_7745c5c3_Var22, templ_7745c5c3_Err = templ.JoinStringErrs(filer.LastUpdated.Format("2006-01-02 15:04:05"))
+ var templ_7745c5c3_Var23 string
+ templ_7745c5c3_Var23, templ_7745c5c3_Err = templ.JoinStringErrs(filer.LastUpdated.Format("2006-01-02 15:04:05"))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 277, Col: 96}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 285, Col: 96}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var22))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var23))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 31, "</td></tr>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 32, "</td></tr>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
if len(data.FilerNodes) == 0 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 32, "<tr><td colspan=\"4\" class=\"text-center text-muted py-4\"><i class=\"fas fa-info-circle me-2\"></i> No filer nodes found</td></tr>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 33, "<tr><td colspan=\"4\" class=\"text-center text-muted py-4\"><i class=\"fas fa-info-circle me-2\"></i> No filer nodes found</td></tr>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 33, "</tbody></table></div></div></div></div></div><!-- Last Updated --><div class=\"row\"><div class=\"col-12\"><small class=\"text-muted\"><i class=\"fas fa-clock me-1\"></i> Last updated: ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 34, "</tbody></table></div></div></div></div></div><!-- Last Updated --><div class=\"row\"><div class=\"col-12\"><small class=\"text-muted\"><i class=\"fas fa-clock me-1\"></i> Last updated: ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var23 string
- templ_7745c5c3_Var23, templ_7745c5c3_Err = templ.JoinStringErrs(data.LastUpdated.Format("2006-01-02 15:04:05"))
+ var templ_7745c5c3_Var24 string
+ templ_7745c5c3_Var24, templ_7745c5c3_Err = templ.JoinStringErrs(data.LastUpdated.Format("2006-01-02 15:04:05"))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 301, Col: 81}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/admin.templ`, Line: 309, Col: 81}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var23))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var24))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 34, "</small></div></div></div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 35, "</small></div></div></div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
diff --git a/weed/admin/view/app/cluster_brokers.templ b/weed/admin/view/app/cluster_brokers.templ
new file mode 100644
index 000000000..d80a14c69
--- /dev/null
+++ b/weed/admin/view/app/cluster_brokers.templ
@@ -0,0 +1,144 @@
+package app
+
+import (
+ "fmt"
+ "github.com/seaweedfs/seaweedfs/weed/admin/dash"
+)
+
+templ ClusterBrokers(data dash.ClusterBrokersData) {
+ <div class="d-flex justify-content-between flex-wrap flex-md-nowrap align-items-center pt-3 pb-2 mb-3 border-bottom">
+ <h1 class="h2">
+ <i class="fas fa-comments me-2"></i>Message Brokers
+ </h1>
+ <div class="btn-toolbar mb-2 mb-md-0">
+ <div class="btn-group me-2">
+ <button type="button" class="btn btn-sm btn-outline-primary" onclick="exportBrokers()">
+ <i class="fas fa-download me-1"></i>Export
+ </button>
+ </div>
+ </div>
+ </div>
+
+ <div id="brokers-content">
+ <!-- Summary Cards -->
+ <div class="row mb-4">
+ <div class="col-xl-12 col-md-12 mb-4">
+ <div class="card border-left-primary shadow h-100 py-2">
+ <div class="card-body">
+ <div class="row no-gutters align-items-center">
+ <div class="col mr-2">
+ <div class="text-xs font-weight-bold text-primary text-uppercase mb-1">
+ Total Message Brokers
+ </div>
+ <div class="h5 mb-0 font-weight-bold text-gray-800">
+ { fmt.Sprintf("%d", data.TotalBrokers) }
+ </div>
+ </div>
+ <div class="col-auto">
+ <i class="fas fa-comments fa-2x text-gray-300"></i>
+ </div>
+ </div>
+ </div>
+ </div>
+ </div>
+ </div>
+
+ <!-- Brokers Table -->
+ <div class="card shadow mb-4">
+ <div class="card-header py-3">
+ <h6 class="m-0 font-weight-bold text-primary">
+ <i class="fas fa-comments me-2"></i>Message Brokers
+ </h6>
+ </div>
+ <div class="card-body">
+ if len(data.Brokers) > 0 {
+ <div class="table-responsive">
+ <table class="table table-hover" id="brokersTable">
+ <thead>
+ <tr>
+ <th>Address</th>
+ <th>Version</th>
+ <th>Data Center</th>
+ <th>Rack</th>
+ <th>Created At</th>
+ </tr>
+ </thead>
+ <tbody>
+ for _, broker := range data.Brokers {
+ <tr>
+ <td>
+ { broker.Address }
+ </td>
+ <td>
+ <span class="badge bg-light text-dark">{ broker.Version }</span>
+ </td>
+ <td>
+ <span class="badge bg-light text-dark">{ broker.DataCenter }</span>
+ </td>
+ <td>
+ <span class="badge bg-light text-dark">{ broker.Rack }</span>
+ </td>
+ <td>
+ if !broker.CreatedAt.IsZero() {
+ { broker.CreatedAt.Format("2006-01-02 15:04:05") }
+ } else {
+ <span class="text-muted">N/A</span>
+ }
+ </td>
+ </tr>
+ }
+ </tbody>
+ </table>
+ </div>
+ } else {
+ <div class="text-center py-5">
+ <i class="fas fa-comments fa-3x text-muted mb-3"></i>
+ <h5 class="text-muted">No Message Brokers Found</h5>
+ <p class="text-muted">No message broker servers are currently available in the cluster.</p>
+ </div>
+ }
+ </div>
+ </div>
+
+ <!-- Last Updated -->
+ <div class="row">
+ <div class="col-12">
+ <small class="text-muted">
+ <i class="fas fa-clock me-1"></i>
+ Last updated: { data.LastUpdated.Format("2006-01-02 15:04:05") }
+ </small>
+ </div>
+ </div>
+ </div>
+
+ <script>
+ function exportBrokers() {
+ const table = document.getElementById('brokersTable');
+ if (!table) return;
+
+ let csv = 'Address,Version,Data Center,Rack,Created At\n';
+
+ const rows = table.querySelectorAll('tbody tr');
+ rows.forEach(row => {
+ const cells = row.querySelectorAll('td');
+ if (cells.length >= 5) {
+ const address = cells[0].textContent.trim();
+ const version = cells[1].textContent.trim();
+ const dataCenter = cells[2].textContent.trim();
+ const rack = cells[3].textContent.trim();
+ const createdAt = cells[4].textContent.trim();
+
+ csv += `"${address}","${version}","${dataCenter}","${rack}","${createdAt}"\n`;
+ }
+ });
+
+ const blob = new Blob([csv], { type: 'text/csv' });
+ const url = window.URL.createObjectURL(blob);
+ const a = document.createElement('a');
+ a.href = url;
+ a.download = 'message-brokers.csv';
+ a.click();
+ window.URL.revokeObjectURL(url);
+ }
+ </script>
+} \ No newline at end of file
diff --git a/weed/admin/view/app/cluster_brokers_templ.go b/weed/admin/view/app/cluster_brokers_templ.go
new file mode 100644
index 000000000..1d5030d15
--- /dev/null
+++ b/weed/admin/view/app/cluster_brokers_templ.go
@@ -0,0 +1,168 @@
+// Code generated by templ - DO NOT EDIT.
+
+// templ: version: v0.3.833
+package app
+
+//lint:file-ignore SA4006 This context is only used if a nested component is present.
+
+import "github.com/a-h/templ"
+import templruntime "github.com/a-h/templ/runtime"
+
+import (
+ "fmt"
+ "github.com/seaweedfs/seaweedfs/weed/admin/dash"
+)
+
+func ClusterBrokers(data dash.ClusterBrokersData) templ.Component {
+ return templruntime.GeneratedTemplate(func(templ_7745c5c3_Input templruntime.GeneratedComponentInput) (templ_7745c5c3_Err error) {
+ templ_7745c5c3_W, ctx := templ_7745c5c3_Input.Writer, templ_7745c5c3_Input.Context
+ if templ_7745c5c3_CtxErr := ctx.Err(); templ_7745c5c3_CtxErr != nil {
+ return templ_7745c5c3_CtxErr
+ }
+ templ_7745c5c3_Buffer, templ_7745c5c3_IsBuffer := templruntime.GetBuffer(templ_7745c5c3_W)
+ if !templ_7745c5c3_IsBuffer {
+ defer func() {
+ templ_7745c5c3_BufErr := templruntime.ReleaseBuffer(templ_7745c5c3_Buffer)
+ if templ_7745c5c3_Err == nil {
+ templ_7745c5c3_Err = templ_7745c5c3_BufErr
+ }
+ }()
+ }
+ ctx = templ.InitializeContext(ctx)
+ templ_7745c5c3_Var1 := templ.GetChildren(ctx)
+ if templ_7745c5c3_Var1 == nil {
+ templ_7745c5c3_Var1 = templ.NopComponent
+ }
+ ctx = templ.ClearChildren(ctx)
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 1, "<div class=\"d-flex justify-content-between flex-wrap flex-md-nowrap align-items-center pt-3 pb-2 mb-3 border-bottom\"><h1 class=\"h2\"><i class=\"fas fa-comments me-2\"></i>Message Brokers</h1><div class=\"btn-toolbar mb-2 mb-md-0\"><div class=\"btn-group me-2\"><button type=\"button\" class=\"btn btn-sm btn-outline-primary\" onclick=\"exportBrokers()\"><i class=\"fas fa-download me-1\"></i>Export</button></div></div></div><div id=\"brokers-content\"><!-- Summary Cards --><div class=\"row mb-4\"><div class=\"col-xl-12 col-md-12 mb-4\"><div class=\"card border-left-primary shadow h-100 py-2\"><div class=\"card-body\"><div class=\"row no-gutters align-items-center\"><div class=\"col mr-2\"><div class=\"text-xs font-weight-bold text-primary text-uppercase mb-1\">Total Message Brokers</div><div class=\"h5 mb-0 font-weight-bold text-gray-800\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var2 string
+ templ_7745c5c3_Var2, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.TotalBrokers))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_brokers.templ`, Line: 34, Col: 47}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var2))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 2, "</div></div><div class=\"col-auto\"><i class=\"fas fa-comments fa-2x text-gray-300\"></i></div></div></div></div></div></div><!-- Brokers Table --><div class=\"card shadow mb-4\"><div class=\"card-header py-3\"><h6 class=\"m-0 font-weight-bold text-primary\"><i class=\"fas fa-comments me-2\"></i>Message Brokers</h6></div><div class=\"card-body\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if len(data.Brokers) > 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 3, "<div class=\"table-responsive\"><table class=\"table table-hover\" id=\"brokersTable\"><thead><tr><th>Address</th><th>Version</th><th>Data Center</th><th>Rack</th><th>Created At</th></tr></thead> <tbody>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ for _, broker := range data.Brokers {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 4, "<tr><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var3 string
+ templ_7745c5c3_Var3, templ_7745c5c3_Err = templ.JoinStringErrs(broker.Address)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_brokers.templ`, Line: 70, Col: 27}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var3))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 5, "</td><td><span class=\"badge bg-light text-dark\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var4 string
+ templ_7745c5c3_Var4, templ_7745c5c3_Err = templ.JoinStringErrs(broker.Version)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_brokers.templ`, Line: 73, Col: 66}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var4))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 6, "</span></td><td><span class=\"badge bg-light text-dark\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var5 string
+ templ_7745c5c3_Var5, templ_7745c5c3_Err = templ.JoinStringErrs(broker.DataCenter)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_brokers.templ`, Line: 76, Col: 69}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var5))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 7, "</span></td><td><span class=\"badge bg-light text-dark\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var6 string
+ templ_7745c5c3_Var6, templ_7745c5c3_Err = templ.JoinStringErrs(broker.Rack)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_brokers.templ`, Line: 79, Col: 63}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var6))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 8, "</span></td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if !broker.CreatedAt.IsZero() {
+ var templ_7745c5c3_Var7 string
+ templ_7745c5c3_Var7, templ_7745c5c3_Err = templ.JoinStringErrs(broker.CreatedAt.Format("2006-01-02 15:04:05"))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_brokers.templ`, Line: 83, Col: 60}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var7))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 9, "<span class=\"text-muted\">N/A</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 10, "</td></tr>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 11, "</tbody></table></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 12, "<div class=\"text-center py-5\"><i class=\"fas fa-comments fa-3x text-muted mb-3\"></i><h5 class=\"text-muted\">No Message Brokers Found</h5><p class=\"text-muted\">No message broker servers are currently available in the cluster.</p></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, "</div></div><!-- Last Updated --><div class=\"row\"><div class=\"col-12\"><small class=\"text-muted\"><i class=\"fas fa-clock me-1\"></i> Last updated: ")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var8 string
+ templ_7745c5c3_Var8, templ_7745c5c3_Err = templ.JoinStringErrs(data.LastUpdated.Format("2006-01-02 15:04:05"))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_brokers.templ`, Line: 108, Col: 67}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var8))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 14, "</small></div></div></div><script>\n\tfunction exportBrokers() {\n\t\tconst table = document.getElementById('brokersTable');\n\t\tif (!table) return;\n\t\t\n\t\tlet csv = 'Address,Version,Data Center,Rack,Created At\\n';\n\t\t\n\t\tconst rows = table.querySelectorAll('tbody tr');\n\t\trows.forEach(row => {\n\t\t\tconst cells = row.querySelectorAll('td');\n\t\t\tif (cells.length >= 5) {\n\t\t\t\tconst address = cells[0].textContent.trim();\n\t\t\t\tconst version = cells[1].textContent.trim();\n\t\t\t\tconst dataCenter = cells[2].textContent.trim();\n\t\t\t\tconst rack = cells[3].textContent.trim();\n\t\t\t\tconst createdAt = cells[4].textContent.trim();\n\t\t\t\t\n\t\t\t\tcsv += `\"${address}\",\"${version}\",\"${dataCenter}\",\"${rack}\",\"${createdAt}\"\\n`;\n\t\t\t}\n\t\t});\n\t\t\n\t\tconst blob = new Blob([csv], { type: 'text/csv' });\n\t\tconst url = window.URL.createObjectURL(blob);\n\t\tconst a = document.createElement('a');\n\t\ta.href = url;\n\t\ta.download = 'message-brokers.csv';\n\t\ta.click();\n\t\twindow.URL.revokeObjectURL(url);\n\t}\n\t</script>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ return nil
+ })
+}
+
+var _ = templruntime.GeneratedTemplate
diff --git a/weed/admin/view/app/s3_buckets.templ b/weed/admin/view/app/s3_buckets.templ
index dcb8f794e..d6625f5e8 100644
--- a/weed/admin/view/app/s3_buckets.templ
+++ b/weed/admin/view/app/s3_buckets.templ
@@ -117,6 +117,8 @@ templ S3Buckets(data dash.S3BucketsData) {
<th>Objects</th>
<th>Size</th>
<th>Quota</th>
+ <th>Versioning</th>
+ <th>Object Lock</th>
<th>Actions</th>
</tr>
</thead>
@@ -152,6 +154,33 @@ templ S3Buckets(data dash.S3BucketsData) {
}
</td>
<td>
+ if bucket.VersioningEnabled {
+ <span class="badge bg-success">
+ <i class="fas fa-check me-1"></i>Enabled
+ </span>
+ } else {
+ <span class="badge bg-secondary">
+ <i class="fas fa-times me-1"></i>Disabled
+ </span>
+ }
+ </td>
+ <td>
+ if bucket.ObjectLockEnabled {
+ <div>
+ <span class="badge bg-warning">
+ <i class="fas fa-lock me-1"></i>Enabled
+ </span>
+ <div class="small text-muted">
+ {bucket.ObjectLockMode} • {fmt.Sprintf("%d days", bucket.ObjectLockDuration)}
+ </div>
+ </div>
+ } else {
+ <span class="badge bg-secondary">
+ <i class="fas fa-unlock me-1"></i>Disabled
+ </span>
+ }
+ </td>
+ <td>
<div class="btn-group btn-group-sm" role="group">
<a href={templ.SafeURL(fmt.Sprintf("/files?path=/buckets/%s", bucket.Name))}
class="btn btn-outline-success btn-sm"
@@ -183,7 +212,7 @@ templ S3Buckets(data dash.S3BucketsData) {
}
if len(data.Buckets) == 0 {
<tr>
- <td colspan="6" class="text-center text-muted py-4">
+ <td colspan="8" class="text-center text-muted py-4">
<i class="fas fa-cube fa-3x mb-3 text-muted"></i>
<div>
<h5>No Object Store buckets found</h5>
@@ -269,6 +298,53 @@ templ S3Buckets(data dash.S3BucketsData) {
</div>
</div>
+ <div class="mb-3">
+ <div class="form-check">
+ <input class="form-check-input" type="checkbox" id="enableVersioning" name="versioning_enabled">
+ <label class="form-check-label" for="enableVersioning">
+ Enable Object Versioning
+ </label>
+ </div>
+ <div class="form-text">
+ Keep multiple versions of objects in this bucket.
+ </div>
+ </div>
+
+ <div class="mb-3">
+ <div class="form-check">
+ <input class="form-check-input" type="checkbox" id="enableObjectLock" name="object_lock_enabled">
+ <label class="form-check-label" for="enableObjectLock">
+ Enable Object Lock
+ </label>
+ </div>
+ <div class="form-text">
+ Prevent objects from being deleted or overwritten for a specified period. Automatically enables versioning.
+ </div>
+ </div>
+
+ <div class="mb-3" id="objectLockSettings" style="display: none;">
+ <div class="row">
+ <div class="col-md-6">
+ <label for="objectLockMode" class="form-label">Object Lock Mode</label>
+ <select class="form-select" id="objectLockMode" name="object_lock_mode">
+ <option value="GOVERNANCE" selected>Governance</option>
+ <option value="COMPLIANCE">Compliance</option>
+ </select>
+ <div class="form-text">
+ Governance allows override with special permissions, Compliance is immutable.
+ </div>
+ </div>
+ <div class="col-md-6">
+ <label for="objectLockDuration" class="form-label">Default Retention (days)</label>
+ <input type="number" class="form-control" id="objectLockDuration" name="object_lock_duration"
+ placeholder="30" min="1" max="36500" step="1">
+ <div class="form-text">
+ Default retention period for new objects (1-36500 days).
+ </div>
+ </div>
+ </div>
+ </div>
+
</div>
<div class="modal-footer">
<button type="button" class="btn btn-secondary" data-bs-dismiss="modal">Cancel</button>
@@ -365,6 +441,200 @@ templ S3Buckets(data dash.S3BucketsData) {
</div>
</div>
</div>
+
+ <!-- JavaScript for bucket management -->
+ <script>
+ document.addEventListener('DOMContentLoaded', function() {
+ const quotaCheckbox = document.getElementById('enableQuota');
+ const quotaSettings = document.getElementById('quotaSettings');
+ const versioningCheckbox = document.getElementById('enableVersioning');
+ const objectLockCheckbox = document.getElementById('enableObjectLock');
+ const objectLockSettings = document.getElementById('objectLockSettings');
+ const createBucketForm = document.getElementById('createBucketForm');
+
+ // Toggle quota settings
+ quotaCheckbox.addEventListener('change', function() {
+ quotaSettings.style.display = this.checked ? 'block' : 'none';
+ });
+
+ // Toggle object lock settings and automatically enable versioning
+ objectLockCheckbox.addEventListener('change', function() {
+ objectLockSettings.style.display = this.checked ? 'block' : 'none';
+ if (this.checked) {
+ versioningCheckbox.checked = true;
+ versioningCheckbox.disabled = true;
+ } else {
+ versioningCheckbox.disabled = false;
+ }
+ });
+
+ // Handle form submission
+ createBucketForm.addEventListener('submit', function(e) {
+ e.preventDefault();
+
+ const formData = new FormData(this);
+ const data = {
+ name: formData.get('name'),
+ region: formData.get('region') || '',
+ quota_size: quotaCheckbox.checked ? parseInt(formData.get('quota_size')) || 0 : 0,
+ quota_unit: formData.get('quota_unit') || 'MB',
+ quota_enabled: quotaCheckbox.checked,
+ versioning_enabled: versioningCheckbox.checked,
+ object_lock_enabled: objectLockCheckbox.checked,
+ object_lock_mode: formData.get('object_lock_mode') || 'GOVERNANCE',
+ object_lock_duration: objectLockCheckbox.checked ? parseInt(formData.get('object_lock_duration')) || 30 : 0
+ };
+
+ // Validate object lock settings
+ if (data.object_lock_enabled && data.object_lock_duration <= 0) {
+ alert('Please enter a valid retention duration for object lock.');
+ return;
+ }
+
+ fetch('/api/s3/buckets', {
+ method: 'POST',
+ headers: {
+ 'Content-Type': 'application/json',
+ },
+ body: JSON.stringify(data)
+ })
+ .then(response => response.json())
+ .then(data => {
+ if (data.error) {
+ alert('Error creating bucket: ' + data.error);
+ } else {
+ alert('Bucket created successfully!');
+ location.reload();
+ }
+ })
+ .catch(error => {
+ console.error('Error:', error);
+ alert('Error creating bucket: ' + error.message);
+ });
+ });
+
+ // Handle delete bucket
+ document.querySelectorAll('.delete-bucket-btn').forEach(button => {
+ button.addEventListener('click', function() {
+ const bucketName = this.dataset.bucketName;
+ document.getElementById('deleteBucketName').textContent = bucketName;
+ window.currentBucketToDelete = bucketName;
+ new bootstrap.Modal(document.getElementById('deleteBucketModal')).show();
+ });
+ });
+
+ // Handle quota management
+ document.querySelectorAll('.quota-btn').forEach(button => {
+ button.addEventListener('click', function() {
+ const bucketName = this.dataset.bucketName;
+ const currentQuota = parseInt(this.dataset.currentQuota);
+ const quotaEnabled = this.dataset.quotaEnabled === 'true';
+
+ document.getElementById('quotaBucketName').value = bucketName;
+ document.getElementById('quotaEnabled').checked = quotaEnabled;
+ document.getElementById('quotaSizeMB').value = currentQuota;
+
+ // Toggle quota size settings
+ document.getElementById('quotaSizeSettings').style.display = quotaEnabled ? 'block' : 'none';
+
+ window.currentBucketToUpdate = bucketName;
+ new bootstrap.Modal(document.getElementById('manageQuotaModal')).show();
+ });
+ });
+
+ // Handle quota form submission
+ document.getElementById('quotaForm').addEventListener('submit', function(e) {
+ e.preventDefault();
+
+ const formData = new FormData(this);
+ const enabled = document.getElementById('quotaEnabled').checked;
+ const data = {
+ quota_size: enabled ? parseInt(formData.get('quota_size')) || 0 : 0,
+ quota_unit: formData.get('quota_unit') || 'MB',
+ quota_enabled: enabled
+ };
+
+ fetch(`/api/s3/buckets/${window.currentBucketToUpdate}/quota`, {
+ method: 'PUT',
+ headers: {
+ 'Content-Type': 'application/json',
+ },
+ body: JSON.stringify(data)
+ })
+ .then(response => response.json())
+ .then(data => {
+ if (data.error) {
+ alert('Error updating quota: ' + data.error);
+ } else {
+ alert('Quota updated successfully!');
+ location.reload();
+ }
+ })
+ .catch(error => {
+ console.error('Error:', error);
+ alert('Error updating quota: ' + error.message);
+ });
+ });
+
+ // Handle quota enabled checkbox
+ document.getElementById('quotaEnabled').addEventListener('change', function() {
+ document.getElementById('quotaSizeSettings').style.display = this.checked ? 'block' : 'none';
+ });
+ });
+
+ function deleteBucket() {
+ const bucketName = window.currentBucketToDelete;
+ if (!bucketName) return;
+
+ fetch(`/api/s3/buckets/${bucketName}`, {
+ method: 'DELETE'
+ })
+ .then(response => response.json())
+ .then(data => {
+ if (data.error) {
+ alert('Error deleting bucket: ' + data.error);
+ } else {
+ alert('Bucket deleted successfully!');
+ location.reload();
+ }
+ })
+ .catch(error => {
+ console.error('Error:', error);
+ alert('Error deleting bucket: ' + error.message);
+ });
+ }
+
+ function exportBucketList() {
+ // Simple CSV export
+ const buckets = Array.from(document.querySelectorAll('#bucketsTable tbody tr')).map(row => {
+ const cells = row.querySelectorAll('td');
+ if (cells.length > 1) {
+ return {
+ name: cells[0].textContent.trim(),
+ created: cells[1].textContent.trim(),
+ objects: cells[2].textContent.trim(),
+ size: cells[3].textContent.trim(),
+ quota: cells[4].textContent.trim(),
+ versioning: cells[5].textContent.trim(),
+ objectLock: cells[6].textContent.trim()
+ };
+ }
+ return null;
+ }).filter(bucket => bucket !== null);
+
+ const csvContent = "data:text/csv;charset=utf-8," +
+ "Name,Created,Objects,Size,Quota,Versioning,Object Lock\n" +
+ buckets.map(b => `"${b.name}","${b.created}","${b.objects}","${b.size}","${b.quota}","${b.versioning}","${b.objectLock}"`).join("\n");
+
+ const encodedUri = encodeURI(csvContent);
+ const link = document.createElement("a");
+ link.setAttribute("href", encodedUri);
+ link.setAttribute("download", "buckets.csv");
+ document.body.appendChild(link);
+ link.click();
+ document.body.removeChild(link);
+ }
+ </script>
}
// Helper functions for template
diff --git a/weed/admin/view/app/s3_buckets_templ.go b/weed/admin/view/app/s3_buckets_templ.go
index b94e525cf..303f2406e 100644
--- a/weed/admin/view/app/s3_buckets_templ.go
+++ b/weed/admin/view/app/s3_buckets_templ.go
@@ -73,7 +73,7 @@ func S3Buckets(data dash.S3BucketsData) templ.Component {
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 4, "</div></div><div class=\"col-auto\"><i class=\"fas fa-clock fa-2x text-gray-300\"></i></div></div></div></div></div></div><!-- Buckets Table --><div class=\"row\"><div class=\"col-12\"><div class=\"card shadow mb-4\"><div class=\"card-header py-3 d-flex flex-row align-items-center justify-content-between\"><h6 class=\"m-0 font-weight-bold text-primary\"><i class=\"fas fa-cube me-2\"></i>Object Store Buckets</h6><div class=\"dropdown no-arrow\"><a class=\"dropdown-toggle\" href=\"#\" role=\"button\" data-bs-toggle=\"dropdown\"><i class=\"fas fa-ellipsis-v fa-sm fa-fw text-gray-400\"></i></a><div class=\"dropdown-menu dropdown-menu-right shadow animated--fade-in\"><div class=\"dropdown-header\">Actions:</div><a class=\"dropdown-item\" href=\"#\" onclick=\"exportBucketList()\"><i class=\"fas fa-download me-2\"></i>Export List</a></div></div></div><div class=\"card-body\"><div class=\"table-responsive\"><table class=\"table table-hover\" width=\"100%\" cellspacing=\"0\" id=\"bucketsTable\"><thead><tr><th>Name</th><th>Created</th><th>Objects</th><th>Size</th><th>Quota</th><th>Actions</th></tr></thead> <tbody>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 4, "</div></div><div class=\"col-auto\"><i class=\"fas fa-clock fa-2x text-gray-300\"></i></div></div></div></div></div></div><!-- Buckets Table --><div class=\"row\"><div class=\"col-12\"><div class=\"card shadow mb-4\"><div class=\"card-header py-3 d-flex flex-row align-items-center justify-content-between\"><h6 class=\"m-0 font-weight-bold text-primary\"><i class=\"fas fa-cube me-2\"></i>Object Store Buckets</h6><div class=\"dropdown no-arrow\"><a class=\"dropdown-toggle\" href=\"#\" role=\"button\" data-bs-toggle=\"dropdown\"><i class=\"fas fa-ellipsis-v fa-sm fa-fw text-gray-400\"></i></a><div class=\"dropdown-menu dropdown-menu-right shadow animated--fade-in\"><div class=\"dropdown-header\">Actions:</div><a class=\"dropdown-item\" href=\"#\" onclick=\"exportBucketList()\"><i class=\"fas fa-download me-2\"></i>Export List</a></div></div></div><div class=\"card-body\"><div class=\"table-responsive\"><table class=\"table table-hover\" width=\"100%\" cellspacing=\"0\" id=\"bucketsTable\"><thead><tr><th>Name</th><th>Created</th><th>Objects</th><th>Size</th><th>Quota</th><th>Versioning</th><th>Object Lock</th><th>Actions</th></tr></thead> <tbody>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
@@ -94,7 +94,7 @@ func S3Buckets(data dash.S3BucketsData) templ.Component {
var templ_7745c5c3_Var6 string
templ_7745c5c3_Var6, templ_7745c5c3_Err = templ.JoinStringErrs(bucket.Name)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 130, Col: 64}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 132, Col: 64}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var6))
if templ_7745c5c3_Err != nil {
@@ -107,7 +107,7 @@ func S3Buckets(data dash.S3BucketsData) templ.Component {
var templ_7745c5c3_Var7 string
templ_7745c5c3_Var7, templ_7745c5c3_Err = templ.JoinStringErrs(bucket.CreatedAt.Format("2006-01-02 15:04"))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 133, Col: 92}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 135, Col: 92}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var7))
if templ_7745c5c3_Err != nil {
@@ -120,7 +120,7 @@ func S3Buckets(data dash.S3BucketsData) templ.Component {
var templ_7745c5c3_Var8 string
templ_7745c5c3_Var8, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", bucket.ObjectCount))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 134, Col: 86}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 136, Col: 86}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var8))
if templ_7745c5c3_Err != nil {
@@ -133,7 +133,7 @@ func S3Buckets(data dash.S3BucketsData) templ.Component {
var templ_7745c5c3_Var9 string
templ_7745c5c3_Var9, templ_7745c5c3_Err = templ.JoinStringErrs(formatBytes(bucket.Size))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 135, Col: 73}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 137, Col: 73}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var9))
if templ_7745c5c3_Err != nil {
@@ -173,7 +173,7 @@ func S3Buckets(data dash.S3BucketsData) templ.Component {
var templ_7745c5c3_Var12 string
templ_7745c5c3_Var12, templ_7745c5c3_Err = templ.JoinStringErrs(formatBytes(bucket.Quota))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 140, Col: 86}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 142, Col: 86}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var12))
if templ_7745c5c3_Err != nil {
@@ -191,7 +191,7 @@ func S3Buckets(data dash.S3BucketsData) templ.Component {
var templ_7745c5c3_Var13 string
templ_7745c5c3_Var13, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%.1f%% used", float64(bucket.Size)/float64(bucket.Quota)*100))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 144, Col: 139}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 146, Col: 139}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var13))
if templ_7745c5c3_Err != nil {
@@ -217,101 +217,157 @@ func S3Buckets(data dash.S3BucketsData) templ.Component {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, "</td><td><div class=\"btn-group btn-group-sm\" role=\"group\"><a href=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, "</td><td>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var14 templ.SafeURL = templ.SafeURL(fmt.Sprintf("/files?path=/buckets/%s", bucket.Name))
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(string(templ_7745c5c3_Var14)))
+ if bucket.VersioningEnabled {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, "<span class=\"badge bg-success\"><i class=\"fas fa-check me-1\"></i>Enabled</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 22, "<span class=\"badge bg-secondary\"><i class=\"fas fa-times me-1\"></i>Disabled</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 23, "</td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if bucket.ObjectLockEnabled {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, "<div><span class=\"badge bg-warning\"><i class=\"fas fa-lock me-1\"></i>Enabled</span><div class=\"small text-muted\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var14 string
+ templ_7745c5c3_Var14, templ_7745c5c3_Err = templ.JoinStringErrs(bucket.ObjectLockMode)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 174, Col: 82}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var14))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 25, " • ")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var15 string
+ templ_7745c5c3_Var15, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d days", bucket.ObjectLockDuration))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 174, Col: 138}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var15))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, "</div></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, "<span class=\"badge bg-secondary\"><i class=\"fas fa-unlock me-1\"></i>Disabled</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, "</td><td><div class=\"btn-group btn-group-sm\" role=\"group\"><a href=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, "\" class=\"btn btn-outline-success btn-sm\" title=\"Browse Files\"><i class=\"fas fa-folder-open\"></i></a> <a href=\"")
+ var templ_7745c5c3_Var16 templ.SafeURL = templ.SafeURL(fmt.Sprintf("/files?path=/buckets/%s", bucket.Name))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(string(templ_7745c5c3_Var16)))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var15 templ.SafeURL = templ.SafeURL(fmt.Sprintf("/s3/buckets/%s", bucket.Name))
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(string(templ_7745c5c3_Var15)))
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, "\" class=\"btn btn-outline-success btn-sm\" title=\"Browse Files\"><i class=\"fas fa-folder-open\"></i></a> <a href=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 22, "\" class=\"btn btn-outline-primary btn-sm\" title=\"View Details\"><i class=\"fas fa-eye\"></i></a> <button type=\"button\" class=\"btn btn-outline-warning btn-sm quota-btn\" data-bucket-name=\"")
+ var templ_7745c5c3_Var17 templ.SafeURL = templ.SafeURL(fmt.Sprintf("/s3/buckets/%s", bucket.Name))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(string(templ_7745c5c3_Var17)))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var16 string
- templ_7745c5c3_Var16, templ_7745c5c3_Err = templ.JoinStringErrs(bucket.Name)
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 30, "\" class=\"btn btn-outline-primary btn-sm\" title=\"View Details\"><i class=\"fas fa-eye\"></i></a> <button type=\"button\" class=\"btn btn-outline-warning btn-sm quota-btn\" data-bucket-name=\"")
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 168, Col: 89}
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var18 string
+ templ_7745c5c3_Var18, templ_7745c5c3_Err = templ.JoinStringErrs(bucket.Name)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 197, Col: 89}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var16))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var18))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 23, "\" data-current-quota=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 31, "\" data-current-quota=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var17 string
- templ_7745c5c3_Var17, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", getQuotaInMB(bucket.Quota)))
+ var templ_7745c5c3_Var19 string
+ templ_7745c5c3_Var19, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", getQuotaInMB(bucket.Quota)))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 169, Col: 125}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 198, Col: 125}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var17))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var19))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, "\" data-quota-enabled=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 32, "\" data-quota-enabled=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var18 string
- templ_7745c5c3_Var18, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%t", bucket.QuotaEnabled))
+ var templ_7745c5c3_Var20 string
+ templ_7745c5c3_Var20, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%t", bucket.QuotaEnabled))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 170, Col: 118}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 199, Col: 118}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var18))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var20))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 25, "\" title=\"Manage Quota\"><i class=\"fas fa-tachometer-alt\"></i></button> <button type=\"button\" class=\"btn btn-outline-danger btn-sm delete-bucket-btn\" data-bucket-name=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 33, "\" title=\"Manage Quota\"><i class=\"fas fa-tachometer-alt\"></i></button> <button type=\"button\" class=\"btn btn-outline-danger btn-sm delete-bucket-btn\" data-bucket-name=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var19 string
- templ_7745c5c3_Var19, templ_7745c5c3_Err = templ.JoinStringErrs(bucket.Name)
+ var templ_7745c5c3_Var21 string
+ templ_7745c5c3_Var21, templ_7745c5c3_Err = templ.JoinStringErrs(bucket.Name)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 176, Col: 89}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 205, Col: 89}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var19))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var21))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, "\" title=\"Delete Bucket\"><i class=\"fas fa-trash\"></i></button></div></td></tr>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 34, "\" title=\"Delete Bucket\"><i class=\"fas fa-trash\"></i></button></div></td></tr>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
if len(data.Buckets) == 0 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, "<tr><td colspan=\"6\" class=\"text-center text-muted py-4\"><i class=\"fas fa-cube fa-3x mb-3 text-muted\"></i><div><h5>No Object Store buckets found</h5><p>Create your first bucket to get started with S3 storage.</p><button type=\"button\" class=\"btn btn-primary\" data-bs-toggle=\"modal\" data-bs-target=\"#createBucketModal\"><i class=\"fas fa-plus me-1\"></i>Create Bucket</button></div></td></tr>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 35, "<tr><td colspan=\"8\" class=\"text-center text-muted py-4\"><i class=\"fas fa-cube fa-3x mb-3 text-muted\"></i><div><h5>No Object Store buckets found</h5><p>Create your first bucket to get started with S3 storage.</p><button type=\"button\" class=\"btn btn-primary\" data-bs-toggle=\"modal\" data-bs-target=\"#createBucketModal\"><i class=\"fas fa-plus me-1\"></i>Create Bucket</button></div></td></tr>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, "</tbody></table></div></div></div></div></div><!-- Last Updated --><div class=\"row\"><div class=\"col-12\"><small class=\"text-muted\"><i class=\"fas fa-clock me-1\"></i> Last updated: ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 36, "</tbody></table></div></div></div></div></div><!-- Last Updated --><div class=\"row\"><div class=\"col-12\"><small class=\"text-muted\"><i class=\"fas fa-clock me-1\"></i> Last updated: ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var20 string
- templ_7745c5c3_Var20, templ_7745c5c3_Err = templ.JoinStringErrs(data.LastUpdated.Format("2006-01-02 15:04:05"))
+ var templ_7745c5c3_Var22 string
+ templ_7745c5c3_Var22, templ_7745c5c3_Err = templ.JoinStringErrs(data.LastUpdated.Format("2006-01-02 15:04:05"))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 213, Col: 81}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/s3_buckets.templ`, Line: 242, Col: 81}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var20))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var22))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, "</small></div></div></div><!-- Create Bucket Modal --><div class=\"modal fade\" id=\"createBucketModal\" tabindex=\"-1\" aria-labelledby=\"createBucketModalLabel\" aria-hidden=\"true\"><div class=\"modal-dialog\"><div class=\"modal-content\"><div class=\"modal-header\"><h5 class=\"modal-title\" id=\"createBucketModalLabel\"><i class=\"fas fa-plus me-2\"></i>Create New S3 Bucket</h5><button type=\"button\" class=\"btn-close\" data-bs-dismiss=\"modal\" aria-label=\"Close\"></button></div><form id=\"createBucketForm\"><div class=\"modal-body\"><div class=\"mb-3\"><label for=\"bucketName\" class=\"form-label\">Bucket Name</label> <input type=\"text\" class=\"form-control\" id=\"bucketName\" name=\"name\" placeholder=\"my-bucket-name\" required pattern=\"[a-z0-9.-]+\" title=\"Bucket name must contain only lowercase letters, numbers, dots, and hyphens\"><div class=\"form-text\">Bucket names must be between 3 and 63 characters, contain only lowercase letters, numbers, dots, and hyphens.</div></div><div class=\"mb-3\"><div class=\"form-check\"><input class=\"form-check-input\" type=\"checkbox\" id=\"enableQuota\" name=\"quota_enabled\"> <label class=\"form-check-label\" for=\"enableQuota\">Enable Storage Quota</label></div></div><div class=\"mb-3\" id=\"quotaSettings\" style=\"display: none;\"><div class=\"row\"><div class=\"col-md-8\"><label for=\"quotaSize\" class=\"form-label\">Quota Size</label> <input type=\"number\" class=\"form-control\" id=\"quotaSize\" name=\"quota_size\" placeholder=\"1024\" min=\"1\" step=\"1\"></div><div class=\"col-md-4\"><label for=\"quotaUnit\" class=\"form-label\">Unit</label> <select class=\"form-select\" id=\"quotaUnit\" name=\"quota_unit\"><option value=\"MB\" selected>MB</option> <option value=\"GB\">GB</option> <option value=\"TB\">TB</option></select></div></div><div class=\"form-text\">Set the maximum storage size for this bucket.</div></div></div><div class=\"modal-footer\"><button type=\"button\" class=\"btn btn-secondary\" data-bs-dismiss=\"modal\">Cancel</button> <button type=\"submit\" class=\"btn btn-primary\"><i class=\"fas fa-plus me-1\"></i>Create Bucket</button></div></form></div></div></div><!-- Delete Confirmation Modal --><div class=\"modal fade\" id=\"deleteBucketModal\" tabindex=\"-1\" aria-labelledby=\"deleteBucketModalLabel\" aria-hidden=\"true\"><div class=\"modal-dialog\"><div class=\"modal-content\"><div class=\"modal-header\"><h5 class=\"modal-title\" id=\"deleteBucketModalLabel\"><i class=\"fas fa-exclamation-triangle me-2 text-warning\"></i>Delete Bucket</h5><button type=\"button\" class=\"btn-close\" data-bs-dismiss=\"modal\" aria-label=\"Close\"></button></div><div class=\"modal-body\"><p>Are you sure you want to delete the bucket <strong id=\"deleteBucketName\"></strong>?</p><div class=\"alert alert-warning\"><i class=\"fas fa-exclamation-triangle me-2\"></i> <strong>Warning:</strong> This action cannot be undone. All objects in the bucket will be permanently deleted.</div></div><div class=\"modal-footer\"><button type=\"button\" class=\"btn btn-secondary\" data-bs-dismiss=\"modal\">Cancel</button> <button type=\"button\" class=\"btn btn-danger\" onclick=\"deleteBucket()\"><i class=\"fas fa-trash me-1\"></i>Delete Bucket</button></div></div></div></div><!-- Manage Quota Modal --><div class=\"modal fade\" id=\"manageQuotaModal\" tabindex=\"-1\" aria-labelledby=\"manageQuotaModalLabel\" aria-hidden=\"true\"><div class=\"modal-dialog\"><div class=\"modal-content\"><div class=\"modal-header\"><h5 class=\"modal-title\" id=\"manageQuotaModalLabel\"><i class=\"fas fa-tachometer-alt me-2\"></i>Manage Bucket Quota</h5><button type=\"button\" class=\"btn-close\" data-bs-dismiss=\"modal\" aria-label=\"Close\"></button></div><form id=\"quotaForm\"><div class=\"modal-body\"><div class=\"mb-3\"><label class=\"form-label\">Bucket Name</label> <input type=\"text\" class=\"form-control\" id=\"quotaBucketName\" readonly></div><div class=\"mb-3\"><div class=\"form-check\"><input class=\"form-check-input\" type=\"checkbox\" id=\"quotaEnabled\" name=\"quota_enabled\"> <label class=\"form-check-label\" for=\"quotaEnabled\">Enable Storage Quota</label></div></div><div class=\"mb-3\" id=\"quotaSizeSettings\"><div class=\"row\"><div class=\"col-md-8\"><label for=\"quotaSizeMB\" class=\"form-label\">Quota Size</label> <input type=\"number\" class=\"form-control\" id=\"quotaSizeMB\" name=\"quota_size\" placeholder=\"1024\" min=\"0\" step=\"1\"></div><div class=\"col-md-4\"><label for=\"quotaUnitMB\" class=\"form-label\">Unit</label> <select class=\"form-select\" id=\"quotaUnitMB\" name=\"quota_unit\"><option value=\"MB\" selected>MB</option> <option value=\"GB\">GB</option> <option value=\"TB\">TB</option></select></div></div><div class=\"form-text\">Set the maximum storage size for this bucket. Set to 0 to remove quota.</div></div></div><div class=\"modal-footer\"><button type=\"button\" class=\"btn btn-secondary\" data-bs-dismiss=\"modal\">Cancel</button> <button type=\"submit\" class=\"btn btn-warning\"><i class=\"fas fa-save me-1\"></i>Update Quota</button></div></form></div></div></div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 37, "</small></div></div></div><!-- Create Bucket Modal --><div class=\"modal fade\" id=\"createBucketModal\" tabindex=\"-1\" aria-labelledby=\"createBucketModalLabel\" aria-hidden=\"true\"><div class=\"modal-dialog\"><div class=\"modal-content\"><div class=\"modal-header\"><h5 class=\"modal-title\" id=\"createBucketModalLabel\"><i class=\"fas fa-plus me-2\"></i>Create New S3 Bucket</h5><button type=\"button\" class=\"btn-close\" data-bs-dismiss=\"modal\" aria-label=\"Close\"></button></div><form id=\"createBucketForm\"><div class=\"modal-body\"><div class=\"mb-3\"><label for=\"bucketName\" class=\"form-label\">Bucket Name</label> <input type=\"text\" class=\"form-control\" id=\"bucketName\" name=\"name\" placeholder=\"my-bucket-name\" required pattern=\"[a-z0-9.-]+\" title=\"Bucket name must contain only lowercase letters, numbers, dots, and hyphens\"><div class=\"form-text\">Bucket names must be between 3 and 63 characters, contain only lowercase letters, numbers, dots, and hyphens.</div></div><div class=\"mb-3\"><div class=\"form-check\"><input class=\"form-check-input\" type=\"checkbox\" id=\"enableQuota\" name=\"quota_enabled\"> <label class=\"form-check-label\" for=\"enableQuota\">Enable Storage Quota</label></div></div><div class=\"mb-3\" id=\"quotaSettings\" style=\"display: none;\"><div class=\"row\"><div class=\"col-md-8\"><label for=\"quotaSize\" class=\"form-label\">Quota Size</label> <input type=\"number\" class=\"form-control\" id=\"quotaSize\" name=\"quota_size\" placeholder=\"1024\" min=\"1\" step=\"1\"></div><div class=\"col-md-4\"><label for=\"quotaUnit\" class=\"form-label\">Unit</label> <select class=\"form-select\" id=\"quotaUnit\" name=\"quota_unit\"><option value=\"MB\" selected>MB</option> <option value=\"GB\">GB</option> <option value=\"TB\">TB</option></select></div></div><div class=\"form-text\">Set the maximum storage size for this bucket.</div></div><div class=\"mb-3\"><div class=\"form-check\"><input class=\"form-check-input\" type=\"checkbox\" id=\"enableVersioning\" name=\"versioning_enabled\"> <label class=\"form-check-label\" for=\"enableVersioning\">Enable Object Versioning</label></div><div class=\"form-text\">Keep multiple versions of objects in this bucket.</div></div><div class=\"mb-3\"><div class=\"form-check\"><input class=\"form-check-input\" type=\"checkbox\" id=\"enableObjectLock\" name=\"object_lock_enabled\"> <label class=\"form-check-label\" for=\"enableObjectLock\">Enable Object Lock</label></div><div class=\"form-text\">Prevent objects from being deleted or overwritten for a specified period. Automatically enables versioning.</div></div><div class=\"mb-3\" id=\"objectLockSettings\" style=\"display: none;\"><div class=\"row\"><div class=\"col-md-6\"><label for=\"objectLockMode\" class=\"form-label\">Object Lock Mode</label> <select class=\"form-select\" id=\"objectLockMode\" name=\"object_lock_mode\"><option value=\"GOVERNANCE\" selected>Governance</option> <option value=\"COMPLIANCE\">Compliance</option></select><div class=\"form-text\">Governance allows override with special permissions, Compliance is immutable.</div></div><div class=\"col-md-6\"><label for=\"objectLockDuration\" class=\"form-label\">Default Retention (days)</label> <input type=\"number\" class=\"form-control\" id=\"objectLockDuration\" name=\"object_lock_duration\" placeholder=\"30\" min=\"1\" max=\"36500\" step=\"1\"><div class=\"form-text\">Default retention period for new objects (1-36500 days).</div></div></div></div></div><div class=\"modal-footer\"><button type=\"button\" class=\"btn btn-secondary\" data-bs-dismiss=\"modal\">Cancel</button> <button type=\"submit\" class=\"btn btn-primary\"><i class=\"fas fa-plus me-1\"></i>Create Bucket</button></div></form></div></div></div><!-- Delete Confirmation Modal --><div class=\"modal fade\" id=\"deleteBucketModal\" tabindex=\"-1\" aria-labelledby=\"deleteBucketModalLabel\" aria-hidden=\"true\"><div class=\"modal-dialog\"><div class=\"modal-content\"><div class=\"modal-header\"><h5 class=\"modal-title\" id=\"deleteBucketModalLabel\"><i class=\"fas fa-exclamation-triangle me-2 text-warning\"></i>Delete Bucket</h5><button type=\"button\" class=\"btn-close\" data-bs-dismiss=\"modal\" aria-label=\"Close\"></button></div><div class=\"modal-body\"><p>Are you sure you want to delete the bucket <strong id=\"deleteBucketName\"></strong>?</p><div class=\"alert alert-warning\"><i class=\"fas fa-exclamation-triangle me-2\"></i> <strong>Warning:</strong> This action cannot be undone. All objects in the bucket will be permanently deleted.</div></div><div class=\"modal-footer\"><button type=\"button\" class=\"btn btn-secondary\" data-bs-dismiss=\"modal\">Cancel</button> <button type=\"button\" class=\"btn btn-danger\" onclick=\"deleteBucket()\"><i class=\"fas fa-trash me-1\"></i>Delete Bucket</button></div></div></div></div><!-- Manage Quota Modal --><div class=\"modal fade\" id=\"manageQuotaModal\" tabindex=\"-1\" aria-labelledby=\"manageQuotaModalLabel\" aria-hidden=\"true\"><div class=\"modal-dialog\"><div class=\"modal-content\"><div class=\"modal-header\"><h5 class=\"modal-title\" id=\"manageQuotaModalLabel\"><i class=\"fas fa-tachometer-alt me-2\"></i>Manage Bucket Quota</h5><button type=\"button\" class=\"btn-close\" data-bs-dismiss=\"modal\" aria-label=\"Close\"></button></div><form id=\"quotaForm\"><div class=\"modal-body\"><div class=\"mb-3\"><label class=\"form-label\">Bucket Name</label> <input type=\"text\" class=\"form-control\" id=\"quotaBucketName\" readonly></div><div class=\"mb-3\"><div class=\"form-check\"><input class=\"form-check-input\" type=\"checkbox\" id=\"quotaEnabled\" name=\"quota_enabled\"> <label class=\"form-check-label\" for=\"quotaEnabled\">Enable Storage Quota</label></div></div><div class=\"mb-3\" id=\"quotaSizeSettings\"><div class=\"row\"><div class=\"col-md-8\"><label for=\"quotaSizeMB\" class=\"form-label\">Quota Size</label> <input type=\"number\" class=\"form-control\" id=\"quotaSizeMB\" name=\"quota_size\" placeholder=\"1024\" min=\"0\" step=\"1\"></div><div class=\"col-md-4\"><label for=\"quotaUnitMB\" class=\"form-label\">Unit</label> <select class=\"form-select\" id=\"quotaUnitMB\" name=\"quota_unit\"><option value=\"MB\" selected>MB</option> <option value=\"GB\">GB</option> <option value=\"TB\">TB</option></select></div></div><div class=\"form-text\">Set the maximum storage size for this bucket. Set to 0 to remove quota.</div></div></div><div class=\"modal-footer\"><button type=\"button\" class=\"btn btn-secondary\" data-bs-dismiss=\"modal\">Cancel</button> <button type=\"submit\" class=\"btn btn-warning\"><i class=\"fas fa-save me-1\"></i>Update Quota</button></div></form></div></div></div><!-- JavaScript for bucket management --><script>\n document.addEventListener('DOMContentLoaded', function() {\n const quotaCheckbox = document.getElementById('enableQuota');\n const quotaSettings = document.getElementById('quotaSettings');\n const versioningCheckbox = document.getElementById('enableVersioning');\n const objectLockCheckbox = document.getElementById('enableObjectLock');\n const objectLockSettings = document.getElementById('objectLockSettings');\n const createBucketForm = document.getElementById('createBucketForm');\n\n // Toggle quota settings\n quotaCheckbox.addEventListener('change', function() {\n quotaSettings.style.display = this.checked ? 'block' : 'none';\n });\n\n // Toggle object lock settings and automatically enable versioning\n objectLockCheckbox.addEventListener('change', function() {\n objectLockSettings.style.display = this.checked ? 'block' : 'none';\n if (this.checked) {\n versioningCheckbox.checked = true;\n versioningCheckbox.disabled = true;\n } else {\n versioningCheckbox.disabled = false;\n }\n });\n\n // Handle form submission\n createBucketForm.addEventListener('submit', function(e) {\n e.preventDefault();\n \n const formData = new FormData(this);\n const data = {\n name: formData.get('name'),\n region: formData.get('region') || '',\n quota_size: quotaCheckbox.checked ? parseInt(formData.get('quota_size')) || 0 : 0,\n quota_unit: formData.get('quota_unit') || 'MB',\n quota_enabled: quotaCheckbox.checked,\n versioning_enabled: versioningCheckbox.checked,\n object_lock_enabled: objectLockCheckbox.checked,\n object_lock_mode: formData.get('object_lock_mode') || 'GOVERNANCE',\n object_lock_duration: objectLockCheckbox.checked ? parseInt(formData.get('object_lock_duration')) || 30 : 0\n };\n\n // Validate object lock settings\n if (data.object_lock_enabled && data.object_lock_duration <= 0) {\n alert('Please enter a valid retention duration for object lock.');\n return;\n }\n\n fetch('/api/s3/buckets', {\n method: 'POST',\n headers: {\n 'Content-Type': 'application/json',\n },\n body: JSON.stringify(data)\n })\n .then(response => response.json())\n .then(data => {\n if (data.error) {\n alert('Error creating bucket: ' + data.error);\n } else {\n alert('Bucket created successfully!');\n location.reload();\n }\n })\n .catch(error => {\n console.error('Error:', error);\n alert('Error creating bucket: ' + error.message);\n });\n });\n\n // Handle delete bucket\n document.querySelectorAll('.delete-bucket-btn').forEach(button => {\n button.addEventListener('click', function() {\n const bucketName = this.dataset.bucketName;\n document.getElementById('deleteBucketName').textContent = bucketName;\n window.currentBucketToDelete = bucketName;\n new bootstrap.Modal(document.getElementById('deleteBucketModal')).show();\n });\n });\n\n // Handle quota management\n document.querySelectorAll('.quota-btn').forEach(button => {\n button.addEventListener('click', function() {\n const bucketName = this.dataset.bucketName;\n const currentQuota = parseInt(this.dataset.currentQuota);\n const quotaEnabled = this.dataset.quotaEnabled === 'true';\n \n document.getElementById('quotaBucketName').value = bucketName;\n document.getElementById('quotaEnabled').checked = quotaEnabled;\n document.getElementById('quotaSizeMB').value = currentQuota;\n \n // Toggle quota size settings\n document.getElementById('quotaSizeSettings').style.display = quotaEnabled ? 'block' : 'none';\n \n window.currentBucketToUpdate = bucketName;\n new bootstrap.Modal(document.getElementById('manageQuotaModal')).show();\n });\n });\n\n // Handle quota form submission\n document.getElementById('quotaForm').addEventListener('submit', function(e) {\n e.preventDefault();\n \n const formData = new FormData(this);\n const enabled = document.getElementById('quotaEnabled').checked;\n const data = {\n quota_size: enabled ? parseInt(formData.get('quota_size')) || 0 : 0,\n quota_unit: formData.get('quota_unit') || 'MB',\n quota_enabled: enabled\n };\n\n fetch(`/api/s3/buckets/${window.currentBucketToUpdate}/quota`, {\n method: 'PUT',\n headers: {\n 'Content-Type': 'application/json',\n },\n body: JSON.stringify(data)\n })\n .then(response => response.json())\n .then(data => {\n if (data.error) {\n alert('Error updating quota: ' + data.error);\n } else {\n alert('Quota updated successfully!');\n location.reload();\n }\n })\n .catch(error => {\n console.error('Error:', error);\n alert('Error updating quota: ' + error.message);\n });\n });\n\n // Handle quota enabled checkbox\n document.getElementById('quotaEnabled').addEventListener('change', function() {\n document.getElementById('quotaSizeSettings').style.display = this.checked ? 'block' : 'none';\n });\n });\n\n function deleteBucket() {\n const bucketName = window.currentBucketToDelete;\n if (!bucketName) return;\n\n fetch(`/api/s3/buckets/${bucketName}`, {\n method: 'DELETE'\n })\n .then(response => response.json())\n .then(data => {\n if (data.error) {\n alert('Error deleting bucket: ' + data.error);\n } else {\n alert('Bucket deleted successfully!');\n location.reload();\n }\n })\n .catch(error => {\n console.error('Error:', error);\n alert('Error deleting bucket: ' + error.message);\n });\n }\n\n function exportBucketList() {\n // Simple CSV export\n const buckets = Array.from(document.querySelectorAll('#bucketsTable tbody tr')).map(row => {\n const cells = row.querySelectorAll('td');\n if (cells.length > 1) {\n return {\n name: cells[0].textContent.trim(),\n created: cells[1].textContent.trim(),\n objects: cells[2].textContent.trim(),\n size: cells[3].textContent.trim(),\n quota: cells[4].textContent.trim(),\n versioning: cells[5].textContent.trim(),\n objectLock: cells[6].textContent.trim()\n };\n }\n return null;\n }).filter(bucket => bucket !== null);\n\n const csvContent = \"data:text/csv;charset=utf-8,\" + \n \"Name,Created,Objects,Size,Quota,Versioning,Object Lock\\n\" +\n buckets.map(b => `\"${b.name}\",\"${b.created}\",\"${b.objects}\",\"${b.size}\",\"${b.quota}\",\"${b.versioning}\",\"${b.objectLock}\"`).join(\"\\n\");\n\n const encodedUri = encodeURI(csvContent);\n const link = document.createElement(\"a\");\n link.setAttribute(\"href\", encodedUri);\n link.setAttribute(\"download\", \"buckets.csv\");\n document.body.appendChild(link);\n link.click();\n document.body.removeChild(link);\n }\n </script>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
diff --git a/weed/admin/view/app/subscribers.templ b/weed/admin/view/app/subscribers.templ
new file mode 100644
index 000000000..edcaf8a7b
--- /dev/null
+++ b/weed/admin/view/app/subscribers.templ
@@ -0,0 +1,151 @@
+package app
+
+import "fmt"
+import "github.com/seaweedfs/seaweedfs/weed/admin/dash"
+
+templ Subscribers(data dash.SubscribersData) {
+ <div class="container-fluid">
+ <div class="row">
+ <div class="col-12">
+ <div class="d-flex justify-content-between align-items-center mb-4">
+ <h1 class="h3 mb-0">Message Queue Subscribers</h1>
+ <small class="text-muted">Last updated: {data.LastUpdated.Format("2006-01-02 15:04:05")}</small>
+ </div>
+
+ <!-- Summary Cards -->
+ <div class="row mb-4">
+ <div class="col-md-4">
+ <div class="card text-center">
+ <div class="card-body">
+ <h5 class="card-title">Total Subscribers</h5>
+ <h3 class="text-primary">{fmt.Sprintf("%d", data.TotalSubscribers)}</h3>
+ </div>
+ </div>
+ </div>
+ <div class="col-md-4">
+ <div class="card text-center">
+ <div class="card-body">
+ <h5 class="card-title">Active Subscribers</h5>
+ <h3 class="text-success">{fmt.Sprintf("%d", data.ActiveSubscribers)}</h3>
+ </div>
+ </div>
+ </div>
+ <div class="col-md-4">
+ <div class="card text-center">
+ <div class="card-body">
+ <h5 class="card-title">Inactive Subscribers</h5>
+ <h3 class="text-warning">{fmt.Sprintf("%d", data.TotalSubscribers - data.ActiveSubscribers)}</h3>
+ </div>
+ </div>
+ </div>
+ </div>
+
+ <!-- Subscribers Table -->
+ <div class="card">
+ <div class="card-header d-flex justify-content-between align-items-center">
+ <h5 class="mb-0">Subscribers</h5>
+ <div>
+ <button class="btn btn-sm btn-outline-secondary" onclick="exportSubscribersCSV()">
+ <i class="fas fa-download me-1"></i>Export CSV
+ </button>
+ </div>
+ </div>
+ <div class="card-body">
+ if len(data.Subscribers) == 0 {
+ <div class="text-center py-4">
+ <i class="fas fa-user-friends fa-3x text-muted mb-3"></i>
+ <h5>No Subscribers Found</h5>
+ <p class="text-muted">No message queue subscribers are currently active.</p>
+ </div>
+ } else {
+ <div class="table-responsive">
+ <table class="table table-striped" id="subscribersTable">
+ <thead>
+ <tr>
+ <th>Subscriber Name</th>
+ <th>Topic</th>
+ <th>Consumer Group</th>
+ <th>Status</th>
+ <th>Messages Processed</th>
+ <th>Last Seen</th>
+ <th>Created</th>
+ </tr>
+ </thead>
+ <tbody>
+ for _, subscriber := range data.Subscribers {
+ <tr>
+ <td>
+ <strong>{subscriber.Name}</strong>
+ </td>
+ <td>
+ <span class="badge bg-info">{subscriber.Topic}</span>
+ </td>
+ <td>{subscriber.ConsumerGroup}</td>
+ <td>
+ if subscriber.Status == "active" {
+ <span class="badge bg-success">Active</span>
+ } else if subscriber.Status == "inactive" {
+ <span class="badge bg-warning">Inactive</span>
+ } else {
+ <span class="badge bg-secondary">{subscriber.Status}</span>
+ }
+ </td>
+ <td>{fmt.Sprintf("%d", subscriber.MessageCount)}</td>
+ <td>
+ if !subscriber.LastSeen.IsZero() {
+ <span class="text-muted">{subscriber.LastSeen.Format("2006-01-02 15:04:05")}</span>
+ } else {
+ <span class="text-muted">Never</span>
+ }
+ </td>
+ <td>
+ <span class="text-muted">{subscriber.CreatedAt.Format("2006-01-02 15:04:05")}</span>
+ </td>
+ </tr>
+ }
+ </tbody>
+ </table>
+ </div>
+ }
+ </div>
+ </div>
+ </div>
+ </div>
+ </div>
+
+ <script>
+ function exportSubscribersCSV() {
+ const table = document.getElementById('subscribersTable');
+ if (!table) return;
+
+ let csv = 'Subscriber Name,Topic,Consumer Group,Status,Messages Processed,Last Seen,Created\n';
+
+ const rows = table.querySelectorAll('tbody tr');
+ rows.forEach(row => {
+ const cells = row.querySelectorAll('td');
+ if (cells.length >= 7) {
+ const rowData = [
+ cells[0].querySelector('strong')?.textContent || '',
+ cells[1].querySelector('.badge')?.textContent || '',
+ cells[2].textContent || '',
+ cells[3].querySelector('.badge')?.textContent || '',
+ cells[4].textContent || '',
+ cells[5].querySelector('span')?.textContent || '',
+ cells[6].querySelector('span')?.textContent || ''
+ ];
+ csv += rowData.map(field => `"${field.replace(/"/g, '""')}"`).join(',') + '\n';
+ }
+ });
+
+ const blob = new Blob([csv], { type: 'text/csv;charset=utf-8;' });
+ const link = document.createElement('a');
+ const url = URL.createObjectURL(blob);
+ link.setAttribute('href', url);
+ link.setAttribute('download', 'subscribers.csv');
+ link.style.visibility = 'hidden';
+ document.body.appendChild(link);
+ link.click();
+ document.body.removeChild(link);
+ }
+ </script>
+} \ No newline at end of file
diff --git a/weed/admin/view/app/subscribers_templ.go b/weed/admin/view/app/subscribers_templ.go
new file mode 100644
index 000000000..4bb944df0
--- /dev/null
+++ b/weed/admin/view/app/subscribers_templ.go
@@ -0,0 +1,246 @@
+// Code generated by templ - DO NOT EDIT.
+
+// templ: version: v0.3.833
+package app
+
+//lint:file-ignore SA4006 This context is only used if a nested component is present.
+
+import "github.com/a-h/templ"
+import templruntime "github.com/a-h/templ/runtime"
+
+import "fmt"
+import "github.com/seaweedfs/seaweedfs/weed/admin/dash"
+
+func Subscribers(data dash.SubscribersData) templ.Component {
+ return templruntime.GeneratedTemplate(func(templ_7745c5c3_Input templruntime.GeneratedComponentInput) (templ_7745c5c3_Err error) {
+ templ_7745c5c3_W, ctx := templ_7745c5c3_Input.Writer, templ_7745c5c3_Input.Context
+ if templ_7745c5c3_CtxErr := ctx.Err(); templ_7745c5c3_CtxErr != nil {
+ return templ_7745c5c3_CtxErr
+ }
+ templ_7745c5c3_Buffer, templ_7745c5c3_IsBuffer := templruntime.GetBuffer(templ_7745c5c3_W)
+ if !templ_7745c5c3_IsBuffer {
+ defer func() {
+ templ_7745c5c3_BufErr := templruntime.ReleaseBuffer(templ_7745c5c3_Buffer)
+ if templ_7745c5c3_Err == nil {
+ templ_7745c5c3_Err = templ_7745c5c3_BufErr
+ }
+ }()
+ }
+ ctx = templ.InitializeContext(ctx)
+ templ_7745c5c3_Var1 := templ.GetChildren(ctx)
+ if templ_7745c5c3_Var1 == nil {
+ templ_7745c5c3_Var1 = templ.NopComponent
+ }
+ ctx = templ.ClearChildren(ctx)
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 1, "<div class=\"container-fluid\"><div class=\"row\"><div class=\"col-12\"><div class=\"d-flex justify-content-between align-items-center mb-4\"><h1 class=\"h3 mb-0\">Message Queue Subscribers</h1><small class=\"text-muted\">Last updated: ")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var2 string
+ templ_7745c5c3_Var2, templ_7745c5c3_Err = templ.JoinStringErrs(data.LastUpdated.Format("2006-01-02 15:04:05"))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/subscribers.templ`, Line: 12, Col: 107}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var2))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 2, "</small></div><!-- Summary Cards --><div class=\"row mb-4\"><div class=\"col-md-4\"><div class=\"card text-center\"><div class=\"card-body\"><h5 class=\"card-title\">Total Subscribers</h5><h3 class=\"text-primary\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var3 string
+ templ_7745c5c3_Var3, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.TotalSubscribers))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/subscribers.templ`, Line: 21, Col: 98}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var3))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 3, "</h3></div></div></div><div class=\"col-md-4\"><div class=\"card text-center\"><div class=\"card-body\"><h5 class=\"card-title\">Active Subscribers</h5><h3 class=\"text-success\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var4 string
+ templ_7745c5c3_Var4, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.ActiveSubscribers))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/subscribers.templ`, Line: 29, Col: 99}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var4))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 4, "</h3></div></div></div><div class=\"col-md-4\"><div class=\"card text-center\"><div class=\"card-body\"><h5 class=\"card-title\">Inactive Subscribers</h5><h3 class=\"text-warning\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var5 string
+ templ_7745c5c3_Var5, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.TotalSubscribers-data.ActiveSubscribers))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/subscribers.templ`, Line: 37, Col: 123}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var5))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 5, "</h3></div></div></div></div><!-- Subscribers Table --><div class=\"card\"><div class=\"card-header d-flex justify-content-between align-items-center\"><h5 class=\"mb-0\">Subscribers</h5><div><button class=\"btn btn-sm btn-outline-secondary\" onclick=\"exportSubscribersCSV()\"><i class=\"fas fa-download me-1\"></i>Export CSV</button></div></div><div class=\"card-body\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if len(data.Subscribers) == 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 6, "<div class=\"text-center py-4\"><i class=\"fas fa-user-friends fa-3x text-muted mb-3\"></i><h5>No Subscribers Found</h5><p class=\"text-muted\">No message queue subscribers are currently active.</p></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 7, "<div class=\"table-responsive\"><table class=\"table table-striped\" id=\"subscribersTable\"><thead><tr><th>Subscriber Name</th><th>Topic</th><th>Consumer Group</th><th>Status</th><th>Messages Processed</th><th>Last Seen</th><th>Created</th></tr></thead> <tbody>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ for _, subscriber := range data.Subscribers {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 8, "<tr><td><strong>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var6 string
+ templ_7745c5c3_Var6, templ_7745c5c3_Err = templ.JoinStringErrs(subscriber.Name)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/subscribers.templ`, Line: 78, Col: 76}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var6))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 9, "</strong></td><td><span class=\"badge bg-info\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var7 string
+ templ_7745c5c3_Var7, templ_7745c5c3_Err = templ.JoinStringErrs(subscriber.Topic)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/subscribers.templ`, Line: 81, Col: 97}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var7))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 10, "</span></td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var8 string
+ templ_7745c5c3_Var8, templ_7745c5c3_Err = templ.JoinStringErrs(subscriber.ConsumerGroup)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/subscribers.templ`, Line: 83, Col: 77}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var8))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 11, "</td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if subscriber.Status == "active" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 12, "<span class=\"badge bg-success\">Active</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else if subscriber.Status == "inactive" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, "<span class=\"badge bg-warning\">Inactive</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 14, "<span class=\"badge bg-secondary\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var9 string
+ templ_7745c5c3_Var9, templ_7745c5c3_Err = templ.JoinStringErrs(subscriber.Status)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/subscribers.templ`, Line: 90, Col: 107}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var9))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, "</td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var10 string
+ templ_7745c5c3_Var10, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", subscriber.MessageCount))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/subscribers.templ`, Line: 93, Col: 95}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var10))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, "</td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if !subscriber.LastSeen.IsZero() {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 18, "<span class=\"text-muted\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var11 string
+ templ_7745c5c3_Var11, templ_7745c5c3_Err = templ.JoinStringErrs(subscriber.LastSeen.Format("2006-01-02 15:04:05"))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/subscribers.templ`, Line: 96, Col: 131}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var11))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, "<span class=\"text-muted\">Never</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, "</td><td><span class=\"text-muted\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var12 string
+ templ_7745c5c3_Var12, templ_7745c5c3_Err = templ.JoinStringErrs(subscriber.CreatedAt.Format("2006-01-02 15:04:05"))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/subscribers.templ`, Line: 102, Col: 128}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var12))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 22, "</span></td></tr>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 23, "</tbody></table></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, "</div></div></div></div></div><script>\n function exportSubscribersCSV() {\n const table = document.getElementById('subscribersTable');\n if (!table) return;\n \n let csv = 'Subscriber Name,Topic,Consumer Group,Status,Messages Processed,Last Seen,Created\\n';\n \n const rows = table.querySelectorAll('tbody tr');\n rows.forEach(row => {\n const cells = row.querySelectorAll('td');\n if (cells.length >= 7) {\n const rowData = [\n cells[0].querySelector('strong')?.textContent || '',\n cells[1].querySelector('.badge')?.textContent || '',\n cells[2].textContent || '',\n cells[3].querySelector('.badge')?.textContent || '',\n cells[4].textContent || '',\n cells[5].querySelector('span')?.textContent || '',\n cells[6].querySelector('span')?.textContent || ''\n ];\n csv += rowData.map(field => `\"${field.replace(/\"/g, '\"\"')}\"`).join(',') + '\\n';\n }\n });\n \n const blob = new Blob([csv], { type: 'text/csv;charset=utf-8;' });\n const link = document.createElement('a');\n const url = URL.createObjectURL(blob);\n link.setAttribute('href', url);\n link.setAttribute('download', 'subscribers.csv');\n link.style.visibility = 'hidden';\n document.body.appendChild(link);\n link.click();\n document.body.removeChild(link);\n }\n </script>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ return nil
+ })
+}
+
+var _ = templruntime.GeneratedTemplate
diff --git a/weed/admin/view/app/topic_details.templ b/weed/admin/view/app/topic_details.templ
new file mode 100644
index 000000000..f82ba58a8
--- /dev/null
+++ b/weed/admin/view/app/topic_details.templ
@@ -0,0 +1,677 @@
+package app
+
+import "fmt"
+import "github.com/seaweedfs/seaweedfs/weed/admin/dash"
+import "github.com/seaweedfs/seaweedfs/weed/util"
+
+templ TopicDetails(data dash.TopicDetailsData) {
+ <div class="container-fluid">
+ <div class="row">
+ <div class="col-12">
+ <!-- Header -->
+ <div class="d-flex justify-content-between align-items-center mb-4">
+ <div>
+ <nav aria-label="breadcrumb">
+ <ol class="breadcrumb">
+ <li class="breadcrumb-item"><a href="/mq/topics">Topics</a></li>
+ <li class="breadcrumb-item active" aria-current="page">{data.TopicName}</li>
+ </ol>
+ </nav>
+ <h1 class="h3 mb-0">Topic Details: {data.TopicName}</h1>
+ </div>
+ <small class="text-muted">Last updated: {data.LastUpdated.Format("2006-01-02 15:04:05")}</small>
+ </div>
+
+ <!-- Summary Cards -->
+ <div class="row mb-4">
+ <div class="col-md-2">
+ <div class="card text-center">
+ <div class="card-body">
+ <h5 class="card-title">Partitions</h5>
+ <h3 class="text-primary">{fmt.Sprintf("%d", len(data.Partitions))}</h3>
+ </div>
+ </div>
+ </div>
+ <div class="col-md-2">
+ <div class="card text-center">
+ <div class="card-body">
+ <h5 class="card-title">Schema Fields</h5>
+ <h3 class="text-info">{fmt.Sprintf("%d", len(data.Schema))}</h3>
+ </div>
+ </div>
+ </div>
+ <div class="col-md-2">
+ <div class="card text-center">
+ <div class="card-body">
+ <h5 class="card-title">Total Messages</h5>
+ <h3 class="text-success">{fmt.Sprintf("%d", data.MessageCount)}</h3>
+ </div>
+ </div>
+ </div>
+ <div class="col-md-2">
+ <div class="card text-center">
+ <div class="card-body">
+ <h5 class="card-title">Total Size</h5>
+ <h3 class="text-warning">{util.BytesToHumanReadable(uint64(data.TotalSize))}</h3>
+ </div>
+ </div>
+ </div>
+ <div class="col-md-2">
+ <div class="card text-center">
+ <div class="card-body">
+ <h5 class="card-title">Publishers</h5>
+ <h3 class="text-success">{fmt.Sprintf("%d", len(data.Publishers))}</h3>
+ </div>
+ </div>
+ </div>
+ <div class="col-md-2">
+ <div class="card text-center">
+ <div class="card-body">
+ <h5 class="card-title">Subscribers</h5>
+ <h3 class="text-info">{fmt.Sprintf("%d", len(data.Subscribers))}</h3>
+ </div>
+ </div>
+ </div>
+ </div>
+
+ <!-- Consumer Group Offsets Summary -->
+ <div class="row mb-4">
+ <div class="col-md-12">
+ <div class="card text-center">
+ <div class="card-body">
+ <h5 class="card-title">Consumer Group Offsets</h5>
+ <h3 class="text-warning">{fmt.Sprintf("%d", len(data.ConsumerGroupOffsets))}</h3>
+ <p class="text-muted">Saved consumer progress checkpoints</p>
+ </div>
+ </div>
+ </div>
+ </div>
+
+ <!-- Topic Information -->
+ <div class="row mb-4">
+ <div class="col-md-6">
+ <div class="card">
+ <div class="card-header">
+ <h5 class="mb-0">Topic Information</h5>
+ </div>
+ <div class="card-body">
+ <dl class="row">
+ <dt class="col-sm-4">Namespace:</dt>
+ <dd class="col-sm-8">{data.Namespace}</dd>
+ <dt class="col-sm-4">Name:</dt>
+ <dd class="col-sm-8">{data.Name}</dd>
+ <dt class="col-sm-4">Full Name:</dt>
+ <dd class="col-sm-8">{data.TopicName}</dd>
+ <dt class="col-sm-4">Created:</dt>
+ <dd class="col-sm-8">{data.CreatedAt.Format("2006-01-02 15:04:05")}</dd>
+ </dl>
+ </div>
+ </div>
+ </div>
+ <div class="col-md-6">
+ <div class="card">
+ <div class="card-header d-flex justify-content-between align-items-center">
+ <h5 class="mb-0">
+ <i class="fas fa-clock me-2"></i>Retention Policy
+ </h5>
+ <button type="button" class="btn btn-sm btn-outline-primary" onclick="showEditRetentionModal()">
+ <i class="fas fa-edit me-1"></i>Edit
+ </button>
+ </div>
+ <div class="card-body">
+ <dl class="row">
+ <dt class="col-sm-4">Status:</dt>
+ <dd class="col-sm-8">
+ if data.Retention.Enabled {
+ <span class="badge bg-success">Enabled</span>
+ } else {
+ <span class="badge bg-secondary">Disabled</span>
+ }
+ </dd>
+ <dt class="col-sm-4">Duration:</dt>
+ <dd class="col-sm-8">
+ if data.Retention.Enabled {
+ <span class="text-success">
+ {fmt.Sprintf("%d", data.Retention.DisplayValue)} {data.Retention.DisplayUnit}
+ </span>
+ } else {
+ <span class="text-muted">No retention configured</span>
+ }
+ </dd>
+ </dl>
+ </div>
+ </div>
+ </div>
+ </div>
+
+ <!-- Schema Information -->
+ <div class="row mb-4">
+ <div class="col-12">
+ <div class="card">
+ <div class="card-header">
+ <h5 class="mb-0">Schema Definition</h5>
+ </div>
+ <div class="card-body">
+ if len(data.Schema) == 0 {
+ <p class="text-muted">No schema information available</p>
+ } else {
+ <div class="table-responsive">
+ <table class="table table-sm">
+ <thead>
+ <tr>
+ <th>Field</th>
+ <th>Type</th>
+ <th>Required</th>
+ </tr>
+ </thead>
+ <tbody>
+ for _, field := range data.Schema {
+ <tr>
+ <td><code>{field.Name}</code></td>
+ <td><span class="badge bg-secondary">{field.Type}</span></td>
+ <td>
+ if field.Required {
+ <i class="fas fa-check text-success"></i>
+ } else {
+ <i class="fas fa-times text-muted"></i>
+ }
+ </td>
+ </tr>
+ }
+ </tbody>
+ </table>
+ </div>
+ }
+ </div>
+ </div>
+ </div>
+ </div>
+
+ <!-- Partitions Table -->
+ <div class="card">
+ <div class="card-header d-flex justify-content-between align-items-center">
+ <h5 class="mb-0">Partitions</h5>
+ <div>
+ <button class="btn btn-sm btn-outline-secondary" onclick="exportPartitionsCSV()">
+ <i class="fas fa-download me-1"></i>Export CSV
+ </button>
+ </div>
+ </div>
+ <div class="card-body">
+ if len(data.Partitions) == 0 {
+ <div class="text-center py-4">
+ <i class="fas fa-server fa-3x text-muted mb-3"></i>
+ <h5>No Partitions Found</h5>
+ <p class="text-muted">No partitions are configured for this topic.</p>
+ </div>
+ } else {
+ <div class="table-responsive">
+ <table class="table table-striped" id="partitionsTable">
+ <thead>
+ <tr>
+ <th>Partition ID</th>
+ <th>Leader Broker</th>
+ <th>Follower Broker</th>
+ <th>Messages</th>
+ <th>Size</th>
+ <th>Last Data Time</th>
+ <th>Created</th>
+ </tr>
+ </thead>
+ <tbody>
+ for _, partition := range data.Partitions {
+ <tr>
+ <td>
+ <span class="badge bg-primary">{fmt.Sprintf("%d", partition.ID)}</span>
+ </td>
+ <td>
+ <strong>{partition.LeaderBroker}</strong>
+ </td>
+ <td>
+ if partition.FollowerBroker != "" {
+ <span class="text-muted">{partition.FollowerBroker}</span>
+ } else {
+ <span class="text-muted">None</span>
+ }
+ </td>
+ <td>{fmt.Sprintf("%d", partition.MessageCount)}</td>
+ <td>{util.BytesToHumanReadable(uint64(partition.TotalSize))}</td>
+ <td>
+ if !partition.LastDataTime.IsZero() {
+ <span class="text-muted">{partition.LastDataTime.Format("2006-01-02 15:04:05")}</span>
+ } else {
+ <span class="text-muted">Never</span>
+ }
+ </td>
+ <td>
+ <span class="text-muted">{partition.CreatedAt.Format("2006-01-02 15:04:05")}</span>
+ </td>
+ </tr>
+ }
+ </tbody>
+ </table>
+ </div>
+ }
+ </div>
+ </div>
+
+ <!-- Publishers and Subscribers -->
+ <div class="row mb-4">
+ <div class="col-12">
+ <div class="card">
+ <div class="card-header">
+ <h5 class="mb-0">Active Publishers <span class="badge bg-success">{fmt.Sprintf("%d", len(data.Publishers))}</span></h5>
+ </div>
+ <div class="card-body">
+ if len(data.Publishers) == 0 {
+ <div class="alert alert-info mb-0">
+ <i class="fas fa-info-circle"></i> No active publishers found for this topic.
+ </div>
+ } else {
+ <div class="table-responsive">
+ <table class="table table-sm">
+ <thead>
+ <tr>
+ <th>Publisher</th>
+ <th>Partition</th>
+ <th>Broker</th>
+ <th>Status</th>
+ <th>Published</th>
+ <th>Acknowledged</th>
+ <th>Last Seen</th>
+ </tr>
+ </thead>
+ <tbody>
+ for _, publisher := range data.Publishers {
+ <tr>
+ <td>{publisher.PublisherName}</td>
+ <td><span class="badge bg-primary">{fmt.Sprintf("%d", publisher.PartitionID)}</span></td>
+ <td>{publisher.Broker}</td>
+ <td>
+ if publisher.IsActive {
+ <span class="badge bg-success">Active</span>
+ } else {
+ <span class="badge bg-secondary">Inactive</span>
+ }
+ </td>
+ <td>
+ if publisher.LastPublishedOffset > 0 {
+ <span class="text-muted">{fmt.Sprintf("%d", publisher.LastPublishedOffset)}</span>
+ } else {
+ <span class="text-muted">-</span>
+ }
+ </td>
+ <td>
+ if publisher.LastAckedOffset > 0 {
+ <span class="text-muted">{fmt.Sprintf("%d", publisher.LastAckedOffset)}</span>
+ } else {
+ <span class="text-muted">-</span>
+ }
+ </td>
+ <td>
+ if !publisher.LastSeenTime.IsZero() {
+ <span class="text-muted">{publisher.LastSeenTime.Format("15:04:05")}</span>
+ } else {
+ <span class="text-muted">-</span>
+ }
+ </td>
+ </tr>
+ }
+ </tbody>
+ </table>
+ </div>
+ }
+ </div>
+ </div>
+ </div>
+ </div>
+
+ <div class="row mb-4">
+ <div class="col-12">
+ <div class="card">
+ <div class="card-header">
+ <h5 class="mb-0">Active Subscribers <span class="badge bg-info">{fmt.Sprintf("%d", len(data.Subscribers))}</span></h5>
+ </div>
+ <div class="card-body">
+ if len(data.Subscribers) == 0 {
+ <div class="alert alert-info mb-0">
+ <i class="fas fa-info-circle"></i> No active subscribers found for this topic.
+ </div>
+ } else {
+ <div class="table-responsive">
+ <table class="table table-sm">
+ <thead>
+ <tr>
+ <th>Consumer Group</th>
+ <th>Consumer ID</th>
+ <th>Partition</th>
+ <th>Broker</th>
+ <th>Status</th>
+ <th>Received</th>
+ <th>Acknowledged</th>
+ <th>Last Seen</th>
+ </tr>
+ </thead>
+ <tbody>
+ for _, subscriber := range data.Subscribers {
+ <tr>
+ <td>{subscriber.ConsumerGroup}</td>
+ <td>{subscriber.ConsumerID}</td>
+ <td><span class="badge bg-primary">{fmt.Sprintf("%d", subscriber.PartitionID)}</span></td>
+ <td>{subscriber.Broker}</td>
+ <td>
+ if subscriber.IsActive {
+ <span class="badge bg-success">Active</span>
+ } else {
+ <span class="badge bg-secondary">Inactive</span>
+ }
+ </td>
+ <td>
+ if subscriber.LastReceivedOffset > 0 {
+ <span class="text-muted">{fmt.Sprintf("%d", subscriber.LastReceivedOffset)}</span>
+ } else {
+ <span class="text-muted">-</span>
+ }
+ </td>
+ <td>
+ if subscriber.CurrentOffset > 0 {
+ <span class="text-muted">{fmt.Sprintf("%d", subscriber.CurrentOffset)}</span>
+ } else {
+ <span class="text-muted">-</span>
+ }
+ </td>
+ <td>
+ if !subscriber.LastSeenTime.IsZero() {
+ <span class="text-muted">{subscriber.LastSeenTime.Format("15:04:05")}</span>
+ } else {
+ <span class="text-muted">-</span>
+ }
+ </td>
+ </tr>
+ }
+ </tbody>
+ </table>
+ </div>
+ }
+ </div>
+ </div>
+ </div>
+ </div>
+
+ <!-- Consumer Group Offsets -->
+ <div class="row mb-4">
+ <div class="col-12">
+ <div class="card">
+ <div class="card-header">
+ <h5 class="mb-0">Consumer Group Offsets <span class="badge bg-warning">{fmt.Sprintf("%d", len(data.ConsumerGroupOffsets))}</span></h5>
+ </div>
+ <div class="card-body">
+ if len(data.ConsumerGroupOffsets) == 0 {
+ <div class="alert alert-info mb-0">
+ <i class="fas fa-info-circle"></i> No consumer group offsets found for this topic.
+ </div>
+ } else {
+ <div class="table-responsive">
+ <table class="table table-sm">
+ <thead>
+ <tr>
+ <th>Consumer Group</th>
+ <th>Partition</th>
+ <th>Offset</th>
+ <th>Last Updated</th>
+ </tr>
+ </thead>
+ <tbody>
+ for _, offset := range data.ConsumerGroupOffsets {
+ <tr>
+ <td>
+ <span class="badge bg-secondary">{offset.ConsumerGroup}</span>
+ </td>
+ <td>
+ <span class="badge bg-primary">{fmt.Sprintf("%d", offset.PartitionID)}</span>
+ </td>
+ <td>
+ <strong>{fmt.Sprintf("%d", offset.Offset)}</strong>
+ </td>
+ <td>
+ <span class="text-muted">{offset.LastUpdated.Format("2006-01-02 15:04:05")}</span>
+ </td>
+ </tr>
+ }
+ </tbody>
+ </table>
+ </div>
+ }
+ </div>
+ </div>
+ </div>
+ </div>
+ </div>
+ </div>
+ </div>
+
+ <script>
+ function exportPartitionsCSV() {
+ const table = document.getElementById('partitionsTable');
+ if (!table) return;
+
+ let csv = 'Partition ID,Leader Broker,Follower Broker,Messages,Size,Last Data Time,Created\n';
+
+ const rows = table.querySelectorAll('tbody tr');
+ rows.forEach(row => {
+ const cells = row.querySelectorAll('td');
+ if (cells.length >= 7) {
+ const rowData = [
+ cells[0].querySelector('.badge')?.textContent || '',
+ cells[1].querySelector('strong')?.textContent || '',
+ cells[2].textContent || '',
+ cells[3].textContent || '',
+ cells[4].textContent || '',
+ cells[5].querySelector('span')?.textContent || '',
+ cells[6].querySelector('span')?.textContent || ''
+ ];
+ csv += rowData.map(field => `"${field.replace(/"/g, '""')}"`).join(',') + '\n';
+ }
+ });
+
+ const blob = new Blob([csv], { type: 'text/csv;charset=utf-8;' });
+ const link = document.createElement('a');
+ const url = URL.createObjectURL(blob);
+ link.setAttribute('href', url);
+ link.setAttribute('download', 'topic_partitions.csv');
+ link.style.visibility = 'hidden';
+ document.body.appendChild(link);
+ link.click();
+ document.body.removeChild(link);
+ }
+
+ // Edit retention functions
+ function showEditRetentionModal() {
+ const modal = new bootstrap.Modal(document.getElementById('editRetentionModal'));
+
+ // Get current retention values from the page
+ const currentEnabled = document.querySelector('dd .badge.bg-success') !== null;
+ const currentDurationElement = document.querySelector('dd .text-success');
+
+ let currentValue = 7;
+ let currentUnit = 'days';
+
+ if (currentEnabled && currentDurationElement) {
+ const durationText = currentDurationElement.textContent.trim();
+ const parts = durationText.split(' ');
+ if (parts.length >= 2) {
+ currentValue = parseInt(parts[0]) || 7;
+ currentUnit = parts[1].toLowerCase();
+ // Handle plural forms
+ if (currentUnit.endsWith('s')) {
+ currentUnit = currentUnit.slice(0, -1);
+ }
+ // Map to our dropdown values
+ if (currentUnit === 'hour') {
+ currentUnit = 'hours';
+ } else if (currentUnit === 'day') {
+ currentUnit = 'days';
+ }
+ }
+ }
+
+ // Set current values in the modal
+ document.getElementById('editEnableRetention').checked = currentEnabled;
+ document.getElementById('editRetentionValue').value = currentValue;
+ document.getElementById('editRetentionUnit').value = currentUnit;
+
+ // Show/hide retention fields based on current state
+ toggleEditRetentionFields();
+
+ modal.show();
+ }
+
+ function toggleEditRetentionFields() {
+ const enableRetention = document.getElementById('editEnableRetention');
+ const retentionFields = document.getElementById('editRetentionFields');
+
+ if (enableRetention.checked) {
+ retentionFields.style.display = 'block';
+ } else {
+ retentionFields.style.display = 'none';
+ }
+ }
+
+ function updateRetention() {
+ const form = document.getElementById('editRetentionForm');
+ const formData = new FormData(form);
+
+ // Get topic details from the page
+ const topicName = document.querySelector('h1').textContent.replace('Topic Details: ', '');
+ const parts = topicName.split('.');
+
+ if (parts.length < 2) {
+ alert('Invalid topic name format');
+ return;
+ }
+
+ const namespace = parts[0];
+ const name = parts.slice(1).join('.');
+
+ // Convert form data to JSON
+ const data = {
+ namespace: namespace,
+ name: name,
+ retention: {
+ enabled: formData.get('editEnableRetention') === 'on',
+ retention_seconds: 0
+ }
+ };
+
+ // Calculate retention seconds if enabled
+ if (data.retention.enabled) {
+ const retentionValue = parseInt(formData.get('editRetentionValue'));
+ const retentionUnit = formData.get('editRetentionUnit');
+
+ if (retentionUnit === 'hours') {
+ data.retention.retention_seconds = retentionValue * 3600;
+ } else if (retentionUnit === 'days') {
+ data.retention.retention_seconds = retentionValue * 86400;
+ }
+ }
+
+ // Show loading state
+ const updateButton = document.querySelector('#editRetentionModal .btn-primary');
+ updateButton.disabled = true;
+ updateButton.innerHTML = '<i class="fas fa-spinner fa-spin me-1"></i>Updating...';
+
+ // Send API request
+ fetch('/api/mq/topics/retention/update', {
+ method: 'POST',
+ headers: {
+ 'Content-Type': 'application/json',
+ },
+ body: JSON.stringify(data)
+ })
+ .then(response => response.json())
+ .then(result => {
+ if (result.error) {
+ alert('Failed to update retention: ' + result.error);
+ } else {
+ alert('Retention policy updated successfully!');
+ // Close modal and refresh page
+ const modal = bootstrap.Modal.getInstance(document.getElementById('editRetentionModal'));
+ modal.hide();
+ window.location.reload();
+ }
+ })
+ .catch(error => {
+ alert('Failed to update retention: ' + error.message);
+ })
+ .finally(() => {
+ // Reset button state
+ updateButton.disabled = false;
+ updateButton.innerHTML = '<i class="fas fa-save me-1"></i>Update Retention';
+ });
+ }
+ </script>
+
+ <!-- Edit Retention Modal -->
+ <div class="modal fade" id="editRetentionModal" tabindex="-1" role="dialog">
+ <div class="modal-dialog modal-lg" role="document">
+ <div class="modal-content">
+ <div class="modal-header">
+ <h5 class="modal-title">
+ <i class="fas fa-edit me-2"></i>Edit Retention Policy
+ </h5>
+ <button type="button" class="btn-close" data-bs-dismiss="modal"></button>
+ </div>
+ <div class="modal-body">
+ <form id="editRetentionForm">
+ <div class="card">
+ <div class="card-header">
+ <h6 class="mb-0">
+ <i class="fas fa-clock me-2"></i>Retention Configuration
+ </h6>
+ </div>
+ <div class="card-body">
+ <div class="form-check mb-3">
+ <input class="form-check-input" type="checkbox" id="editEnableRetention"
+ name="editEnableRetention" onchange="toggleEditRetentionFields()">
+ <label class="form-check-label" for="editEnableRetention">
+ Enable data retention
+ </label>
+ </div>
+ <div id="editRetentionFields" style="display: none;">
+ <div class="row">
+ <div class="col-md-6">
+ <div class="mb-3">
+ <label for="editRetentionValue" class="form-label">Retention Duration</label>
+ <input type="number" class="form-control" id="editRetentionValue"
+ name="editRetentionValue" min="1" value="7">
+ </div>
+ </div>
+ <div class="col-md-6">
+ <div class="mb-3">
+ <label for="editRetentionUnit" class="form-label">Unit</label>
+ <select class="form-control" id="editRetentionUnit" name="editRetentionUnit">
+ <option value="hours">Hours</option>
+ <option value="days" selected>Days</option>
+ </select>
+ </div>
+ </div>
+ </div>
+ <div class="alert alert-info">
+ <i class="fas fa-info-circle me-2"></i>
+ Data older than this duration will be automatically purged to save storage space.
+ </div>
+ </div>
+ </div>
+ </div>
+ </form>
+ </div>
+ <div class="modal-footer">
+ <button type="button" class="btn btn-secondary" data-bs-dismiss="modal">Cancel</button>
+ <button type="button" class="btn btn-primary" onclick="updateRetention()">
+ <i class="fas fa-save me-1"></i>Update Retention
+ </button>
+ </div>
+ </div>
+ </div>
+ </div>
+} \ No newline at end of file
diff --git a/weed/admin/view/app/topic_details_templ.go b/weed/admin/view/app/topic_details_templ.go
new file mode 100644
index 000000000..4dd53583d
--- /dev/null
+++ b/weed/admin/view/app/topic_details_templ.go
@@ -0,0 +1,949 @@
+// Code generated by templ - DO NOT EDIT.
+
+// templ: version: v0.3.833
+package app
+
+//lint:file-ignore SA4006 This context is only used if a nested component is present.
+
+import "github.com/a-h/templ"
+import templruntime "github.com/a-h/templ/runtime"
+
+import "fmt"
+import "github.com/seaweedfs/seaweedfs/weed/admin/dash"
+import "github.com/seaweedfs/seaweedfs/weed/util"
+
+func TopicDetails(data dash.TopicDetailsData) templ.Component {
+ return templruntime.GeneratedTemplate(func(templ_7745c5c3_Input templruntime.GeneratedComponentInput) (templ_7745c5c3_Err error) {
+ templ_7745c5c3_W, ctx := templ_7745c5c3_Input.Writer, templ_7745c5c3_Input.Context
+ if templ_7745c5c3_CtxErr := ctx.Err(); templ_7745c5c3_CtxErr != nil {
+ return templ_7745c5c3_CtxErr
+ }
+ templ_7745c5c3_Buffer, templ_7745c5c3_IsBuffer := templruntime.GetBuffer(templ_7745c5c3_W)
+ if !templ_7745c5c3_IsBuffer {
+ defer func() {
+ templ_7745c5c3_BufErr := templruntime.ReleaseBuffer(templ_7745c5c3_Buffer)
+ if templ_7745c5c3_Err == nil {
+ templ_7745c5c3_Err = templ_7745c5c3_BufErr
+ }
+ }()
+ }
+ ctx = templ.InitializeContext(ctx)
+ templ_7745c5c3_Var1 := templ.GetChildren(ctx)
+ if templ_7745c5c3_Var1 == nil {
+ templ_7745c5c3_Var1 = templ.NopComponent
+ }
+ ctx = templ.ClearChildren(ctx)
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 1, "<div class=\"container-fluid\"><div class=\"row\"><div class=\"col-12\"><!-- Header --><div class=\"d-flex justify-content-between align-items-center mb-4\"><div><nav aria-label=\"breadcrumb\"><ol class=\"breadcrumb\"><li class=\"breadcrumb-item\"><a href=\"/mq/topics\">Topics</a></li><li class=\"breadcrumb-item active\" aria-current=\"page\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var2 string
+ templ_7745c5c3_Var2, templ_7745c5c3_Err = templ.JoinStringErrs(data.TopicName)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 17, Col: 102}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var2))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 2, "</li></ol></nav><h1 class=\"h3 mb-0\">Topic Details: ")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var3 string
+ templ_7745c5c3_Var3, templ_7745c5c3_Err = templ.JoinStringErrs(data.TopicName)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 20, Col: 74}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var3))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 3, "</h1></div><small class=\"text-muted\">Last updated: ")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var4 string
+ templ_7745c5c3_Var4, templ_7745c5c3_Err = templ.JoinStringErrs(data.LastUpdated.Format("2006-01-02 15:04:05"))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 22, Col: 107}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var4))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 4, "</small></div><!-- Summary Cards --><div class=\"row mb-4\"><div class=\"col-md-2\"><div class=\"card text-center\"><div class=\"card-body\"><h5 class=\"card-title\">Partitions</h5><h3 class=\"text-primary\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var5 string
+ templ_7745c5c3_Var5, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", len(data.Partitions)))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 31, Col: 97}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var5))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 5, "</h3></div></div></div><div class=\"col-md-2\"><div class=\"card text-center\"><div class=\"card-body\"><h5 class=\"card-title\">Schema Fields</h5><h3 class=\"text-info\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var6 string
+ templ_7745c5c3_Var6, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", len(data.Schema)))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 39, Col: 90}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var6))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 6, "</h3></div></div></div><div class=\"col-md-2\"><div class=\"card text-center\"><div class=\"card-body\"><h5 class=\"card-title\">Total Messages</h5><h3 class=\"text-success\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var7 string
+ templ_7745c5c3_Var7, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.MessageCount))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 47, Col: 94}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var7))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 7, "</h3></div></div></div><div class=\"col-md-2\"><div class=\"card text-center\"><div class=\"card-body\"><h5 class=\"card-title\">Total Size</h5><h3 class=\"text-warning\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var8 string
+ templ_7745c5c3_Var8, templ_7745c5c3_Err = templ.JoinStringErrs(util.BytesToHumanReadable(uint64(data.TotalSize)))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 55, Col: 107}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var8))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 8, "</h3></div></div></div><div class=\"col-md-2\"><div class=\"card text-center\"><div class=\"card-body\"><h5 class=\"card-title\">Publishers</h5><h3 class=\"text-success\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var9 string
+ templ_7745c5c3_Var9, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", len(data.Publishers)))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 63, Col: 97}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var9))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 9, "</h3></div></div></div><div class=\"col-md-2\"><div class=\"card text-center\"><div class=\"card-body\"><h5 class=\"card-title\">Subscribers</h5><h3 class=\"text-info\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var10 string
+ templ_7745c5c3_Var10, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", len(data.Subscribers)))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 71, Col: 95}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var10))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 10, "</h3></div></div></div></div><!-- Consumer Group Offsets Summary --><div class=\"row mb-4\"><div class=\"col-md-12\"><div class=\"card text-center\"><div class=\"card-body\"><h5 class=\"card-title\">Consumer Group Offsets</h5><h3 class=\"text-warning\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var11 string
+ templ_7745c5c3_Var11, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", len(data.ConsumerGroupOffsets)))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 83, Col: 107}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var11))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 11, "</h3><p class=\"text-muted\">Saved consumer progress checkpoints</p></div></div></div></div><!-- Topic Information --><div class=\"row mb-4\"><div class=\"col-md-6\"><div class=\"card\"><div class=\"card-header\"><h5 class=\"mb-0\">Topic Information</h5></div><div class=\"card-body\"><dl class=\"row\"><dt class=\"col-sm-4\">Namespace:</dt><dd class=\"col-sm-8\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var12 string
+ templ_7745c5c3_Var12, templ_7745c5c3_Err = templ.JoinStringErrs(data.Namespace)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 100, Col: 72}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var12))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 12, "</dd><dt class=\"col-sm-4\">Name:</dt><dd class=\"col-sm-8\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var13 string
+ templ_7745c5c3_Var13, templ_7745c5c3_Err = templ.JoinStringErrs(data.Name)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 102, Col: 67}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var13))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, "</dd><dt class=\"col-sm-4\">Full Name:</dt><dd class=\"col-sm-8\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var14 string
+ templ_7745c5c3_Var14, templ_7745c5c3_Err = templ.JoinStringErrs(data.TopicName)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 104, Col: 72}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var14))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 14, "</dd><dt class=\"col-sm-4\">Created:</dt><dd class=\"col-sm-8\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var15 string
+ templ_7745c5c3_Var15, templ_7745c5c3_Err = templ.JoinStringErrs(data.CreatedAt.Format("2006-01-02 15:04:05"))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 106, Col: 102}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var15))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "</dd></dl></div></div></div><div class=\"col-md-6\"><div class=\"card\"><div class=\"card-header d-flex justify-content-between align-items-center\"><h5 class=\"mb-0\"><i class=\"fas fa-clock me-2\"></i>Retention Policy</h5><button type=\"button\" class=\"btn btn-sm btn-outline-primary\" onclick=\"showEditRetentionModal()\"><i class=\"fas fa-edit me-1\"></i>Edit</button></div><div class=\"card-body\"><dl class=\"row\"><dt class=\"col-sm-4\">Status:</dt><dd class=\"col-sm-8\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if data.Retention.Enabled {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, "<span class=\"badge bg-success\">Enabled</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, "<span class=\"badge bg-secondary\">Disabled</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 18, "</dd><dt class=\"col-sm-4\">Duration:</dt><dd class=\"col-sm-8\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if data.Retention.Enabled {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, "<span class=\"text-success\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var16 string
+ templ_7745c5c3_Var16, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.Retention.DisplayValue))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 135, Col: 95}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var16))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, " ")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var17 string
+ templ_7745c5c3_Var17, templ_7745c5c3_Err = templ.JoinStringErrs(data.Retention.DisplayUnit)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 135, Col: 124}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var17))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 22, "<span class=\"text-muted\">No retention configured</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 23, "</dd></dl></div></div></div></div><!-- Schema Information --><div class=\"row mb-4\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header\"><h5 class=\"mb-0\">Schema Definition</h5></div><div class=\"card-body\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if len(data.Schema) == 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, "<p class=\"text-muted\">No schema information available</p>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 25, "<div class=\"table-responsive\"><table class=\"table table-sm\"><thead><tr><th>Field</th><th>Type</th><th>Required</th></tr></thead> <tbody>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ for _, field := range data.Schema {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, "<tr><td><code>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var18 string
+ templ_7745c5c3_Var18, templ_7745c5c3_Err = templ.JoinStringErrs(field.Name)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 170, Col: 77}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var18))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, "</code></td><td><span class=\"badge bg-secondary\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var19 string
+ templ_7745c5c3_Var19, templ_7745c5c3_Err = templ.JoinStringErrs(field.Type)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 171, Col: 104}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var19))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, "</span></td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if field.Required {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, "<i class=\"fas fa-check text-success\"></i>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 30, "<i class=\"fas fa-times text-muted\"></i>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 31, "</td></tr>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 32, "</tbody></table></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 33, "</div></div></div></div><!-- Partitions Table --><div class=\"card\"><div class=\"card-header d-flex justify-content-between align-items-center\"><h5 class=\"mb-0\">Partitions</h5><div><button class=\"btn btn-sm btn-outline-secondary\" onclick=\"exportPartitionsCSV()\"><i class=\"fas fa-download me-1\"></i>Export CSV</button></div></div><div class=\"card-body\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if len(data.Partitions) == 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 34, "<div class=\"text-center py-4\"><i class=\"fas fa-server fa-3x text-muted mb-3\"></i><h5>No Partitions Found</h5><p class=\"text-muted\">No partitions are configured for this topic.</p></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 35, "<div class=\"table-responsive\"><table class=\"table table-striped\" id=\"partitionsTable\"><thead><tr><th>Partition ID</th><th>Leader Broker</th><th>Follower Broker</th><th>Messages</th><th>Size</th><th>Last Data Time</th><th>Created</th></tr></thead> <tbody>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ for _, partition := range data.Partitions {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 36, "<tr><td><span class=\"badge bg-primary\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var20 string
+ templ_7745c5c3_Var20, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", partition.ID))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 225, Col: 115}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var20))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 37, "</span></td><td><strong>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var21 string
+ templ_7745c5c3_Var21, templ_7745c5c3_Err = templ.JoinStringErrs(partition.LeaderBroker)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 228, Col: 83}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var21))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 38, "</strong></td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if partition.FollowerBroker != "" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 39, "<span class=\"text-muted\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var22 string
+ templ_7745c5c3_Var22, templ_7745c5c3_Err = templ.JoinStringErrs(partition.FollowerBroker)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 232, Col: 106}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var22))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 40, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 41, "<span class=\"text-muted\">None</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 42, "</td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var23 string
+ templ_7745c5c3_Var23, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", partition.MessageCount))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 237, Col: 94}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var23))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 43, "</td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var24 string
+ templ_7745c5c3_Var24, templ_7745c5c3_Err = templ.JoinStringErrs(util.BytesToHumanReadable(uint64(partition.TotalSize)))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 238, Col: 107}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var24))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 44, "</td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if !partition.LastDataTime.IsZero() {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 45, "<span class=\"text-muted\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var25 string
+ templ_7745c5c3_Var25, templ_7745c5c3_Err = templ.JoinStringErrs(partition.LastDataTime.Format("2006-01-02 15:04:05"))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 241, Col: 134}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var25))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 46, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 47, "<span class=\"text-muted\">Never</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 48, "</td><td><span class=\"text-muted\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var26 string
+ templ_7745c5c3_Var26, templ_7745c5c3_Err = templ.JoinStringErrs(partition.CreatedAt.Format("2006-01-02 15:04:05"))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 247, Col: 127}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var26))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 49, "</span></td></tr>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 50, "</tbody></table></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 51, "</div></div><!-- Publishers and Subscribers --><div class=\"row mb-4\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header\"><h5 class=\"mb-0\">Active Publishers <span class=\"badge bg-success\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var27 string
+ templ_7745c5c3_Var27, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", len(data.Publishers)))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 263, Col: 138}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var27))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 52, "</span></h5></div><div class=\"card-body\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if len(data.Publishers) == 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 53, "<div class=\"alert alert-info mb-0\"><i class=\"fas fa-info-circle\"></i> No active publishers found for this topic.</div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 54, "<div class=\"table-responsive\"><table class=\"table table-sm\"><thead><tr><th>Publisher</th><th>Partition</th><th>Broker</th><th>Status</th><th>Published</th><th>Acknowledged</th><th>Last Seen</th></tr></thead> <tbody>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ for _, publisher := range data.Publishers {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 55, "<tr><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var28 string
+ templ_7745c5c3_Var28, templ_7745c5c3_Err = templ.JoinStringErrs(publisher.PublisherName)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 287, Col: 84}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var28))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 56, "</td><td><span class=\"badge bg-primary\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var29 string
+ templ_7745c5c3_Var29, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", publisher.PartitionID))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 288, Col: 132}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var29))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 57, "</span></td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var30 string
+ templ_7745c5c3_Var30, templ_7745c5c3_Err = templ.JoinStringErrs(publisher.Broker)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 289, Col: 77}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var30))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 58, "</td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if publisher.IsActive {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 59, "<span class=\"badge bg-success\">Active</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 60, "<span class=\"badge bg-secondary\">Inactive</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 61, "</td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if publisher.LastPublishedOffset > 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 62, "<span class=\"text-muted\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var31 string
+ templ_7745c5c3_Var31, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", publisher.LastPublishedOffset))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 299, Col: 138}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var31))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 63, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 64, "<span class=\"text-muted\">-</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 65, "</td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if publisher.LastAckedOffset > 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 66, "<span class=\"text-muted\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var32 string
+ templ_7745c5c3_Var32, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", publisher.LastAckedOffset))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 306, Col: 134}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var32))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 67, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 68, "<span class=\"text-muted\">-</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 69, "</td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if !publisher.LastSeenTime.IsZero() {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 70, "<span class=\"text-muted\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var33 string
+ templ_7745c5c3_Var33, templ_7745c5c3_Err = templ.JoinStringErrs(publisher.LastSeenTime.Format("15:04:05"))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 313, Col: 131}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var33))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 71, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 72, "<span class=\"text-muted\">-</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 73, "</td></tr>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 74, "</tbody></table></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 75, "</div></div></div></div><div class=\"row mb-4\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header\"><h5 class=\"mb-0\">Active Subscribers <span class=\"badge bg-info\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var34 string
+ templ_7745c5c3_Var34, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", len(data.Subscribers)))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 333, Col: 137}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var34))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 76, "</span></h5></div><div class=\"card-body\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if len(data.Subscribers) == 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 77, "<div class=\"alert alert-info mb-0\"><i class=\"fas fa-info-circle\"></i> No active subscribers found for this topic.</div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 78, "<div class=\"table-responsive\"><table class=\"table table-sm\"><thead><tr><th>Consumer Group</th><th>Consumer ID</th><th>Partition</th><th>Broker</th><th>Status</th><th>Received</th><th>Acknowledged</th><th>Last Seen</th></tr></thead> <tbody>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ for _, subscriber := range data.Subscribers {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 79, "<tr><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var35 string
+ templ_7745c5c3_Var35, templ_7745c5c3_Err = templ.JoinStringErrs(subscriber.ConsumerGroup)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 358, Col: 85}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var35))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 80, "</td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var36 string
+ templ_7745c5c3_Var36, templ_7745c5c3_Err = templ.JoinStringErrs(subscriber.ConsumerID)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 359, Col: 82}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var36))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 81, "</td><td><span class=\"badge bg-primary\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var37 string
+ templ_7745c5c3_Var37, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", subscriber.PartitionID))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 360, Col: 133}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var37))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 82, "</span></td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var38 string
+ templ_7745c5c3_Var38, templ_7745c5c3_Err = templ.JoinStringErrs(subscriber.Broker)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 361, Col: 78}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var38))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 83, "</td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if subscriber.IsActive {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 84, "<span class=\"badge bg-success\">Active</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 85, "<span class=\"badge bg-secondary\">Inactive</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 86, "</td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if subscriber.LastReceivedOffset > 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 87, "<span class=\"text-muted\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var39 string
+ templ_7745c5c3_Var39, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", subscriber.LastReceivedOffset))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 371, Col: 138}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var39))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 88, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 89, "<span class=\"text-muted\">-</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 90, "</td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if subscriber.CurrentOffset > 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 91, "<span class=\"text-muted\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var40 string
+ templ_7745c5c3_Var40, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", subscriber.CurrentOffset))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 378, Col: 133}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var40))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 92, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 93, "<span class=\"text-muted\">-</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 94, "</td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if !subscriber.LastSeenTime.IsZero() {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 95, "<span class=\"text-muted\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var41 string
+ templ_7745c5c3_Var41, templ_7745c5c3_Err = templ.JoinStringErrs(subscriber.LastSeenTime.Format("15:04:05"))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 385, Col: 132}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var41))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 96, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 97, "<span class=\"text-muted\">-</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 98, "</td></tr>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 99, "</tbody></table></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 100, "</div></div></div></div><!-- Consumer Group Offsets --><div class=\"row mb-4\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header\"><h5 class=\"mb-0\">Consumer Group Offsets <span class=\"badge bg-warning\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var42 string
+ templ_7745c5c3_Var42, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", len(data.ConsumerGroupOffsets)))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 406, Col: 153}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var42))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 101, "</span></h5></div><div class=\"card-body\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if len(data.ConsumerGroupOffsets) == 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 102, "<div class=\"alert alert-info mb-0\"><i class=\"fas fa-info-circle\"></i> No consumer group offsets found for this topic.</div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 103, "<div class=\"table-responsive\"><table class=\"table table-sm\"><thead><tr><th>Consumer Group</th><th>Partition</th><th>Offset</th><th>Last Updated</th></tr></thead> <tbody>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ for _, offset := range data.ConsumerGroupOffsets {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 104, "<tr><td><span class=\"badge bg-secondary\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var43 string
+ templ_7745c5c3_Var43, templ_7745c5c3_Err = templ.JoinStringErrs(offset.ConsumerGroup)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 428, Col: 114}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var43))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 105, "</span></td><td><span class=\"badge bg-primary\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var44 string
+ templ_7745c5c3_Var44, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", offset.PartitionID))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 431, Col: 129}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var44))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 106, "</span></td><td><strong>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var45 string
+ templ_7745c5c3_Var45, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", offset.Offset))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 434, Col: 101}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var45))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 107, "</strong></td><td><span class=\"text-muted\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var46 string
+ templ_7745c5c3_Var46, templ_7745c5c3_Err = templ.JoinStringErrs(offset.LastUpdated.Format("2006-01-02 15:04:05"))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topic_details.templ`, Line: 437, Col: 134}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var46))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 108, "</span></td></tr>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 109, "</tbody></table></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 110, "</div></div></div></div></div></div></div><script>\n function exportPartitionsCSV() {\n const table = document.getElementById('partitionsTable');\n if (!table) return;\n \n let csv = 'Partition ID,Leader Broker,Follower Broker,Messages,Size,Last Data Time,Created\\n';\n \n const rows = table.querySelectorAll('tbody tr');\n rows.forEach(row => {\n const cells = row.querySelectorAll('td');\n if (cells.length >= 7) {\n const rowData = [\n cells[0].querySelector('.badge')?.textContent || '',\n cells[1].querySelector('strong')?.textContent || '',\n cells[2].textContent || '',\n cells[3].textContent || '',\n cells[4].textContent || '',\n cells[5].querySelector('span')?.textContent || '',\n cells[6].querySelector('span')?.textContent || ''\n ];\n csv += rowData.map(field => `\"${field.replace(/\"/g, '\"\"')}\"`).join(',') + '\\n';\n }\n });\n \n const blob = new Blob([csv], { type: 'text/csv;charset=utf-8;' });\n const link = document.createElement('a');\n const url = URL.createObjectURL(blob);\n link.setAttribute('href', url);\n link.setAttribute('download', 'topic_partitions.csv');\n link.style.visibility = 'hidden';\n document.body.appendChild(link);\n link.click();\n document.body.removeChild(link);\n }\n\n // Edit retention functions\n function showEditRetentionModal() {\n const modal = new bootstrap.Modal(document.getElementById('editRetentionModal'));\n \n // Get current retention values from the page\n const currentEnabled = document.querySelector('dd .badge.bg-success') !== null;\n const currentDurationElement = document.querySelector('dd .text-success');\n \n let currentValue = 7;\n let currentUnit = 'days';\n \n if (currentEnabled && currentDurationElement) {\n const durationText = currentDurationElement.textContent.trim();\n const parts = durationText.split(' ');\n if (parts.length >= 2) {\n currentValue = parseInt(parts[0]) || 7;\n currentUnit = parts[1].toLowerCase();\n // Handle plural forms\n if (currentUnit.endsWith('s')) {\n currentUnit = currentUnit.slice(0, -1);\n }\n // Map to our dropdown values\n if (currentUnit === 'hour') {\n currentUnit = 'hours';\n } else if (currentUnit === 'day') {\n currentUnit = 'days';\n }\n }\n }\n \n // Set current values in the modal\n document.getElementById('editEnableRetention').checked = currentEnabled;\n document.getElementById('editRetentionValue').value = currentValue;\n document.getElementById('editRetentionUnit').value = currentUnit;\n \n // Show/hide retention fields based on current state\n toggleEditRetentionFields();\n \n modal.show();\n }\n\n function toggleEditRetentionFields() {\n const enableRetention = document.getElementById('editEnableRetention');\n const retentionFields = document.getElementById('editRetentionFields');\n \n if (enableRetention.checked) {\n retentionFields.style.display = 'block';\n } else {\n retentionFields.style.display = 'none';\n }\n }\n\n function updateRetention() {\n const form = document.getElementById('editRetentionForm');\n const formData = new FormData(form);\n \n // Get topic details from the page\n const topicName = document.querySelector('h1').textContent.replace('Topic Details: ', '');\n const parts = topicName.split('.');\n \n if (parts.length < 2) {\n alert('Invalid topic name format');\n return;\n }\n \n const namespace = parts[0];\n const name = parts.slice(1).join('.');\n \n // Convert form data to JSON\n const data = {\n namespace: namespace,\n name: name,\n retention: {\n enabled: formData.get('editEnableRetention') === 'on',\n retention_seconds: 0\n }\n };\n\n // Calculate retention seconds if enabled\n if (data.retention.enabled) {\n const retentionValue = parseInt(formData.get('editRetentionValue'));\n const retentionUnit = formData.get('editRetentionUnit');\n \n if (retentionUnit === 'hours') {\n data.retention.retention_seconds = retentionValue * 3600;\n } else if (retentionUnit === 'days') {\n data.retention.retention_seconds = retentionValue * 86400;\n }\n }\n\n // Show loading state\n const updateButton = document.querySelector('#editRetentionModal .btn-primary');\n updateButton.disabled = true;\n updateButton.innerHTML = '<i class=\"fas fa-spinner fa-spin me-1\"></i>Updating...';\n\n // Send API request\n fetch('/api/mq/topics/retention/update', {\n method: 'POST',\n headers: {\n 'Content-Type': 'application/json',\n },\n body: JSON.stringify(data)\n })\n .then(response => response.json())\n .then(result => {\n if (result.error) {\n alert('Failed to update retention: ' + result.error);\n } else {\n alert('Retention policy updated successfully!');\n // Close modal and refresh page\n const modal = bootstrap.Modal.getInstance(document.getElementById('editRetentionModal'));\n modal.hide();\n window.location.reload();\n }\n })\n .catch(error => {\n alert('Failed to update retention: ' + error.message);\n })\n .finally(() => {\n // Reset button state\n updateButton.disabled = false;\n updateButton.innerHTML = '<i class=\"fas fa-save me-1\"></i>Update Retention';\n });\n }\n </script><!-- Edit Retention Modal --><div class=\"modal fade\" id=\"editRetentionModal\" tabindex=\"-1\" role=\"dialog\"><div class=\"modal-dialog modal-lg\" role=\"document\"><div class=\"modal-content\"><div class=\"modal-header\"><h5 class=\"modal-title\"><i class=\"fas fa-edit me-2\"></i>Edit Retention Policy</h5><button type=\"button\" class=\"btn-close\" data-bs-dismiss=\"modal\"></button></div><div class=\"modal-body\"><form id=\"editRetentionForm\"><div class=\"card\"><div class=\"card-header\"><h6 class=\"mb-0\"><i class=\"fas fa-clock me-2\"></i>Retention Configuration</h6></div><div class=\"card-body\"><div class=\"form-check mb-3\"><input class=\"form-check-input\" type=\"checkbox\" id=\"editEnableRetention\" name=\"editEnableRetention\" onchange=\"toggleEditRetentionFields()\"> <label class=\"form-check-label\" for=\"editEnableRetention\">Enable data retention</label></div><div id=\"editRetentionFields\" style=\"display: none;\"><div class=\"row\"><div class=\"col-md-6\"><div class=\"mb-3\"><label for=\"editRetentionValue\" class=\"form-label\">Retention Duration</label> <input type=\"number\" class=\"form-control\" id=\"editRetentionValue\" name=\"editRetentionValue\" min=\"1\" value=\"7\"></div></div><div class=\"col-md-6\"><div class=\"mb-3\"><label for=\"editRetentionUnit\" class=\"form-label\">Unit</label> <select class=\"form-control\" id=\"editRetentionUnit\" name=\"editRetentionUnit\"><option value=\"hours\">Hours</option> <option value=\"days\" selected>Days</option></select></div></div></div><div class=\"alert alert-info\"><i class=\"fas fa-info-circle me-2\"></i> Data older than this duration will be automatically purged to save storage space.</div></div></div></div></form></div><div class=\"modal-footer\"><button type=\"button\" class=\"btn btn-secondary\" data-bs-dismiss=\"modal\">Cancel</button> <button type=\"button\" class=\"btn btn-primary\" onclick=\"updateRetention()\"><i class=\"fas fa-save me-1\"></i>Update Retention</button></div></div></div></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ return nil
+ })
+}
+
+var _ = templruntime.GeneratedTemplate
diff --git a/weed/admin/view/app/topics.templ b/weed/admin/view/app/topics.templ
new file mode 100644
index 000000000..47c6625ae
--- /dev/null
+++ b/weed/admin/view/app/topics.templ
@@ -0,0 +1,511 @@
+package app
+
+import "fmt"
+import "strings"
+import "github.com/seaweedfs/seaweedfs/weed/admin/dash"
+
+templ Topics(data dash.TopicsData) {
+ <div class="container-fluid">
+ <div class="row">
+ <div class="col-12">
+ <div class="d-flex justify-content-between align-items-center mb-4">
+ <h1 class="h3 mb-0">Message Queue Topics</h1>
+ <small class="text-muted">Last updated: {data.LastUpdated.Format("2006-01-02 15:04:05")}</small>
+ </div>
+
+ <!-- Summary Cards -->
+ <div class="row mb-4">
+ <div class="col-md-6">
+ <div class="card text-center">
+ <div class="card-body">
+ <h5 class="card-title">Total Topics</h5>
+ <h3 class="text-primary">{fmt.Sprintf("%d", data.TotalTopics)}</h3>
+ </div>
+ </div>
+ </div>
+ <div class="col-md-6">
+ <div class="card text-center">
+ <div class="card-body">
+ <h5 class="card-title">Available Topics</h5>
+ <h3 class="text-info">{fmt.Sprintf("%d", len(data.Topics))}</h3>
+ </div>
+ </div>
+ </div>
+ </div>
+
+ <!-- Topics Table -->
+ <div class="card">
+ <div class="card-header d-flex justify-content-between align-items-center">
+ <h5 class="mb-0">Topics</h5>
+ <div>
+ <button class="btn btn-sm btn-primary me-2" onclick="showCreateTopicModal()">
+ <i class="fas fa-plus me-1"></i>Create Topic
+ </button>
+ <button class="btn btn-sm btn-outline-secondary" onclick="exportTopicsCSV()">
+ <i class="fas fa-download me-1"></i>Export CSV
+ </button>
+ </div>
+ </div>
+ <div class="card-body">
+ if len(data.Topics) == 0 {
+ <div class="text-center py-4">
+ <i class="fas fa-list-alt fa-3x text-muted mb-3"></i>
+ <h5>No Topics Found</h5>
+ <p class="text-muted">No message queue topics are currently configured.</p>
+ </div>
+ } else {
+ <div class="table-responsive">
+ <table class="table table-striped" id="topicsTable">
+ <thead>
+ <tr>
+ <th>Namespace</th>
+ <th>Topic Name</th>
+ <th>Partitions</th>
+ <th>Retention</th>
+ <th>Actions</th>
+ </tr>
+ </thead>
+ <tbody>
+ for _, topic := range data.Topics {
+ <tr class="topic-row" data-topic-name={topic.Name} style="cursor: pointer;">
+ <td>
+ <span class="badge bg-secondary">{func() string {
+ idx := strings.LastIndex(topic.Name, ".")
+ if idx == -1 {
+ return "default"
+ }
+ return topic.Name[:idx]
+ }()}</span>
+ </td>
+ <td>
+ <strong>{func() string {
+ idx := strings.LastIndex(topic.Name, ".")
+ if idx == -1 {
+ return topic.Name
+ }
+ return topic.Name[idx+1:]
+ }()}</strong>
+ </td>
+ <td>
+ <span class="badge bg-info">{fmt.Sprintf("%d", topic.Partitions)}</span>
+ </td>
+ <td>
+ if topic.Retention.Enabled {
+ <span class="badge bg-success">
+ <i class="fas fa-clock me-1"></i>
+ {fmt.Sprintf("%d %s", topic.Retention.DisplayValue, topic.Retention.DisplayUnit)}
+ </span>
+ } else {
+ <span class="badge bg-secondary">
+ <i class="fas fa-times me-1"></i>Disabled
+ </span>
+ }
+ </td>
+ <td>
+ <button class="btn btn-sm btn-outline-primary" onclick={ templ.ComponentScript{Call: fmt.Sprintf("viewTopicDetails('%s')", topic.Name)} }>
+ <i class="fas fa-info-circle me-1"></i>Details
+ </button>
+ </td>
+ </tr>
+ <tr class="topic-details-row" id={ fmt.Sprintf("details-%s", strings.ReplaceAll(topic.Name, ".", "_")) } style="display: none;">
+ <td colspan="5">
+ <div class="topic-details-content">
+ <div class="text-center py-3">
+ <i class="fas fa-spinner fa-spin"></i> Loading topic details...
+ </div>
+ </div>
+ </td>
+ </tr>
+ }
+ </tbody>
+ </table>
+ </div>
+ }
+ </div>
+ </div>
+ </div>
+ </div>
+ </div>
+
+ <script>
+ function viewTopicDetails(topicName) {
+ const parts = topicName.split('.');
+ if (parts.length >= 2) {
+ const namespace = parts[0];
+ const topic = parts.slice(1).join('.');
+ window.location.href = `/mq/topics/${namespace}/${topic}`;
+ }
+ }
+
+ function toggleTopicDetails(topicName) {
+ const safeName = topicName.replace(/\./g, '_');
+ const detailsRow = document.getElementById(`details-${safeName}`);
+ if (!detailsRow) return;
+
+ if (detailsRow.style.display === 'none') {
+ // Show details row and load data
+ detailsRow.style.display = 'table-row';
+ loadTopicDetails(topicName);
+ } else {
+ // Hide details row
+ detailsRow.style.display = 'none';
+ }
+ }
+
+ function loadTopicDetails(topicName) {
+ const parts = topicName.split('.');
+ if (parts.length < 2) return;
+
+ const namespace = parts[0];
+ const topic = parts.slice(1).join('.');
+ const safeName = topicName.replace(/\./g, '_');
+ const contentDiv = document.querySelector(`#details-${safeName} .topic-details-content`);
+
+ if (!contentDiv) return;
+
+ // Show loading spinner
+ contentDiv.innerHTML = `
+ <div class="text-center py-3">
+ <i class="fas fa-spinner fa-spin"></i> Loading topic details...
+ </div>
+ `;
+
+ // Make AJAX call to get topic details
+ fetch(`/api/mq/topics/${namespace}/${topic}`)
+ .then(response => response.json())
+ .then(data => {
+ if (data.error) {
+ contentDiv.innerHTML = `
+ <div class="alert alert-danger" role="alert">
+ <i class="fas fa-exclamation-triangle"></i> Error: ${data.error}
+ </div>
+ `;
+ return;
+ }
+
+ // Render topic details
+ contentDiv.innerHTML = renderTopicDetails(data);
+ })
+ .catch(error => {
+ contentDiv.innerHTML = `
+ <div class="alert alert-danger" role="alert">
+ <i class="fas fa-exclamation-triangle"></i> Failed to load topic details: ${error.message}
+ </div>
+ `;
+ });
+ }
+
+ function renderTopicDetails(data) {
+ const createdAt = new Date(data.created_at).toLocaleString();
+ const lastUpdated = new Date(data.last_updated).toLocaleString();
+
+ let schemaHtml = '';
+ if (data.schema && data.schema.length > 0) {
+ schemaHtml = `
+ <div class="col-md-6">
+ <h6>Schema Fields</h6>
+ <div class="table-responsive">
+ <table class="table table-sm">
+ <thead>
+ <tr>
+ <th>Field</th>
+ <th>Type</th>
+ <th>Required</th>
+ </tr>
+ </thead>
+ <tbody>
+ ${data.schema.map(field => `
+ <tr>
+ <td>${field.name}</td>
+ <td><span class="badge bg-secondary">${field.type}</span></td>
+ <td>${field.required ? '<span class="badge bg-success">Yes</span>' : '<span class="badge bg-light text-dark">No</span>'}</td>
+ </tr>
+ `).join('')}
+ </tbody>
+ </table>
+ </div>
+ </div>
+ `;
+ }
+
+ let partitionsHtml = '';
+ if (data.partitions && data.partitions.length > 0) {
+ partitionsHtml = `
+ <div class="col-md-6">
+ <h6>Partitions</h6>
+ <div class="table-responsive">
+ <table class="table table-sm">
+ <thead>
+ <tr>
+ <th>ID</th>
+ <th>Leader</th>
+ <th>Follower</th>
+ </tr>
+ </thead>
+ <tbody>
+ ${data.partitions.map(partition => `
+ <tr>
+ <td>${partition.id}</td>
+ <td>${partition.leader_broker || 'N/A'}</td>
+ <td>${partition.follower_broker || 'N/A'}</td>
+ </tr>
+ `).join('')}
+ </tbody>
+ </table>
+ </div>
+ </div>
+ `;
+ }
+
+
+
+
+
+ return `
+ <div class="card">
+ <div class="card-header">
+ <h5>Topic Details: ${data.namespace}.${data.name}</h5>
+ </div>
+ <div class="card-body">
+ <div class="row mb-3">
+ <div class="col-md-3">
+ <strong>Namespace:</strong> ${data.namespace}
+ </div>
+ <div class="col-md-3">
+ <strong>Topic Name:</strong> ${data.name}
+ </div>
+ <div class="col-md-3">
+ <strong>Created:</strong> ${createdAt}
+ </div>
+ <div class="col-md-3">
+ <strong>Last Updated:</strong> ${lastUpdated}
+ </div>
+ </div>
+ <div class="row mb-3">
+ ${schemaHtml}
+ ${partitionsHtml}
+ </div>
+
+ </div>
+ </div>
+ `;
+ }
+
+ function exportTopicsCSV() {
+ const table = document.getElementById('topicsTable');
+ if (!table) return;
+
+ let csv = 'Namespace,Topic Name,Partitions,Retention\n';
+
+ const rows = table.querySelectorAll('tbody tr.topic-row');
+ rows.forEach(row => {
+ const cells = row.querySelectorAll('td');
+ if (cells.length >= 4) {
+ const rowData = [
+ cells[0].querySelector('.badge')?.textContent || '', // Namespace
+ cells[1].querySelector('strong')?.textContent || '', // Topic Name
+ cells[2].querySelector('.badge')?.textContent || '', // Partitions
+ cells[3].querySelector('.badge')?.textContent || '' // Retention
+ ];
+ csv += rowData.map(field => `"${field.replace(/"/g, '""')}"`).join(',') + '\n';
+ }
+ });
+
+ const blob = new Blob([csv], { type: 'text/csv;charset=utf-8;' });
+ const link = document.createElement('a');
+ const url = URL.createObjectURL(blob);
+ link.setAttribute('href', url);
+ link.setAttribute('download', 'topics.csv');
+ link.style.visibility = 'hidden';
+ document.body.appendChild(link);
+ link.click();
+ document.body.removeChild(link);
+ }
+
+ // Topic creation functions
+ function showCreateTopicModal() {
+ const modal = new bootstrap.Modal(document.getElementById('createTopicModal'));
+ modal.show();
+ }
+
+ function toggleRetentionFields() {
+ const enableRetention = document.getElementById('enableRetention');
+ const retentionFields = document.getElementById('retentionFields');
+
+ if (enableRetention.checked) {
+ retentionFields.style.display = 'block';
+ } else {
+ retentionFields.style.display = 'none';
+ }
+ }
+
+ function createTopic() {
+ const form = document.getElementById('createTopicForm');
+ const formData = new FormData(form);
+
+ // Convert form data to JSON
+ const data = {
+ namespace: formData.get('namespace'),
+ name: formData.get('name'),
+ partition_count: parseInt(formData.get('partitionCount')),
+ retention: {
+ enabled: formData.get('enableRetention') === 'on',
+ retention_seconds: 0
+ }
+ };
+
+ // Calculate retention seconds if enabled
+ if (data.retention.enabled) {
+ const retentionValue = parseInt(formData.get('retentionValue'));
+ const retentionUnit = formData.get('retentionUnit');
+
+ if (retentionUnit === 'hours') {
+ data.retention.retention_seconds = retentionValue * 3600;
+ } else if (retentionUnit === 'days') {
+ data.retention.retention_seconds = retentionValue * 86400;
+ }
+ }
+
+ // Validate required fields
+ if (!data.namespace || !data.name || !data.partition_count) {
+ alert('Please fill in all required fields');
+ return;
+ }
+
+ // Show loading state
+ const createButton = document.querySelector('#createTopicModal .btn-primary');
+ createButton.disabled = true;
+ createButton.innerHTML = '<i class="fas fa-spinner fa-spin me-1"></i>Creating...';
+
+ // Send API request
+ fetch('/api/mq/topics/create', {
+ method: 'POST',
+ headers: {
+ 'Content-Type': 'application/json',
+ },
+ body: JSON.stringify(data)
+ })
+ .then(response => response.json())
+ .then(result => {
+ if (result.error) {
+ alert('Failed to create topic: ' + result.error);
+ } else {
+ alert('Topic created successfully!');
+ // Close modal and refresh page
+ const modal = bootstrap.Modal.getInstance(document.getElementById('createTopicModal'));
+ modal.hide();
+ window.location.reload();
+ }
+ })
+ .catch(error => {
+ alert('Failed to create topic: ' + error.message);
+ })
+ .finally(() => {
+ // Reset button state
+ createButton.disabled = false;
+ createButton.innerHTML = '<i class="fas fa-plus me-1"></i>Create Topic';
+ });
+ }
+
+ // Add click event listeners to topic rows
+ document.addEventListener('DOMContentLoaded', function() {
+ document.querySelectorAll('.topic-row').forEach(row => {
+ row.addEventListener('click', function() {
+ const topicName = this.getAttribute('data-topic-name');
+ toggleTopicDetails(topicName);
+ });
+ });
+ });
+ </script>
+
+ <!-- Create Topic Modal -->
+ <div class="modal fade" id="createTopicModal" tabindex="-1" role="dialog">
+ <div class="modal-dialog modal-lg" role="document">
+ <div class="modal-content">
+ <div class="modal-header">
+ <h5 class="modal-title">
+ <i class="fas fa-plus me-2"></i>Create New Topic
+ </h5>
+ <button type="button" class="btn-close" data-bs-dismiss="modal"></button>
+ </div>
+ <div class="modal-body">
+ <form id="createTopicForm">
+ <div class="row">
+ <div class="col-md-6">
+ <div class="mb-3">
+ <label for="topicNamespace" class="form-label">Namespace *</label>
+ <input type="text" class="form-control" id="topicNamespace" name="namespace" required
+ placeholder="e.g., default">
+ </div>
+ </div>
+ <div class="col-md-6">
+ <div class="mb-3">
+ <label for="topicName" class="form-label">Topic Name *</label>
+ <input type="text" class="form-control" id="topicName" name="name" required
+ placeholder="e.g., user-events">
+ </div>
+ </div>
+ </div>
+ <div class="row">
+ <div class="col-md-6">
+ <div class="mb-3">
+ <label for="partitionCount" class="form-label">Partition Count *</label>
+ <input type="number" class="form-control" id="partitionCount" name="partitionCount"
+ required min="1" max="100" value="6">
+ </div>
+ </div>
+ </div>
+
+ <!-- Retention Configuration -->
+ <div class="card mt-3">
+ <div class="card-header">
+ <h6 class="mb-0">
+ <i class="fas fa-clock me-2"></i>Retention Policy
+ </h6>
+ </div>
+ <div class="card-body">
+ <div class="form-check mb-3">
+ <input class="form-check-input" type="checkbox" id="enableRetention"
+ name="enableRetention" onchange="toggleRetentionFields()">
+ <label class="form-check-label" for="enableRetention">
+ Enable data retention
+ </label>
+ </div>
+ <div id="retentionFields" style="display: none;">
+ <div class="row">
+ <div class="col-md-6">
+ <div class="mb-3">
+ <label for="retentionValue" class="form-label">Retention Duration</label>
+ <input type="number" class="form-control" id="retentionValue"
+ name="retentionValue" min="1" value="7">
+ </div>
+ </div>
+ <div class="col-md-6">
+ <div class="mb-3">
+ <label for="retentionUnit" class="form-label">Unit</label>
+ <select class="form-control" id="retentionUnit" name="retentionUnit">
+ <option value="hours">Hours</option>
+ <option value="days" selected>Days</option>
+ </select>
+ </div>
+ </div>
+ </div>
+ <div class="alert alert-info">
+ <i class="fas fa-info-circle me-2"></i>
+ Data older than this duration will be automatically purged to save storage space.
+ </div>
+ </div>
+ </div>
+ </div>
+ </form>
+ </div>
+ <div class="modal-footer">
+ <button type="button" class="btn btn-secondary" data-bs-dismiss="modal">Cancel</button>
+ <button type="button" class="btn btn-primary" onclick="createTopic()">
+ <i class="fas fa-plus me-1"></i>Create Topic
+ </button>
+ </div>
+ </div>
+ </div>
+ </div>
+} \ No newline at end of file
diff --git a/weed/admin/view/app/topics_templ.go b/weed/admin/view/app/topics_templ.go
new file mode 100644
index 000000000..2b753adfc
--- /dev/null
+++ b/weed/admin/view/app/topics_templ.go
@@ -0,0 +1,230 @@
+// Code generated by templ - DO NOT EDIT.
+
+// templ: version: v0.3.833
+package app
+
+//lint:file-ignore SA4006 This context is only used if a nested component is present.
+
+import "github.com/a-h/templ"
+import templruntime "github.com/a-h/templ/runtime"
+
+import "fmt"
+import "strings"
+import "github.com/seaweedfs/seaweedfs/weed/admin/dash"
+
+func Topics(data dash.TopicsData) templ.Component {
+ return templruntime.GeneratedTemplate(func(templ_7745c5c3_Input templruntime.GeneratedComponentInput) (templ_7745c5c3_Err error) {
+ templ_7745c5c3_W, ctx := templ_7745c5c3_Input.Writer, templ_7745c5c3_Input.Context
+ if templ_7745c5c3_CtxErr := ctx.Err(); templ_7745c5c3_CtxErr != nil {
+ return templ_7745c5c3_CtxErr
+ }
+ templ_7745c5c3_Buffer, templ_7745c5c3_IsBuffer := templruntime.GetBuffer(templ_7745c5c3_W)
+ if !templ_7745c5c3_IsBuffer {
+ defer func() {
+ templ_7745c5c3_BufErr := templruntime.ReleaseBuffer(templ_7745c5c3_Buffer)
+ if templ_7745c5c3_Err == nil {
+ templ_7745c5c3_Err = templ_7745c5c3_BufErr
+ }
+ }()
+ }
+ ctx = templ.InitializeContext(ctx)
+ templ_7745c5c3_Var1 := templ.GetChildren(ctx)
+ if templ_7745c5c3_Var1 == nil {
+ templ_7745c5c3_Var1 = templ.NopComponent
+ }
+ ctx = templ.ClearChildren(ctx)
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 1, "<div class=\"container-fluid\"><div class=\"row\"><div class=\"col-12\"><div class=\"d-flex justify-content-between align-items-center mb-4\"><h1 class=\"h3 mb-0\">Message Queue Topics</h1><small class=\"text-muted\">Last updated: ")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var2 string
+ templ_7745c5c3_Var2, templ_7745c5c3_Err = templ.JoinStringErrs(data.LastUpdated.Format("2006-01-02 15:04:05"))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topics.templ`, Line: 13, Col: 107}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var2))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 2, "</small></div><!-- Summary Cards --><div class=\"row mb-4\"><div class=\"col-md-6\"><div class=\"card text-center\"><div class=\"card-body\"><h5 class=\"card-title\">Total Topics</h5><h3 class=\"text-primary\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var3 string
+ templ_7745c5c3_Var3, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.TotalTopics))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topics.templ`, Line: 22, Col: 93}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var3))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 3, "</h3></div></div></div><div class=\"col-md-6\"><div class=\"card text-center\"><div class=\"card-body\"><h5 class=\"card-title\">Available Topics</h5><h3 class=\"text-info\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var4 string
+ templ_7745c5c3_Var4, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", len(data.Topics)))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topics.templ`, Line: 30, Col: 90}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var4))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 4, "</h3></div></div></div></div><!-- Topics Table --><div class=\"card\"><div class=\"card-header d-flex justify-content-between align-items-center\"><h5 class=\"mb-0\">Topics</h5><div><button class=\"btn btn-sm btn-primary me-2\" onclick=\"showCreateTopicModal()\"><i class=\"fas fa-plus me-1\"></i>Create Topic</button> <button class=\"btn btn-sm btn-outline-secondary\" onclick=\"exportTopicsCSV()\"><i class=\"fas fa-download me-1\"></i>Export CSV</button></div></div><div class=\"card-body\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if len(data.Topics) == 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 5, "<div class=\"text-center py-4\"><i class=\"fas fa-list-alt fa-3x text-muted mb-3\"></i><h5>No Topics Found</h5><p class=\"text-muted\">No message queue topics are currently configured.</p></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 6, "<div class=\"table-responsive\"><table class=\"table table-striped\" id=\"topicsTable\"><thead><tr><th>Namespace</th><th>Topic Name</th><th>Partitions</th><th>Retention</th><th>Actions</th></tr></thead> <tbody>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ for _, topic := range data.Topics {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 7, "<tr class=\"topic-row\" data-topic-name=\"")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var5 string
+ templ_7745c5c3_Var5, templ_7745c5c3_Err = templ.JoinStringErrs(topic.Name)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topics.templ`, Line: 70, Col: 93}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var5))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 8, "\" style=\"cursor: pointer;\"><td><span class=\"badge bg-secondary\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var6 string
+ templ_7745c5c3_Var6, templ_7745c5c3_Err = templ.JoinStringErrs(func() string {
+ idx := strings.LastIndex(topic.Name, ".")
+ if idx == -1 {
+ return "default"
+ }
+ return topic.Name[:idx]
+ }())
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topics.templ`, Line: 78, Col: 55}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var6))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 9, "</span></td><td><strong>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var7 string
+ templ_7745c5c3_Var7, templ_7745c5c3_Err = templ.JoinStringErrs(func() string {
+ idx := strings.LastIndex(topic.Name, ".")
+ if idx == -1 {
+ return topic.Name
+ }
+ return topic.Name[idx+1:]
+ }())
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topics.templ`, Line: 87, Col: 55}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var7))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 10, "</strong></td><td><span class=\"badge bg-info\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var8 string
+ templ_7745c5c3_Var8, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", topic.Partitions))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topics.templ`, Line: 90, Col: 116}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var8))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 11, "</span></td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if topic.Retention.Enabled {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 12, "<span class=\"badge bg-success\"><i class=\"fas fa-clock me-1\"></i> ")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var9 string
+ templ_7745c5c3_Var9, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d %s", topic.Retention.DisplayValue, topic.Retention.DisplayUnit))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topics.templ`, Line: 96, Col: 140}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var9))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 14, "<span class=\"badge bg-secondary\"><i class=\"fas fa-times me-1\"></i>Disabled</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "</td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templ.RenderScriptItems(ctx, templ_7745c5c3_Buffer, templ.ComponentScript{Call: fmt.Sprintf("viewTopicDetails('%s')", topic.Name)})
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, "<button class=\"btn btn-sm btn-outline-primary\" onclick=\"")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var10 templ.ComponentScript = templ.ComponentScript{Call: fmt.Sprintf("viewTopicDetails('%s')", topic.Name)}
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ_7745c5c3_Var10.Call)
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, "\"><i class=\"fas fa-info-circle me-1\"></i>Details</button></td></tr><tr class=\"topic-details-row\" id=\"")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var11 string
+ templ_7745c5c3_Var11, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("details-%s", strings.ReplaceAll(topic.Name, ".", "_")))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/topics.templ`, Line: 110, Col: 146}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var11))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 18, "\" style=\"display: none;\"><td colspan=\"5\"><div class=\"topic-details-content\"><div class=\"text-center py-3\"><i class=\"fas fa-spinner fa-spin\"></i> Loading topic details...</div></div></td></tr>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, "</tbody></table></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, "</div></div></div></div></div><script>\n function viewTopicDetails(topicName) {\n const parts = topicName.split('.');\n if (parts.length >= 2) {\n const namespace = parts[0];\n const topic = parts.slice(1).join('.');\n window.location.href = `/mq/topics/${namespace}/${topic}`;\n }\n }\n\n function toggleTopicDetails(topicName) {\n const safeName = topicName.replace(/\\./g, '_');\n const detailsRow = document.getElementById(`details-${safeName}`);\n if (!detailsRow) return;\n\n if (detailsRow.style.display === 'none') {\n // Show details row and load data\n detailsRow.style.display = 'table-row';\n loadTopicDetails(topicName);\n } else {\n // Hide details row\n detailsRow.style.display = 'none';\n }\n }\n\n function loadTopicDetails(topicName) {\n const parts = topicName.split('.');\n if (parts.length < 2) return;\n \n const namespace = parts[0];\n const topic = parts.slice(1).join('.');\n const safeName = topicName.replace(/\\./g, '_');\n const contentDiv = document.querySelector(`#details-${safeName} .topic-details-content`);\n \n if (!contentDiv) return;\n\n // Show loading spinner\n contentDiv.innerHTML = `\n <div class=\"text-center py-3\">\n <i class=\"fas fa-spinner fa-spin\"></i> Loading topic details...\n </div>\n `;\n\n // Make AJAX call to get topic details\n fetch(`/api/mq/topics/${namespace}/${topic}`)\n .then(response => response.json())\n .then(data => {\n if (data.error) {\n contentDiv.innerHTML = `\n <div class=\"alert alert-danger\" role=\"alert\">\n <i class=\"fas fa-exclamation-triangle\"></i> Error: ${data.error}\n </div>\n `;\n return;\n }\n\n // Render topic details\n contentDiv.innerHTML = renderTopicDetails(data);\n })\n .catch(error => {\n contentDiv.innerHTML = `\n <div class=\"alert alert-danger\" role=\"alert\">\n <i class=\"fas fa-exclamation-triangle\"></i> Failed to load topic details: ${error.message}\n </div>\n `;\n });\n }\n\n function renderTopicDetails(data) {\n const createdAt = new Date(data.created_at).toLocaleString();\n const lastUpdated = new Date(data.last_updated).toLocaleString();\n\n let schemaHtml = '';\n if (data.schema && data.schema.length > 0) {\n schemaHtml = `\n <div class=\"col-md-6\">\n <h6>Schema Fields</h6>\n <div class=\"table-responsive\">\n <table class=\"table table-sm\">\n <thead>\n <tr>\n <th>Field</th>\n <th>Type</th>\n <th>Required</th>\n </tr>\n </thead>\n <tbody>\n ${data.schema.map(field => `\n <tr>\n <td>${field.name}</td>\n <td><span class=\"badge bg-secondary\">${field.type}</span></td>\n <td>${field.required ? '<span class=\"badge bg-success\">Yes</span>' : '<span class=\"badge bg-light text-dark\">No</span>'}</td>\n </tr>\n `).join('')}\n </tbody>\n </table>\n </div>\n </div>\n `;\n }\n\n let partitionsHtml = '';\n if (data.partitions && data.partitions.length > 0) {\n partitionsHtml = `\n <div class=\"col-md-6\">\n <h6>Partitions</h6>\n <div class=\"table-responsive\">\n <table class=\"table table-sm\">\n <thead>\n <tr>\n <th>ID</th>\n <th>Leader</th>\n <th>Follower</th>\n </tr>\n </thead>\n <tbody>\n ${data.partitions.map(partition => `\n <tr>\n <td>${partition.id}</td>\n <td>${partition.leader_broker || 'N/A'}</td>\n <td>${partition.follower_broker || 'N/A'}</td>\n </tr>\n `).join('')}\n </tbody>\n </table>\n </div>\n </div>\n `;\n }\n\n\n\n\n\n return `\n <div class=\"card\">\n <div class=\"card-header\">\n <h5>Topic Details: ${data.namespace}.${data.name}</h5>\n </div>\n <div class=\"card-body\">\n <div class=\"row mb-3\">\n <div class=\"col-md-3\">\n <strong>Namespace:</strong> ${data.namespace}\n </div>\n <div class=\"col-md-3\">\n <strong>Topic Name:</strong> ${data.name}\n </div>\n <div class=\"col-md-3\">\n <strong>Created:</strong> ${createdAt}\n </div>\n <div class=\"col-md-3\">\n <strong>Last Updated:</strong> ${lastUpdated}\n </div>\n </div>\n <div class=\"row mb-3\">\n ${schemaHtml}\n ${partitionsHtml}\n </div>\n\n </div>\n </div>\n `;\n }\n\n function exportTopicsCSV() {\n const table = document.getElementById('topicsTable');\n if (!table) return;\n \n let csv = 'Namespace,Topic Name,Partitions,Retention\\n';\n \n const rows = table.querySelectorAll('tbody tr.topic-row');\n rows.forEach(row => {\n const cells = row.querySelectorAll('td');\n if (cells.length >= 4) {\n const rowData = [\n cells[0].querySelector('.badge')?.textContent || '', // Namespace\n cells[1].querySelector('strong')?.textContent || '', // Topic Name\n cells[2].querySelector('.badge')?.textContent || '', // Partitions\n cells[3].querySelector('.badge')?.textContent || '' // Retention\n ];\n csv += rowData.map(field => `\"${field.replace(/\"/g, '\"\"')}\"`).join(',') + '\\n';\n }\n });\n \n const blob = new Blob([csv], { type: 'text/csv;charset=utf-8;' });\n const link = document.createElement('a');\n const url = URL.createObjectURL(blob);\n link.setAttribute('href', url);\n link.setAttribute('download', 'topics.csv');\n link.style.visibility = 'hidden';\n document.body.appendChild(link);\n link.click();\n document.body.removeChild(link);\n }\n\n // Topic creation functions\n function showCreateTopicModal() {\n const modal = new bootstrap.Modal(document.getElementById('createTopicModal'));\n modal.show();\n }\n\n function toggleRetentionFields() {\n const enableRetention = document.getElementById('enableRetention');\n const retentionFields = document.getElementById('retentionFields');\n \n if (enableRetention.checked) {\n retentionFields.style.display = 'block';\n } else {\n retentionFields.style.display = 'none';\n }\n }\n\n function createTopic() {\n const form = document.getElementById('createTopicForm');\n const formData = new FormData(form);\n \n // Convert form data to JSON\n const data = {\n namespace: formData.get('namespace'),\n name: formData.get('name'),\n partition_count: parseInt(formData.get('partitionCount')),\n retention: {\n enabled: formData.get('enableRetention') === 'on',\n retention_seconds: 0\n }\n };\n\n // Calculate retention seconds if enabled\n if (data.retention.enabled) {\n const retentionValue = parseInt(formData.get('retentionValue'));\n const retentionUnit = formData.get('retentionUnit');\n \n if (retentionUnit === 'hours') {\n data.retention.retention_seconds = retentionValue * 3600;\n } else if (retentionUnit === 'days') {\n data.retention.retention_seconds = retentionValue * 86400;\n }\n }\n\n // Validate required fields\n if (!data.namespace || !data.name || !data.partition_count) {\n alert('Please fill in all required fields');\n return;\n }\n\n // Show loading state\n const createButton = document.querySelector('#createTopicModal .btn-primary');\n createButton.disabled = true;\n createButton.innerHTML = '<i class=\"fas fa-spinner fa-spin me-1\"></i>Creating...';\n\n // Send API request\n fetch('/api/mq/topics/create', {\n method: 'POST',\n headers: {\n 'Content-Type': 'application/json',\n },\n body: JSON.stringify(data)\n })\n .then(response => response.json())\n .then(result => {\n if (result.error) {\n alert('Failed to create topic: ' + result.error);\n } else {\n alert('Topic created successfully!');\n // Close modal and refresh page\n const modal = bootstrap.Modal.getInstance(document.getElementById('createTopicModal'));\n modal.hide();\n window.location.reload();\n }\n })\n .catch(error => {\n alert('Failed to create topic: ' + error.message);\n })\n .finally(() => {\n // Reset button state\n createButton.disabled = false;\n createButton.innerHTML = '<i class=\"fas fa-plus me-1\"></i>Create Topic';\n });\n }\n\n // Add click event listeners to topic rows\n document.addEventListener('DOMContentLoaded', function() {\n document.querySelectorAll('.topic-row').forEach(row => {\n row.addEventListener('click', function() {\n const topicName = this.getAttribute('data-topic-name');\n toggleTopicDetails(topicName);\n });\n });\n });\n </script><!-- Create Topic Modal --><div class=\"modal fade\" id=\"createTopicModal\" tabindex=\"-1\" role=\"dialog\"><div class=\"modal-dialog modal-lg\" role=\"document\"><div class=\"modal-content\"><div class=\"modal-header\"><h5 class=\"modal-title\"><i class=\"fas fa-plus me-2\"></i>Create New Topic</h5><button type=\"button\" class=\"btn-close\" data-bs-dismiss=\"modal\"></button></div><div class=\"modal-body\"><form id=\"createTopicForm\"><div class=\"row\"><div class=\"col-md-6\"><div class=\"mb-3\"><label for=\"topicNamespace\" class=\"form-label\">Namespace *</label> <input type=\"text\" class=\"form-control\" id=\"topicNamespace\" name=\"namespace\" required placeholder=\"e.g., default\"></div></div><div class=\"col-md-6\"><div class=\"mb-3\"><label for=\"topicName\" class=\"form-label\">Topic Name *</label> <input type=\"text\" class=\"form-control\" id=\"topicName\" name=\"name\" required placeholder=\"e.g., user-events\"></div></div></div><div class=\"row\"><div class=\"col-md-6\"><div class=\"mb-3\"><label for=\"partitionCount\" class=\"form-label\">Partition Count *</label> <input type=\"number\" class=\"form-control\" id=\"partitionCount\" name=\"partitionCount\" required min=\"1\" max=\"100\" value=\"6\"></div></div></div><!-- Retention Configuration --><div class=\"card mt-3\"><div class=\"card-header\"><h6 class=\"mb-0\"><i class=\"fas fa-clock me-2\"></i>Retention Policy</h6></div><div class=\"card-body\"><div class=\"form-check mb-3\"><input class=\"form-check-input\" type=\"checkbox\" id=\"enableRetention\" name=\"enableRetention\" onchange=\"toggleRetentionFields()\"> <label class=\"form-check-label\" for=\"enableRetention\">Enable data retention</label></div><div id=\"retentionFields\" style=\"display: none;\"><div class=\"row\"><div class=\"col-md-6\"><div class=\"mb-3\"><label for=\"retentionValue\" class=\"form-label\">Retention Duration</label> <input type=\"number\" class=\"form-control\" id=\"retentionValue\" name=\"retentionValue\" min=\"1\" value=\"7\"></div></div><div class=\"col-md-6\"><div class=\"mb-3\"><label for=\"retentionUnit\" class=\"form-label\">Unit</label> <select class=\"form-control\" id=\"retentionUnit\" name=\"retentionUnit\"><option value=\"hours\">Hours</option> <option value=\"days\" selected>Days</option></select></div></div></div><div class=\"alert alert-info\"><i class=\"fas fa-info-circle me-2\"></i> Data older than this duration will be automatically purged to save storage space.</div></div></div></div></form></div><div class=\"modal-footer\"><button type=\"button\" class=\"btn btn-secondary\" data-bs-dismiss=\"modal\">Cancel</button> <button type=\"button\" class=\"btn btn-primary\" onclick=\"createTopic()\"><i class=\"fas fa-plus me-1\"></i>Create Topic</button></div></div></div></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ return nil
+ })
+}
+
+var _ = templruntime.GeneratedTemplate
diff --git a/weed/admin/view/layout/layout.templ b/weed/admin/view/layout/layout.templ
index e540ac7e4..2261f1e41 100644
--- a/weed/admin/view/layout/layout.templ
+++ b/weed/admin/view/layout/layout.templ
@@ -18,6 +18,9 @@ templ Layout(c *gin.Context, content templ.Component) {
// Detect if we're on a configuration page to keep submenu expanded
currentPath := c.Request.URL.Path
isConfigPage := strings.HasPrefix(currentPath, "/maintenance/config") || currentPath == "/config"
+
+ // Detect if we're on a message queue page to keep submenu expanded
+ isMQPage := strings.HasPrefix(currentPath, "/mq/")
}}
<!DOCTYPE html>
<html lang="en">
@@ -148,6 +151,64 @@ templ Layout(c *gin.Context, content templ.Component) {
</div>
</li>
<li class="nav-item">
+ if isMQPage {
+ <a class="nav-link" href="#" data-bs-toggle="collapse" data-bs-target="#messageQueueSubmenu" aria-expanded="true" aria-controls="messageQueueSubmenu">
+ <i class="fas fa-comments me-2"></i>Message Queue
+ <i class="fas fa-chevron-down ms-auto"></i>
+ </a>
+ } else {
+ <a class="nav-link collapsed" href="#" data-bs-toggle="collapse" data-bs-target="#messageQueueSubmenu" aria-expanded="false" aria-controls="messageQueueSubmenu">
+ <i class="fas fa-comments me-2"></i>Message Queue
+ <i class="fas fa-chevron-down ms-auto"></i>
+ </a>
+ }
+ if isMQPage {
+ <div class="collapse show" id="messageQueueSubmenu">
+ <ul class="nav flex-column ms-3">
+ <li class="nav-item">
+ if currentPath == "/mq/brokers" {
+ <a class="nav-link py-2 active" href="/mq/brokers">
+ <i class="fas fa-server me-2"></i>Brokers
+ </a>
+ } else {
+ <a class="nav-link py-2" href="/mq/brokers">
+ <i class="fas fa-server me-2"></i>Brokers
+ </a>
+ }
+ </li>
+ <li class="nav-item">
+ if currentPath == "/mq/topics" {
+ <a class="nav-link py-2 active" href="/mq/topics">
+ <i class="fas fa-list-alt me-2"></i>Topics
+ </a>
+ } else {
+ <a class="nav-link py-2" href="/mq/topics">
+ <i class="fas fa-list-alt me-2"></i>Topics
+ </a>
+ }
+ </li>
+
+ </ul>
+ </div>
+ } else {
+ <div class="collapse" id="messageQueueSubmenu">
+ <ul class="nav flex-column ms-3">
+ <li class="nav-item">
+ <a class="nav-link py-2" href="/mq/brokers">
+ <i class="fas fa-server me-2"></i>Brokers
+ </a>
+ </li>
+ <li class="nav-item">
+ <a class="nav-link py-2" href="/mq/topics">
+ <i class="fas fa-list-alt me-2"></i>Topics
+ </a>
+ </li>
+
+ </ul>
+ </div>
+ }
+ </li>
+ <li class="nav-item">
<a class="nav-link" href="/metrics">
<i class="fas fa-chart-line me-2"></i>Metrics
</a>
diff --git a/weed/admin/view/layout/layout_templ.go b/weed/admin/view/layout/layout_templ.go
index 9ffe596bb..3ff64975c 100644
--- a/weed/admin/view/layout/layout_templ.go
+++ b/weed/admin/view/layout/layout_templ.go
@@ -46,6 +46,9 @@ func Layout(c *gin.Context, content templ.Component) templ.Component {
// Detect if we're on a configuration page to keep submenu expanded
currentPath := c.Request.URL.Path
isConfigPage := strings.HasPrefix(currentPath, "/maintenance/config") || currentPath == "/config"
+
+ // Detect if we're on a message queue page to keep submenu expanded
+ isMQPage := strings.HasPrefix(currentPath, "/mq/")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 1, "<!doctype html><html lang=\"en\"><head><meta charset=\"UTF-8\"><title>SeaweedFS Admin</title><meta name=\"viewport\" content=\"width=device-width, initial-scale=1\"><link rel=\"icon\" href=\"/static/favicon.ico\" type=\"image/x-icon\"><!-- Bootstrap CSS --><link href=\"/static/css/bootstrap.min.css\" rel=\"stylesheet\"><!-- Font Awesome CSS --><link href=\"/static/css/fontawesome.min.css\" rel=\"stylesheet\"><!-- HTMX --><script src=\"/static/js/htmx.min.js\"></script><!-- Custom CSS --><link rel=\"stylesheet\" href=\"/static/css/admin.css\"></head><body><div class=\"container-fluid\"><!-- Header --><header class=\"navbar navbar-expand-lg navbar-dark bg-primary sticky-top\"><div class=\"container-fluid\"><a class=\"navbar-brand fw-bold\" href=\"/admin\"><i class=\"fas fa-server me-2\"></i> SeaweedFS Admin <span class=\"badge bg-warning text-dark ms-2\">ALPHA</span></a> <button class=\"navbar-toggler\" type=\"button\" data-bs-toggle=\"collapse\" data-bs-target=\"#navbarNav\"><span class=\"navbar-toggler-icon\"></span></button><div class=\"collapse navbar-collapse\" id=\"navbarNav\"><ul class=\"navbar-nav ms-auto\"><li class=\"nav-item dropdown\"><a class=\"nav-link dropdown-toggle\" href=\"#\" role=\"button\" data-bs-toggle=\"dropdown\"><i class=\"fas fa-user me-1\"></i>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
@@ -53,41 +56,97 @@ func Layout(c *gin.Context, content templ.Component) templ.Component {
var templ_7745c5c3_Var2 string
templ_7745c5c3_Var2, templ_7745c5c3_Err = templ.JoinStringErrs(username)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/layout/layout.templ`, Line: 58, Col: 73}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/layout/layout.templ`, Line: 61, Col: 73}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var2))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 2, "</a><ul class=\"dropdown-menu\"><li><a class=\"dropdown-item\" href=\"/logout\"><i class=\"fas fa-sign-out-alt me-2\"></i>Logout</a></li></ul></li></ul></div></div></header><div class=\"row g-0\"><!-- Sidebar --><div class=\"col-md-3 col-lg-2 d-md-block bg-light sidebar collapse\"><div class=\"position-sticky pt-3\"><h6 class=\"sidebar-heading px-3 mt-4 mb-1 text-muted\"><span>MAIN</span></h6><ul class=\"nav flex-column\"><li class=\"nav-item\"><a class=\"nav-link\" href=\"/admin\"><i class=\"fas fa-tachometer-alt me-2\"></i>Dashboard</a></li><li class=\"nav-item\"><a class=\"nav-link collapsed\" href=\"#\" data-bs-toggle=\"collapse\" data-bs-target=\"#clusterSubmenu\" aria-expanded=\"false\" aria-controls=\"clusterSubmenu\"><i class=\"fas fa-sitemap me-2\"></i>Cluster <i class=\"fas fa-chevron-down ms-auto\"></i></a><div class=\"collapse\" id=\"clusterSubmenu\"><ul class=\"nav flex-column ms-3\"><li class=\"nav-item\"><a class=\"nav-link py-2\" href=\"/cluster/masters\"><i class=\"fas fa-crown me-2\"></i>Masters</a></li><li class=\"nav-item\"><a class=\"nav-link py-2\" href=\"/cluster/volume-servers\"><i class=\"fas fa-server me-2\"></i>Volume Servers</a></li><li class=\"nav-item\"><a class=\"nav-link py-2\" href=\"/cluster/filers\"><i class=\"fas fa-folder-open me-2\"></i>Filers</a></li><li class=\"nav-item\"><a class=\"nav-link py-2\" href=\"/cluster/volumes\"><i class=\"fas fa-database me-2\"></i>Volumes</a></li><li class=\"nav-item\"><a class=\"nav-link py-2\" href=\"/cluster/collections\"><i class=\"fas fa-layer-group me-2\"></i>Collections</a></li></ul></div></li></ul><h6 class=\"sidebar-heading px-3 mt-4 mb-1 text-muted\"><span>MANAGEMENT</span></h6><ul class=\"nav flex-column\"><li class=\"nav-item\"><a class=\"nav-link\" href=\"/files\"><i class=\"fas fa-folder me-2\"></i>File Browser</a></li><li class=\"nav-item\"><a class=\"nav-link collapsed\" href=\"#\" data-bs-toggle=\"collapse\" data-bs-target=\"#objectStoreSubmenu\" aria-expanded=\"false\" aria-controls=\"objectStoreSubmenu\"><i class=\"fas fa-cloud me-2\"></i>Object Store <i class=\"fas fa-chevron-down ms-auto\"></i></a><div class=\"collapse\" id=\"objectStoreSubmenu\"><ul class=\"nav flex-column ms-3\"><li class=\"nav-item\"><a class=\"nav-link py-2\" href=\"/object-store/buckets\"><i class=\"fas fa-cube me-2\"></i>Buckets</a></li><li class=\"nav-item\"><a class=\"nav-link py-2\" href=\"/object-store/users\"><i class=\"fas fa-users me-2\"></i>Users</a></li></ul></div></li><li class=\"nav-item\"><a class=\"nav-link\" href=\"/metrics\"><i class=\"fas fa-chart-line me-2\"></i>Metrics</a></li><li class=\"nav-item\"><a class=\"nav-link\" href=\"/logs\"><i class=\"fas fa-file-alt me-2\"></i>Logs</a></li></ul><h6 class=\"sidebar-heading px-3 mt-4 mb-1 text-muted\"><span>SYSTEM</span></h6><ul class=\"nav flex-column\"><li class=\"nav-item\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 2, "</a><ul class=\"dropdown-menu\"><li><a class=\"dropdown-item\" href=\"/logout\"><i class=\"fas fa-sign-out-alt me-2\"></i>Logout</a></li></ul></li></ul></div></div></header><div class=\"row g-0\"><!-- Sidebar --><div class=\"col-md-3 col-lg-2 d-md-block bg-light sidebar collapse\"><div class=\"position-sticky pt-3\"><h6 class=\"sidebar-heading px-3 mt-4 mb-1 text-muted\"><span>MAIN</span></h6><ul class=\"nav flex-column\"><li class=\"nav-item\"><a class=\"nav-link\" href=\"/admin\"><i class=\"fas fa-tachometer-alt me-2\"></i>Dashboard</a></li><li class=\"nav-item\"><a class=\"nav-link collapsed\" href=\"#\" data-bs-toggle=\"collapse\" data-bs-target=\"#clusterSubmenu\" aria-expanded=\"false\" aria-controls=\"clusterSubmenu\"><i class=\"fas fa-sitemap me-2\"></i>Cluster <i class=\"fas fa-chevron-down ms-auto\"></i></a><div class=\"collapse\" id=\"clusterSubmenu\"><ul class=\"nav flex-column ms-3\"><li class=\"nav-item\"><a class=\"nav-link py-2\" href=\"/cluster/masters\"><i class=\"fas fa-crown me-2\"></i>Masters</a></li><li class=\"nav-item\"><a class=\"nav-link py-2\" href=\"/cluster/volume-servers\"><i class=\"fas fa-server me-2\"></i>Volume Servers</a></li><li class=\"nav-item\"><a class=\"nav-link py-2\" href=\"/cluster/filers\"><i class=\"fas fa-folder-open me-2\"></i>Filers</a></li><li class=\"nav-item\"><a class=\"nav-link py-2\" href=\"/cluster/volumes\"><i class=\"fas fa-database me-2\"></i>Volumes</a></li><li class=\"nav-item\"><a class=\"nav-link py-2\" href=\"/cluster/collections\"><i class=\"fas fa-layer-group me-2\"></i>Collections</a></li></ul></div></li></ul><h6 class=\"sidebar-heading px-3 mt-4 mb-1 text-muted\"><span>MANAGEMENT</span></h6><ul class=\"nav flex-column\"><li class=\"nav-item\"><a class=\"nav-link\" href=\"/files\"><i class=\"fas fa-folder me-2\"></i>File Browser</a></li><li class=\"nav-item\"><a class=\"nav-link collapsed\" href=\"#\" data-bs-toggle=\"collapse\" data-bs-target=\"#objectStoreSubmenu\" aria-expanded=\"false\" aria-controls=\"objectStoreSubmenu\"><i class=\"fas fa-cloud me-2\"></i>Object Store <i class=\"fas fa-chevron-down ms-auto\"></i></a><div class=\"collapse\" id=\"objectStoreSubmenu\"><ul class=\"nav flex-column ms-3\"><li class=\"nav-item\"><a class=\"nav-link py-2\" href=\"/object-store/buckets\"><i class=\"fas fa-cube me-2\"></i>Buckets</a></li><li class=\"nav-item\"><a class=\"nav-link py-2\" href=\"/object-store/users\"><i class=\"fas fa-users me-2\"></i>Users</a></li></ul></div></li><li class=\"nav-item\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if isMQPage {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 3, "<a class=\"nav-link\" href=\"#\" data-bs-toggle=\"collapse\" data-bs-target=\"#messageQueueSubmenu\" aria-expanded=\"true\" aria-controls=\"messageQueueSubmenu\"><i class=\"fas fa-comments me-2\"></i>Message Queue <i class=\"fas fa-chevron-down ms-auto\"></i></a> ")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 4, "<a class=\"nav-link collapsed\" href=\"#\" data-bs-toggle=\"collapse\" data-bs-target=\"#messageQueueSubmenu\" aria-expanded=\"false\" aria-controls=\"messageQueueSubmenu\"><i class=\"fas fa-comments me-2\"></i>Message Queue <i class=\"fas fa-chevron-down ms-auto\"></i></a> ")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ if isMQPage {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 5, "<div class=\"collapse show\" id=\"messageQueueSubmenu\"><ul class=\"nav flex-column ms-3\"><li class=\"nav-item\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if currentPath == "/mq/brokers" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 6, "<a class=\"nav-link py-2 active\" href=\"/mq/brokers\"><i class=\"fas fa-server me-2\"></i>Brokers</a>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 7, "<a class=\"nav-link py-2\" href=\"/mq/brokers\"><i class=\"fas fa-server me-2\"></i>Brokers</a>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 8, "</li><li class=\"nav-item\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if currentPath == "/mq/topics" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 9, "<a class=\"nav-link py-2 active\" href=\"/mq/topics\"><i class=\"fas fa-list-alt me-2\"></i>Topics</a>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 10, "<a class=\"nav-link py-2\" href=\"/mq/topics\"><i class=\"fas fa-list-alt me-2\"></i>Topics</a>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 11, "</li></ul></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 12, "<div class=\"collapse\" id=\"messageQueueSubmenu\"><ul class=\"nav flex-column ms-3\"><li class=\"nav-item\"><a class=\"nav-link py-2\" href=\"/mq/brokers\"><i class=\"fas fa-server me-2\"></i>Brokers</a></li><li class=\"nav-item\"><a class=\"nav-link py-2\" href=\"/mq/topics\"><i class=\"fas fa-list-alt me-2\"></i>Topics</a></li></ul></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, "</li><li class=\"nav-item\"><a class=\"nav-link\" href=\"/metrics\"><i class=\"fas fa-chart-line me-2\"></i>Metrics</a></li><li class=\"nav-item\"><a class=\"nav-link\" href=\"/logs\"><i class=\"fas fa-file-alt me-2\"></i>Logs</a></li></ul><h6 class=\"sidebar-heading px-3 mt-4 mb-1 text-muted\"><span>SYSTEM</span></h6><ul class=\"nav flex-column\"><li class=\"nav-item\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if isConfigPage {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 3, "<a class=\"nav-link\" href=\"#\" data-bs-toggle=\"collapse\" data-bs-target=\"#configurationSubmenu\" aria-expanded=\"true\" aria-controls=\"configurationSubmenu\"><i class=\"fas fa-cogs me-2\"></i>Configuration <i class=\"fas fa-chevron-down ms-auto\"></i></a> ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 14, "<a class=\"nav-link\" href=\"#\" data-bs-toggle=\"collapse\" data-bs-target=\"#configurationSubmenu\" aria-expanded=\"true\" aria-controls=\"configurationSubmenu\"><i class=\"fas fa-cogs me-2\"></i>Configuration <i class=\"fas fa-chevron-down ms-auto\"></i></a> ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 4, "<a class=\"nav-link collapsed\" href=\"#\" data-bs-toggle=\"collapse\" data-bs-target=\"#configurationSubmenu\" aria-expanded=\"false\" aria-controls=\"configurationSubmenu\"><i class=\"fas fa-cogs me-2\"></i>Configuration <i class=\"fas fa-chevron-right ms-auto\"></i></a> ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "<a class=\"nav-link collapsed\" href=\"#\" data-bs-toggle=\"collapse\" data-bs-target=\"#configurationSubmenu\" aria-expanded=\"false\" aria-controls=\"configurationSubmenu\"><i class=\"fas fa-cogs me-2\"></i>Configuration <i class=\"fas fa-chevron-right ms-auto\"></i></a> ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
if isConfigPage {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 5, "<div class=\"collapse show\" id=\"configurationSubmenu\"><ul class=\"nav flex-column ms-3\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, "<div class=\"collapse show\" id=\"configurationSubmenu\"><ul class=\"nav flex-column ms-3\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
for _, menuItem := range GetConfigurationMenuItems() {
isActiveItem := currentPath == menuItem.URL
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 6, "<li class=\"nav-item\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, "<li class=\"nav-item\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if isActiveItem {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 7, "<a class=\"nav-link py-2 active\" href=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 18, "<a class=\"nav-link py-2 active\" href=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
@@ -96,7 +155,7 @@ func Layout(c *gin.Context, content templ.Component) templ.Component {
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 8, "\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, "\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
@@ -105,7 +164,7 @@ func Layout(c *gin.Context, content templ.Component) templ.Component {
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 9, "<i class=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, "<i class=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
@@ -118,25 +177,25 @@ func Layout(c *gin.Context, content templ.Component) templ.Component {
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 10, "\"></i>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, "\"></i>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var6 string
templ_7745c5c3_Var6, templ_7745c5c3_Err = templ.JoinStringErrs(menuItem.Name)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/layout/layout.templ`, Line: 188, Col: 109}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/layout/layout.templ`, Line: 249, Col: 109}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var6))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 11, "</a>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 22, "</a>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 12, "<a class=\"nav-link py-2\" href=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 23, "<a class=\"nav-link py-2\" href=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
@@ -145,7 +204,7 @@ func Layout(c *gin.Context, content templ.Component) templ.Component {
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, "\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, "\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
@@ -154,7 +213,7 @@ func Layout(c *gin.Context, content templ.Component) templ.Component {
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 14, "<i class=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 25, "<i class=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
@@ -167,40 +226,40 @@ func Layout(c *gin.Context, content templ.Component) templ.Component {
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "\"></i>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, "\"></i>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var10 string
templ_7745c5c3_Var10, templ_7745c5c3_Err = templ.JoinStringErrs(menuItem.Name)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/layout/layout.templ`, Line: 192, Col: 109}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/layout/layout.templ`, Line: 253, Col: 109}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var10))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, "</a>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, "</a>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, "</li>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, "</li>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 18, "</ul></div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, "</ul></div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, "<div class=\"collapse\" id=\"configurationSubmenu\"><ul class=\"nav flex-column ms-3\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 30, "<div class=\"collapse\" id=\"configurationSubmenu\"><ul class=\"nav flex-column ms-3\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
for _, menuItem := range GetConfigurationMenuItems() {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, "<li class=\"nav-item\"><a class=\"nav-link py-2\" href=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 31, "<li class=\"nav-item\"><a class=\"nav-link py-2\" href=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
@@ -209,7 +268,7 @@ func Layout(c *gin.Context, content templ.Component) templ.Component {
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, "\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 32, "\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
@@ -218,7 +277,7 @@ func Layout(c *gin.Context, content templ.Component) templ.Component {
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 22, "<i class=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 33, "<i class=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
@@ -231,60 +290,60 @@ func Layout(c *gin.Context, content templ.Component) templ.Component {
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 23, "\"></i>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 34, "\"></i>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var14 string
templ_7745c5c3_Var14, templ_7745c5c3_Err = templ.JoinStringErrs(menuItem.Name)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/layout/layout.templ`, Line: 205, Col: 105}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/layout/layout.templ`, Line: 266, Col: 105}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var14))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, "</a></li>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 35, "</a></li>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 25, "</ul></div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 36, "</ul></div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, "</li><li class=\"nav-item\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 37, "</li><li class=\"nav-item\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if currentPath == "/maintenance" {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, "<a class=\"nav-link active\" href=\"/maintenance\"><i class=\"fas fa-list me-2\"></i>Maintenance Queue</a>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 38, "<a class=\"nav-link active\" href=\"/maintenance\"><i class=\"fas fa-list me-2\"></i>Maintenance Queue</a>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, "<a class=\"nav-link\" href=\"/maintenance\"><i class=\"fas fa-list me-2\"></i>Maintenance Queue</a>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 39, "<a class=\"nav-link\" href=\"/maintenance\"><i class=\"fas fa-list me-2\"></i>Maintenance Queue</a>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, "</li><li class=\"nav-item\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 40, "</li><li class=\"nav-item\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if currentPath == "/maintenance/workers" {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 30, "<a class=\"nav-link active\" href=\"/maintenance/workers\"><i class=\"fas fa-user-cog me-2\"></i>Maintenance Workers</a>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 41, "<a class=\"nav-link active\" href=\"/maintenance/workers\"><i class=\"fas fa-user-cog me-2\"></i>Maintenance Workers</a>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 31, "<a class=\"nav-link\" href=\"/maintenance/workers\"><i class=\"fas fa-user-cog me-2\"></i>Maintenance Workers</a>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 42, "<a class=\"nav-link\" href=\"/maintenance/workers\"><i class=\"fas fa-user-cog me-2\"></i>Maintenance Workers</a>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 32, "</li></ul></div></div><!-- Main content --><main class=\"col-md-9 ms-sm-auto col-lg-10 px-md-4\"><div class=\"pt-3\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 43, "</li></ul></div></div><!-- Main content --><main class=\"col-md-9 ms-sm-auto col-lg-10 px-md-4\"><div class=\"pt-3\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
@@ -292,43 +351,43 @@ func Layout(c *gin.Context, content templ.Component) templ.Component {
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 33, "</div></main></div></div><!-- Footer --><footer class=\"footer mt-auto py-3 bg-light\"><div class=\"container-fluid text-center\"><small class=\"text-muted\">&copy; ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 44, "</div></main></div></div><!-- Footer --><footer class=\"footer mt-auto py-3 bg-light\"><div class=\"container-fluid text-center\"><small class=\"text-muted\">&copy; ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var15 string
templ_7745c5c3_Var15, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", time.Now().Year()))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/layout/layout.templ`, Line: 252, Col: 60}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/layout/layout.templ`, Line: 313, Col: 60}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var15))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 34, " SeaweedFS Admin v")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 45, " SeaweedFS Admin v")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var16 string
templ_7745c5c3_Var16, templ_7745c5c3_Err = templ.JoinStringErrs(version.VERSION_NUMBER)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/layout/layout.templ`, Line: 252, Col: 102}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/layout/layout.templ`, Line: 313, Col: 102}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var16))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 35, " ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 46, " ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if !strings.Contains(version.VERSION, "enterprise") {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 36, "<span class=\"mx-2\">•</span> <a href=\"https://seaweedfs.com\" target=\"_blank\" class=\"text-decoration-none\"><i class=\"fas fa-star me-1\"></i>Enterprise Version Available</a>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 47, "<span class=\"mx-2\">•</span> <a href=\"https://seaweedfs.com\" target=\"_blank\" class=\"text-decoration-none\"><i class=\"fas fa-star me-1\"></i>Enterprise Version Available</a>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 37, "</small></div></footer><!-- Bootstrap JS --><script src=\"/static/js/bootstrap.bundle.min.js\"></script><!-- Custom JS --><script src=\"/static/js/admin.js\"></script></body></html>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 48, "</small></div></footer><!-- Bootstrap JS --><script src=\"/static/js/bootstrap.bundle.min.js\"></script><!-- Custom JS --><script src=\"/static/js/admin.js\"></script></body></html>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
@@ -357,56 +416,56 @@ func LoginForm(c *gin.Context, title string, errorMessage string) templ.Componen
templ_7745c5c3_Var17 = templ.NopComponent
}
ctx = templ.ClearChildren(ctx)
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 38, "<!doctype html><html lang=\"en\"><head><meta charset=\"UTF-8\"><title>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 49, "<!doctype html><html lang=\"en\"><head><meta charset=\"UTF-8\"><title>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var18 string
templ_7745c5c3_Var18, templ_7745c5c3_Err = templ.JoinStringErrs(title)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/layout/layout.templ`, Line: 276, Col: 17}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/layout/layout.templ`, Line: 337, Col: 17}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var18))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 39, " - Login</title><link rel=\"icon\" href=\"/static/favicon.ico\" type=\"image/x-icon\"><meta name=\"viewport\" content=\"width=device-width, initial-scale=1\"><link href=\"/static/css/bootstrap.min.css\" rel=\"stylesheet\"><link href=\"/static/css/fontawesome.min.css\" rel=\"stylesheet\"></head><body class=\"bg-light\"><div class=\"container\"><div class=\"row justify-content-center min-vh-100 align-items-center\"><div class=\"col-md-6 col-lg-4\"><div class=\"card shadow\"><div class=\"card-body p-5\"><div class=\"text-center mb-4\"><i class=\"fas fa-server fa-3x text-primary mb-3\"></i><h4 class=\"card-title\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 50, " - Login</title><link rel=\"icon\" href=\"/static/favicon.ico\" type=\"image/x-icon\"><meta name=\"viewport\" content=\"width=device-width, initial-scale=1\"><link href=\"/static/css/bootstrap.min.css\" rel=\"stylesheet\"><link href=\"/static/css/fontawesome.min.css\" rel=\"stylesheet\"></head><body class=\"bg-light\"><div class=\"container\"><div class=\"row justify-content-center min-vh-100 align-items-center\"><div class=\"col-md-6 col-lg-4\"><div class=\"card shadow\"><div class=\"card-body p-5\"><div class=\"text-center mb-4\"><i class=\"fas fa-server fa-3x text-primary mb-3\"></i><h4 class=\"card-title\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var19 string
templ_7745c5c3_Var19, templ_7745c5c3_Err = templ.JoinStringErrs(title)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/layout/layout.templ`, Line: 290, Col: 57}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/layout/layout.templ`, Line: 351, Col: 57}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var19))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 40, "</h4><p class=\"text-muted\">Please sign in to continue</p></div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 51, "</h4><p class=\"text-muted\">Please sign in to continue</p></div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if errorMessage != "" {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 41, "<div class=\"alert alert-danger\" role=\"alert\"><i class=\"fas fa-exclamation-triangle me-2\"></i> ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 52, "<div class=\"alert alert-danger\" role=\"alert\"><i class=\"fas fa-exclamation-triangle me-2\"></i> ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var20 string
templ_7745c5c3_Var20, templ_7745c5c3_Err = templ.JoinStringErrs(errorMessage)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/layout/layout.templ`, Line: 297, Col: 45}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/layout/layout.templ`, Line: 358, Col: 45}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var20))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 42, "</div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 53, "</div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 43, "<form method=\"POST\" action=\"/login\"><div class=\"mb-3\"><label for=\"username\" class=\"form-label\">Username</label><div class=\"input-group\"><span class=\"input-group-text\"><i class=\"fas fa-user\"></i></span> <input type=\"text\" class=\"form-control\" id=\"username\" name=\"username\" required></div></div><div class=\"mb-4\"><label for=\"password\" class=\"form-label\">Password</label><div class=\"input-group\"><span class=\"input-group-text\"><i class=\"fas fa-lock\"></i></span> <input type=\"password\" class=\"form-control\" id=\"password\" name=\"password\" required></div></div><button type=\"submit\" class=\"btn btn-primary w-100\"><i class=\"fas fa-sign-in-alt me-2\"></i>Sign In</button></form></div></div></div></div></div><script src=\"/static/js/bootstrap.bundle.min.js\"></script></body></html>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 54, "<form method=\"POST\" action=\"/login\"><div class=\"mb-3\"><label for=\"username\" class=\"form-label\">Username</label><div class=\"input-group\"><span class=\"input-group-text\"><i class=\"fas fa-user\"></i></span> <input type=\"text\" class=\"form-control\" id=\"username\" name=\"username\" required></div></div><div class=\"mb-4\"><label for=\"password\" class=\"form-label\">Password</label><div class=\"input-group\"><span class=\"input-group-text\"><i class=\"fas fa-lock\"></i></span> <input type=\"password\" class=\"form-control\" id=\"password\" name=\"password\" required></div></div><button type=\"submit\" class=\"btn btn-primary w-100\"><i class=\"fas fa-sign-in-alt me-2\"></i>Sign In</button></form></div></div></div></div></div><script src=\"/static/js/bootstrap.bundle.min.js\"></script></body></html>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
diff --git a/weed/command/admin.go b/weed/command/admin.go
index 027fbec68..f88ca0514 100644
--- a/weed/command/admin.go
+++ b/weed/command/admin.go
@@ -19,6 +19,7 @@ import (
"github.com/gin-gonic/gin"
"github.com/spf13/viper"
+ "github.com/seaweedfs/seaweedfs/weed/admin"
"github.com/seaweedfs/seaweedfs/weed/admin/dash"
"github.com/seaweedfs/seaweedfs/weed/admin/handlers"
"github.com/seaweedfs/seaweedfs/weed/security"
@@ -181,12 +182,12 @@ func startAdminServer(ctx context.Context, options AdminOptions) error {
store := cookie.NewStore(sessionKeyBytes)
r.Use(sessions.Sessions("admin-session", store))
- // Static files - serve from filesystem
- staticPath := filepath.Join("weed", "admin", "static")
- if _, err := os.Stat(staticPath); err == nil {
- r.Static("/static", staticPath)
+ // Static files - serve from embedded filesystem
+ staticFS, err := admin.GetStaticFS()
+ if err != nil {
+ log.Printf("Warning: Failed to load embedded static files: %v", err)
} else {
- log.Printf("Warning: Static files not found at %s", staticPath)
+ r.StaticFS("/static", http.FS(staticFS))
}
// Create data directory if specified
diff --git a/weed/command/mq_agent.go b/weed/command/mq_agent.go
index ff4023e77..d9c6170a3 100644
--- a/weed/command/mq_agent.go
+++ b/weed/command/mq_agent.go
@@ -60,14 +60,30 @@ func (mqAgentOpt *MessageQueueAgentOptions) startQueueAgent() bool {
}, grpcDialOption)
// start grpc listener
- grpcL, _, err := util.NewIpAndLocalListeners(*mqAgentOpt.ip, *mqAgentOpt.port, 0)
+ grpcL, localL, err := util.NewIpAndLocalListeners(*mqAgentOpt.ip, *mqAgentOpt.port, 0)
if err != nil {
glog.Fatalf("failed to listen on grpc port %d: %v", *mqAgentOpt.port, err)
}
- glog.Infof("Start Seaweed Message Queue Agent on %s:%d", *mqAgentOpt.ip, *mqAgentOpt.port)
+
+ // Create main gRPC server
grpcS := pb.NewGrpcServer()
mq_agent_pb.RegisterSeaweedMessagingAgentServer(grpcS, agentServer)
reflection.Register(grpcS)
+
+ // Start localhost listener if available
+ if localL != nil {
+ localGrpcS := pb.NewGrpcServer()
+ mq_agent_pb.RegisterSeaweedMessagingAgentServer(localGrpcS, agentServer)
+ reflection.Register(localGrpcS)
+ go func() {
+ glog.V(0).Infof("MQ Agent listening on localhost:%d", *mqAgentOpt.port)
+ if err := localGrpcS.Serve(localL); err != nil {
+ glog.Errorf("MQ Agent localhost listener error: %v", err)
+ }
+ }()
+ }
+
+ glog.Infof("Start Seaweed Message Queue Agent on %s:%d", *mqAgentOpt.ip, *mqAgentOpt.port)
grpcS.Serve(grpcL)
return true
diff --git a/weed/command/mq_broker.go b/weed/command/mq_broker.go
index 5eb304204..ac7deac2c 100644
--- a/weed/command/mq_broker.go
+++ b/weed/command/mq_broker.go
@@ -83,13 +83,30 @@ func (mqBrokerOpt *MessageQueueBrokerOptions) startQueueServer() bool {
}
// start grpc listener
- grpcL, _, err := util.NewIpAndLocalListeners("", *mqBrokerOpt.port, 0)
+ grpcL, localL, err := util.NewIpAndLocalListeners("", *mqBrokerOpt.port, 0)
if err != nil {
glog.Fatalf("failed to listen on grpc port %d: %v", *mqBrokerOpt.port, err)
}
+
+ // Create main gRPC server
grpcS := pb.NewGrpcServer(security.LoadServerTLS(util.GetViper(), "grpc.msg_broker"))
mq_pb.RegisterSeaweedMessagingServer(grpcS, qs)
reflection.Register(grpcS)
+
+ // Start localhost listener if available
+ if localL != nil {
+ localGrpcS := pb.NewGrpcServer(security.LoadServerTLS(util.GetViper(), "grpc.msg_broker"))
+ mq_pb.RegisterSeaweedMessagingServer(localGrpcS, qs)
+ reflection.Register(localGrpcS)
+ go func() {
+ glog.V(0).Infof("MQ Broker listening on localhost:%d", *mqBrokerOpt.port)
+ if err := localGrpcS.Serve(localL); err != nil {
+ glog.Errorf("MQ Broker localhost listener error: %v", err)
+ }
+ }()
+ }
+
+ glog.V(0).Infof("MQ Broker listening on %s:%d", *mqBrokerOpt.ip, *mqBrokerOpt.port)
grpcS.Serve(grpcL)
return true
diff --git a/weed/filer_client/filer_client_accessor.go b/weed/filer_client/filer_client_accessor.go
index 20646d343..9ec90195b 100644
--- a/weed/filer_client/filer_client_accessor.go
+++ b/weed/filer_client/filer_client_accessor.go
@@ -41,3 +41,18 @@ func (fca *FilerClientAccessor) ReadTopicConfFromFiler(t topic.Topic) (conf *mq_
return conf, nil
}
+
+// ReadTopicConfFromFilerWithMetadata reads topic configuration along with file creation and modification times
+func (fca *FilerClientAccessor) ReadTopicConfFromFilerWithMetadata(t topic.Topic) (conf *mq_pb.ConfigureTopicResponse, createdAtNs, modifiedAtNs int64, err error) {
+
+ glog.V(1).Infof("load conf with metadata for topic %v from filer", t)
+
+ if err = fca.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
+ conf, createdAtNs, modifiedAtNs, err = t.ReadConfFileWithMetadata(client)
+ return err
+ }); err != nil {
+ return nil, 0, 0, err
+ }
+
+ return conf, createdAtNs, modifiedAtNs, nil
+}
diff --git a/weed/mq/broker/broker_grpc_configure.go b/weed/mq/broker/broker_grpc_configure.go
index f827f0b37..892a43007 100644
--- a/weed/mq/broker/broker_grpc_configure.go
+++ b/weed/mq/broker/broker_grpc_configure.go
@@ -62,6 +62,7 @@ func (b *MessageQueueBroker) ConfigureTopic(ctx context.Context, request *mq_pb.
}
resp.BrokerPartitionAssignments = pub_balancer.AllocateTopicPartitions(b.PubBalancer.Brokers, request.PartitionCount)
resp.RecordType = request.RecordType
+ resp.Retention = request.Retention
// save the topic configuration on filer
if err := b.fca.SaveTopicConfToFiler(t, resp); err != nil {
diff --git a/weed/mq/broker/broker_grpc_lookup.go b/weed/mq/broker/broker_grpc_lookup.go
index 65a1ffda8..e167e4e25 100644
--- a/weed/mq/broker/broker_grpc_lookup.go
+++ b/weed/mq/broker/broker_grpc_lookup.go
@@ -3,9 +3,13 @@ package broker
import (
"context"
"fmt"
+ "strings"
+
+ "github.com/seaweedfs/seaweedfs/weed/filer"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb"
+ "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
)
@@ -50,27 +54,259 @@ func (b *MessageQueueBroker) ListTopics(ctx context.Context, request *mq_pb.List
}
ret := &mq_pb.ListTopicsResponse{}
- knownTopics := make(map[string]struct{})
- for brokerStatsItem := range b.PubBalancer.Brokers.IterBuffered() {
- _, brokerStats := brokerStatsItem.Key, brokerStatsItem.Val
- for topicPartitionStatsItem := range brokerStats.TopicPartitionStats.IterBuffered() {
- topicPartitionStat := topicPartitionStatsItem.Val
- topic := &schema_pb.Topic{
- Namespace: topicPartitionStat.TopicPartition.Namespace,
- Name: topicPartitionStat.TopicPartition.Name,
+
+ // Scan the filer directory structure to find all topics
+ err = b.fca.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
+ // List all namespaces under /topics
+ stream, err := client.ListEntries(ctx, &filer_pb.ListEntriesRequest{
+ Directory: filer.TopicsDir,
+ Limit: 1000,
+ })
+ if err != nil {
+ glog.V(0).Infof("list namespaces in %s: %v", filer.TopicsDir, err)
+ return err
+ }
+
+ // Process each namespace
+ for {
+ resp, err := stream.Recv()
+ if err != nil {
+ if err.Error() == "EOF" {
+ break
+ }
+ return err
}
- topicKey := fmt.Sprintf("%s/%s", topic.Namespace, topic.Name)
- if _, found := knownTopics[topicKey]; found {
+
+ if !resp.Entry.IsDirectory {
continue
}
- knownTopics[topicKey] = struct{}{}
- ret.Topics = append(ret.Topics, topic)
+
+ namespaceName := resp.Entry.Name
+ namespacePath := fmt.Sprintf("%s/%s", filer.TopicsDir, namespaceName)
+
+ // List all topics in this namespace
+ topicStream, err := client.ListEntries(ctx, &filer_pb.ListEntriesRequest{
+ Directory: namespacePath,
+ Limit: 1000,
+ })
+ if err != nil {
+ glog.V(0).Infof("list topics in namespace %s: %v", namespacePath, err)
+ continue
+ }
+
+ // Process each topic in the namespace
+ for {
+ topicResp, err := topicStream.Recv()
+ if err != nil {
+ if err.Error() == "EOF" {
+ break
+ }
+ glog.V(0).Infof("error reading topic stream in namespace %s: %v", namespaceName, err)
+ break
+ }
+
+ if !topicResp.Entry.IsDirectory {
+ continue
+ }
+
+ topicName := topicResp.Entry.Name
+
+ // Check if topic.conf exists
+ topicPath := fmt.Sprintf("%s/%s", namespacePath, topicName)
+ confResp, err := client.LookupDirectoryEntry(ctx, &filer_pb.LookupDirectoryEntryRequest{
+ Directory: topicPath,
+ Name: filer.TopicConfFile,
+ })
+ if err != nil {
+ glog.V(0).Infof("lookup topic.conf in %s: %v", topicPath, err)
+ continue
+ }
+
+ if confResp.Entry != nil {
+ // This is a valid topic
+ topic := &schema_pb.Topic{
+ Namespace: namespaceName,
+ Name: topicName,
+ }
+ ret.Topics = append(ret.Topics, topic)
+ }
+ }
}
+
+ return nil
+ })
+
+ if err != nil {
+ glog.V(0).Infof("list topics from filer: %v", err)
+ // Return empty response on error
+ return &mq_pb.ListTopicsResponse{}, nil
}
return ret, nil
}
+// GetTopicConfiguration returns the complete configuration of a topic including schema and partition assignments
+func (b *MessageQueueBroker) GetTopicConfiguration(ctx context.Context, request *mq_pb.GetTopicConfigurationRequest) (resp *mq_pb.GetTopicConfigurationResponse, err error) {
+ if !b.isLockOwner() {
+ proxyErr := b.withBrokerClient(false, pb.ServerAddress(b.lockAsBalancer.LockOwner()), func(client mq_pb.SeaweedMessagingClient) error {
+ resp, err = client.GetTopicConfiguration(ctx, request)
+ return nil
+ })
+ if proxyErr != nil {
+ return nil, proxyErr
+ }
+ return resp, err
+ }
+
+ t := topic.FromPbTopic(request.Topic)
+ var conf *mq_pb.ConfigureTopicResponse
+ var createdAtNs, modifiedAtNs int64
+
+ if conf, createdAtNs, modifiedAtNs, err = b.fca.ReadTopicConfFromFilerWithMetadata(t); err != nil {
+ glog.V(0).Infof("get topic configuration %s: %v", request.Topic, err)
+ return nil, fmt.Errorf("failed to read topic configuration: %v", err)
+ }
+
+ // Ensure topic assignments are active
+ err = b.ensureTopicActiveAssignments(t, conf)
+ if err != nil {
+ glog.V(0).Infof("ensure topic active assignments %s: %v", request.Topic, err)
+ return nil, fmt.Errorf("failed to ensure topic assignments: %v", err)
+ }
+
+ // Build the response with complete configuration including metadata
+ ret := &mq_pb.GetTopicConfigurationResponse{
+ Topic: request.Topic,
+ PartitionCount: int32(len(conf.BrokerPartitionAssignments)),
+ RecordType: conf.RecordType,
+ BrokerPartitionAssignments: conf.BrokerPartitionAssignments,
+ CreatedAtNs: createdAtNs,
+ LastUpdatedNs: modifiedAtNs,
+ Retention: conf.Retention,
+ }
+
+ return ret, nil
+}
+
+// GetTopicPublishers returns the active publishers for a topic
+func (b *MessageQueueBroker) GetTopicPublishers(ctx context.Context, request *mq_pb.GetTopicPublishersRequest) (resp *mq_pb.GetTopicPublishersResponse, err error) {
+ if !b.isLockOwner() {
+ proxyErr := b.withBrokerClient(false, pb.ServerAddress(b.lockAsBalancer.LockOwner()), func(client mq_pb.SeaweedMessagingClient) error {
+ resp, err = client.GetTopicPublishers(ctx, request)
+ return nil
+ })
+ if proxyErr != nil {
+ return nil, proxyErr
+ }
+ return resp, err
+ }
+
+ t := topic.FromPbTopic(request.Topic)
+ var publishers []*mq_pb.TopicPublisher
+
+ // Get topic configuration to find partition assignments
+ var conf *mq_pb.ConfigureTopicResponse
+ if conf, _, _, err = b.fca.ReadTopicConfFromFilerWithMetadata(t); err != nil {
+ glog.V(0).Infof("get topic configuration for publishers %s: %v", request.Topic, err)
+ return nil, fmt.Errorf("failed to read topic configuration: %v", err)
+ }
+
+ // Collect publishers from each partition that is hosted on this broker
+ for _, assignment := range conf.BrokerPartitionAssignments {
+ // Only collect from partitions where this broker is the leader
+ if assignment.LeaderBroker == b.option.BrokerAddress().String() {
+ partition := topic.FromPbPartition(assignment.Partition)
+ if localPartition := b.localTopicManager.GetLocalPartition(t, partition); localPartition != nil {
+ // Get publisher information from local partition
+ localPartition.Publishers.ForEachPublisher(func(clientName string, publisher *topic.LocalPublisher) {
+ connectTimeNs, lastSeenTimeNs := publisher.GetTimestamps()
+ lastPublishedOffset, lastAckedOffset := publisher.GetOffsets()
+ publishers = append(publishers, &mq_pb.TopicPublisher{
+ PublisherName: clientName,
+ ClientId: clientName, // For now, client name is used as client ID
+ Partition: assignment.Partition,
+ ConnectTimeNs: connectTimeNs,
+ LastSeenTimeNs: lastSeenTimeNs,
+ Broker: assignment.LeaderBroker,
+ IsActive: true,
+ LastPublishedOffset: lastPublishedOffset,
+ LastAckedOffset: lastAckedOffset,
+ })
+ })
+ }
+ }
+ }
+
+ return &mq_pb.GetTopicPublishersResponse{
+ Publishers: publishers,
+ }, nil
+}
+
+// GetTopicSubscribers returns the active subscribers for a topic
+func (b *MessageQueueBroker) GetTopicSubscribers(ctx context.Context, request *mq_pb.GetTopicSubscribersRequest) (resp *mq_pb.GetTopicSubscribersResponse, err error) {
+ if !b.isLockOwner() {
+ proxyErr := b.withBrokerClient(false, pb.ServerAddress(b.lockAsBalancer.LockOwner()), func(client mq_pb.SeaweedMessagingClient) error {
+ resp, err = client.GetTopicSubscribers(ctx, request)
+ return nil
+ })
+ if proxyErr != nil {
+ return nil, proxyErr
+ }
+ return resp, err
+ }
+
+ t := topic.FromPbTopic(request.Topic)
+ var subscribers []*mq_pb.TopicSubscriber
+
+ // Get topic configuration to find partition assignments
+ var conf *mq_pb.ConfigureTopicResponse
+ if conf, _, _, err = b.fca.ReadTopicConfFromFilerWithMetadata(t); err != nil {
+ glog.V(0).Infof("get topic configuration for subscribers %s: %v", request.Topic, err)
+ return nil, fmt.Errorf("failed to read topic configuration: %v", err)
+ }
+
+ // Collect subscribers from each partition that is hosted on this broker
+ for _, assignment := range conf.BrokerPartitionAssignments {
+ // Only collect from partitions where this broker is the leader
+ if assignment.LeaderBroker == b.option.BrokerAddress().String() {
+ partition := topic.FromPbPartition(assignment.Partition)
+ if localPartition := b.localTopicManager.GetLocalPartition(t, partition); localPartition != nil {
+ // Get subscriber information from local partition
+ localPartition.Subscribers.ForEachSubscriber(func(clientName string, subscriber *topic.LocalSubscriber) {
+ // Parse client name to extract consumer group and consumer ID
+ // Format is typically: "consumerGroup/consumerID"
+ consumerGroup := "default"
+ consumerID := clientName
+ if idx := strings.Index(clientName, "/"); idx != -1 {
+ consumerGroup = clientName[:idx]
+ consumerID = clientName[idx+1:]
+ }
+
+ connectTimeNs, lastSeenTimeNs := subscriber.GetTimestamps()
+ lastReceivedOffset, lastAckedOffset := subscriber.GetOffsets()
+
+ subscribers = append(subscribers, &mq_pb.TopicSubscriber{
+ ConsumerGroup: consumerGroup,
+ ConsumerId: consumerID,
+ ClientId: clientName, // Full client name as client ID
+ Partition: assignment.Partition,
+ ConnectTimeNs: connectTimeNs,
+ LastSeenTimeNs: lastSeenTimeNs,
+ Broker: assignment.LeaderBroker,
+ IsActive: true,
+ CurrentOffset: lastAckedOffset, // for compatibility
+ LastReceivedOffset: lastReceivedOffset,
+ })
+ })
+ }
+ }
+ }
+
+ return &mq_pb.GetTopicSubscribersResponse{
+ Subscribers: subscribers,
+ }, nil
+}
+
func (b *MessageQueueBroker) isLockOwner() bool {
return b.lockAsBalancer.LockOwner() == b.option.BrokerAddress().String()
}
diff --git a/weed/mq/broker/broker_grpc_pub.go b/weed/mq/broker/broker_grpc_pub.go
index f31dc7eff..d2224ad99 100644
--- a/weed/mq/broker/broker_grpc_pub.go
+++ b/weed/mq/broker/broker_grpc_pub.go
@@ -3,15 +3,16 @@ package broker
import (
"context"
"fmt"
- "github.com/seaweedfs/seaweedfs/weed/glog"
- "github.com/seaweedfs/seaweedfs/weed/mq/topic"
- "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
- "google.golang.org/grpc/peer"
"io"
"math/rand"
"net"
"sync/atomic"
"time"
+
+ "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/mq/topic"
+ "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
+ "google.golang.org/grpc/peer"
)
// PUB
@@ -69,6 +70,11 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis
var receivedSequence, acknowledgedSequence int64
var isClosed bool
+ // process each published messages
+ clientName := fmt.Sprintf("%v-%4d", findClientAddress(stream.Context()), rand.Intn(10000))
+ publisher := topic.NewLocalPublisher()
+ localTopicPartition.Publishers.AddPublisher(clientName, publisher)
+
// start sending ack to publisher
ackInterval := int64(1)
if initMessage.AckInterval > 0 {
@@ -90,6 +96,8 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis
if err := stream.Send(response); err != nil {
glog.Errorf("Error sending response %v: %v", response, err)
}
+ // Update acknowledged offset for this publisher
+ publisher.UpdateAckedOffset(acknowledgedSequence)
// println("sent ack", acknowledgedSequence, "=>", initMessage.PublisherName)
lastAckTime = time.Now()
} else {
@@ -98,10 +106,6 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis
}
}()
- // process each published messages
- clientName := fmt.Sprintf("%v-%4d/%s/%v", findClientAddress(stream.Context()), rand.Intn(10000), initMessage.Topic, initMessage.Partition)
- localTopicPartition.Publishers.AddPublisher(clientName, topic.NewLocalPublisher())
-
defer func() {
// remove the publisher
localTopicPartition.Publishers.RemovePublisher(clientName)
@@ -143,6 +147,9 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis
if err = localTopicPartition.Publish(dataMessage); err != nil {
return fmt.Errorf("topic %v partition %v publish error: %v", initMessage.Topic, initMessage.Partition, err)
}
+
+ // Update published offset and last seen time for this publisher
+ publisher.UpdatePublishedOffset(dataMessage.TsNs)
}
glog.V(0).Infof("topic %v partition %v publish stream from %s closed.", initMessage.Topic, initMessage.Partition, initMessage.PublisherName)
diff --git a/weed/mq/broker/broker_grpc_sub.go b/weed/mq/broker/broker_grpc_sub.go
index 9cdbe8325..a9fdaaf9f 100644
--- a/weed/mq/broker/broker_grpc_sub.go
+++ b/weed/mq/broker/broker_grpc_sub.go
@@ -4,6 +4,9 @@ import (
"context"
"errors"
"fmt"
+ "io"
+ "time"
+
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/sub_coordinator"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
@@ -12,8 +15,6 @@ import (
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
- "io"
- "time"
)
func (b *MessageQueueBroker) SubscribeMessage(stream mq_pb.SeaweedMessaging_SubscribeMessageServer) error {
@@ -40,7 +41,8 @@ func (b *MessageQueueBroker) SubscribeMessage(stream mq_pb.SeaweedMessaging_Subs
return getOrGenErr
}
- localTopicPartition.Subscribers.AddSubscriber(clientName, topic.NewLocalSubscriber())
+ subscriber := topic.NewLocalSubscriber()
+ localTopicPartition.Subscribers.AddSubscriber(clientName, subscriber)
glog.V(0).Infof("Subscriber %s connected on %v %v", clientName, t, partition)
isConnected := true
sleepIntervalCount := 0
@@ -115,7 +117,10 @@ func (b *MessageQueueBroker) SubscribeMessage(stream mq_pb.SeaweedMessaging_Subs
continue
}
imt.AcknowledgeMessage(ack.GetAck().Key, ack.GetAck().Sequence)
+
currentLastOffset := imt.GetOldestAckedTimestamp()
+ // Update acknowledged offset and last seen time for this subscriber when it sends an ack
+ subscriber.UpdateAckedOffset(currentLastOffset)
// fmt.Printf("%+v recv (%s,%d), oldest %d\n", partition, string(ack.GetAck().Key), ack.GetAck().Sequence, currentLastOffset)
if subscribeFollowMeStream != nil && currentLastOffset > lastOffset {
if err := subscribeFollowMeStream.Send(&mq_pb.SubscribeFollowMeRequest{
@@ -211,6 +216,9 @@ func (b *MessageQueueBroker) SubscribeMessage(stream mq_pb.SeaweedMessaging_Subs
return false, err
}
+ // Update received offset and last seen time for this subscriber
+ subscriber.UpdateReceivedOffset(logEntry.TsNs)
+
counter++
return false, nil
})
diff --git a/weed/mq/topic/local_partition_publishers.go b/weed/mq/topic/local_partition_publishers.go
index e3c4e3ca6..5142a66f2 100644
--- a/weed/mq/topic/local_partition_publishers.go
+++ b/weed/mq/topic/local_partition_publishers.go
@@ -1,20 +1,61 @@
package topic
-import "sync"
+import (
+ "sync"
+ "sync/atomic"
+ "time"
+)
type LocalPartitionPublishers struct {
publishers map[string]*LocalPublisher
publishersLock sync.RWMutex
}
type LocalPublisher struct {
+ connectTimeNs int64 // accessed atomically
+ lastSeenTimeNs int64 // accessed atomically
+ lastPublishedOffset int64 // accessed atomically - offset of last message published
+ lastAckedOffset int64 // accessed atomically - offset of last message acknowledged by broker
}
func NewLocalPublisher() *LocalPublisher {
- return &LocalPublisher{}
+ now := time.Now().UnixNano()
+ publisher := &LocalPublisher{}
+ atomic.StoreInt64(&publisher.connectTimeNs, now)
+ atomic.StoreInt64(&publisher.lastSeenTimeNs, now)
+ atomic.StoreInt64(&publisher.lastPublishedOffset, 0)
+ atomic.StoreInt64(&publisher.lastAckedOffset, 0)
+ return publisher
}
func (p *LocalPublisher) SignalShutdown() {
}
+// UpdateLastSeen updates the last activity time for this publisher
+func (p *LocalPublisher) UpdateLastSeen() {
+ atomic.StoreInt64(&p.lastSeenTimeNs, time.Now().UnixNano())
+}
+
+// UpdatePublishedOffset updates the offset of the last message published by this publisher
+func (p *LocalPublisher) UpdatePublishedOffset(offset int64) {
+ atomic.StoreInt64(&p.lastPublishedOffset, offset)
+ atomic.StoreInt64(&p.lastSeenTimeNs, time.Now().UnixNano())
+}
+
+// UpdateAckedOffset updates the offset of the last message acknowledged by the broker for this publisher
+func (p *LocalPublisher) UpdateAckedOffset(offset int64) {
+ atomic.StoreInt64(&p.lastAckedOffset, offset)
+ atomic.StoreInt64(&p.lastSeenTimeNs, time.Now().UnixNano())
+}
+
+// GetTimestamps returns the connect and last seen timestamps safely
+func (p *LocalPublisher) GetTimestamps() (connectTimeNs, lastSeenTimeNs int64) {
+ return atomic.LoadInt64(&p.connectTimeNs), atomic.LoadInt64(&p.lastSeenTimeNs)
+}
+
+// GetOffsets returns the published and acknowledged offsets safely
+func (p *LocalPublisher) GetOffsets() (lastPublishedOffset, lastAckedOffset int64) {
+ return atomic.LoadInt64(&p.lastPublishedOffset), atomic.LoadInt64(&p.lastAckedOffset)
+}
+
func NewLocalPartitionPublishers() *LocalPartitionPublishers {
return &LocalPartitionPublishers{
publishers: make(map[string]*LocalPublisher),
@@ -50,3 +91,25 @@ func (p *LocalPartitionPublishers) Size() int {
return len(p.publishers)
}
+
+// GetPublisherNames returns the names of all publishers
+func (p *LocalPartitionPublishers) GetPublisherNames() []string {
+ p.publishersLock.RLock()
+ defer p.publishersLock.RUnlock()
+
+ names := make([]string, 0, len(p.publishers))
+ for name := range p.publishers {
+ names = append(names, name)
+ }
+ return names
+}
+
+// ForEachPublisher iterates over all publishers
+func (p *LocalPartitionPublishers) ForEachPublisher(fn func(name string, publisher *LocalPublisher)) {
+ p.publishersLock.RLock()
+ defer p.publishersLock.RUnlock()
+
+ for name, publisher := range p.publishers {
+ fn(name, publisher)
+ }
+}
diff --git a/weed/mq/topic/local_partition_subscribers.go b/weed/mq/topic/local_partition_subscribers.go
index 24341ce7e..9c5d44adf 100644
--- a/weed/mq/topic/local_partition_subscribers.go
+++ b/weed/mq/topic/local_partition_subscribers.go
@@ -1,24 +1,70 @@
package topic
-import "sync"
+import (
+ "sync"
+ "sync/atomic"
+ "time"
+)
type LocalPartitionSubscribers struct {
Subscribers map[string]*LocalSubscriber
SubscribersLock sync.RWMutex
}
type LocalSubscriber struct {
- stopCh chan struct{}
+ connectTimeNs int64 // accessed atomically
+ lastSeenTimeNs int64 // accessed atomically
+ lastReceivedOffset int64 // accessed atomically - offset of last message received
+ lastAckedOffset int64 // accessed atomically - offset of last message acknowledged
+ stopCh chan struct{}
}
func NewLocalSubscriber() *LocalSubscriber {
- return &LocalSubscriber{
+ now := time.Now().UnixNano()
+ subscriber := &LocalSubscriber{
stopCh: make(chan struct{}, 1),
}
+ atomic.StoreInt64(&subscriber.connectTimeNs, now)
+ atomic.StoreInt64(&subscriber.lastSeenTimeNs, now)
+ atomic.StoreInt64(&subscriber.lastReceivedOffset, 0)
+ atomic.StoreInt64(&subscriber.lastAckedOffset, 0)
+ return subscriber
}
func (p *LocalSubscriber) SignalShutdown() {
close(p.stopCh)
}
+// UpdateLastSeen updates the last activity time for this subscriber
+func (p *LocalSubscriber) UpdateLastSeen() {
+ atomic.StoreInt64(&p.lastSeenTimeNs, time.Now().UnixNano())
+}
+
+// UpdateReceivedOffset updates the offset of the last message received by this subscriber
+func (p *LocalSubscriber) UpdateReceivedOffset(offset int64) {
+ atomic.StoreInt64(&p.lastReceivedOffset, offset)
+ atomic.StoreInt64(&p.lastSeenTimeNs, time.Now().UnixNano())
+}
+
+// UpdateAckedOffset updates the offset of the last message acknowledged by this subscriber
+func (p *LocalSubscriber) UpdateAckedOffset(offset int64) {
+ atomic.StoreInt64(&p.lastAckedOffset, offset)
+ atomic.StoreInt64(&p.lastSeenTimeNs, time.Now().UnixNano())
+}
+
+// GetTimestamps returns the connect and last seen timestamps safely
+func (p *LocalSubscriber) GetTimestamps() (connectTimeNs, lastSeenTimeNs int64) {
+ return atomic.LoadInt64(&p.connectTimeNs), atomic.LoadInt64(&p.lastSeenTimeNs)
+}
+
+// GetOffsets returns the received and acknowledged offsets safely
+func (p *LocalSubscriber) GetOffsets() (lastReceivedOffset, lastAckedOffset int64) {
+ return atomic.LoadInt64(&p.lastReceivedOffset), atomic.LoadInt64(&p.lastAckedOffset)
+}
+
+// GetCurrentOffset returns the acknowledged offset (for compatibility)
+func (p *LocalSubscriber) GetCurrentOffset() int64 {
+ return atomic.LoadInt64(&p.lastAckedOffset)
+}
+
func NewLocalPartitionSubscribers() *LocalPartitionSubscribers {
return &LocalPartitionSubscribers{
Subscribers: make(map[string]*LocalSubscriber),
@@ -54,3 +100,25 @@ func (p *LocalPartitionSubscribers) Size() int {
return len(p.Subscribers)
}
+
+// GetSubscriberNames returns the names of all subscribers
+func (p *LocalPartitionSubscribers) GetSubscriberNames() []string {
+ p.SubscribersLock.RLock()
+ defer p.SubscribersLock.RUnlock()
+
+ names := make([]string, 0, len(p.Subscribers))
+ for name := range p.Subscribers {
+ names = append(names, name)
+ }
+ return names
+}
+
+// ForEachSubscriber iterates over all subscribers
+func (p *LocalPartitionSubscribers) ForEachSubscriber(fn func(name string, subscriber *LocalSubscriber)) {
+ p.SubscribersLock.RLock()
+ defer p.SubscribersLock.RUnlock()
+
+ for name, subscriber := range p.Subscribers {
+ fn(name, subscriber)
+ }
+}
diff --git a/weed/mq/topic/topic.go b/weed/mq/topic/topic.go
index f390c7234..1e42c3f4d 100644
--- a/weed/mq/topic/topic.go
+++ b/weed/mq/topic/topic.go
@@ -2,8 +2,10 @@ package topic
import (
"bytes"
+ "context"
"errors"
"fmt"
+
"github.com/seaweedfs/seaweedfs/weed/filer"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
@@ -60,6 +62,38 @@ func (t Topic) ReadConfFile(client filer_pb.SeaweedFilerClient) (*mq_pb.Configur
return conf, nil
}
+// ReadConfFileWithMetadata reads the topic configuration and returns it along with file metadata
+func (t Topic) ReadConfFileWithMetadata(client filer_pb.SeaweedFilerClient) (*mq_pb.ConfigureTopicResponse, int64, int64, error) {
+ // Use LookupDirectoryEntry to get both content and metadata
+ request := &filer_pb.LookupDirectoryEntryRequest{
+ Directory: t.Dir(),
+ Name: filer.TopicConfFile,
+ }
+
+ resp, err := filer_pb.LookupEntry(context.Background(), client, request)
+ if err != nil {
+ if errors.Is(err, filer_pb.ErrNotFound) {
+ return nil, 0, 0, err
+ }
+ return nil, 0, 0, fmt.Errorf("lookup topic.conf of %v: %v", t, err)
+ }
+
+ // Get file metadata
+ var createdAtNs, modifiedAtNs int64
+ if resp.Entry.Attributes != nil {
+ createdAtNs = resp.Entry.Attributes.Crtime * 1e9 // convert seconds to nanoseconds
+ modifiedAtNs = resp.Entry.Attributes.Mtime * 1e9 // convert seconds to nanoseconds
+ }
+
+ // Parse the configuration
+ conf := &mq_pb.ConfigureTopicResponse{}
+ if err = jsonpb.Unmarshal(resp.Entry.Content, conf); err != nil {
+ return nil, 0, 0, fmt.Errorf("unmarshal topic %v conf: %v", t, err)
+ }
+
+ return conf, createdAtNs, modifiedAtNs, nil
+}
+
func (t Topic) WriteConfFile(client filer_pb.SeaweedFilerClient, conf *mq_pb.ConfigureTopicResponse) error {
var buf bytes.Buffer
filer.ProtoToText(&buf, conf)
diff --git a/weed/pb/mq_broker.proto b/weed/pb/mq_broker.proto
index f02ae30ea..1c9619d48 100644
--- a/weed/pb/mq_broker.proto
+++ b/weed/pb/mq_broker.proto
@@ -29,6 +29,12 @@ service SeaweedMessaging {
}
rpc LookupTopicBrokers (LookupTopicBrokersRequest) returns (LookupTopicBrokersResponse) {
}
+ rpc GetTopicConfiguration (GetTopicConfigurationRequest) returns (GetTopicConfigurationResponse) {
+ }
+ rpc GetTopicPublishers (GetTopicPublishersRequest) returns (GetTopicPublishersResponse) {
+ }
+ rpc GetTopicSubscribers (GetTopicSubscribersRequest) returns (GetTopicSubscribersResponse) {
+ }
// invoked by the balancer, running on each broker
rpc AssignTopicPartitions (AssignTopicPartitionsRequest) returns (AssignTopicPartitionsResponse) {
@@ -96,14 +102,21 @@ message BalanceTopicsResponse {
}
//////////////////////////////////////////////////
+message TopicRetention {
+ int64 retention_seconds = 1; // retention duration in seconds
+ bool enabled = 2; // whether retention is enabled
+}
+
message ConfigureTopicRequest {
schema_pb.Topic topic = 1;
int32 partition_count = 2;
schema_pb.RecordType record_type = 3;
+ TopicRetention retention = 4;
}
message ConfigureTopicResponse {
repeated BrokerPartitionAssignment broker_partition_assignments = 2;
schema_pb.RecordType record_type = 3;
+ TopicRetention retention = 4;
}
message ListTopicsRequest {
}
@@ -122,6 +135,57 @@ message BrokerPartitionAssignment {
string leader_broker = 2;
string follower_broker = 3;
}
+message GetTopicConfigurationRequest {
+ schema_pb.Topic topic = 1;
+}
+message GetTopicConfigurationResponse {
+ schema_pb.Topic topic = 1;
+ int32 partition_count = 2;
+ schema_pb.RecordType record_type = 3;
+ repeated BrokerPartitionAssignment broker_partition_assignments = 4;
+ int64 created_at_ns = 5;
+ int64 last_updated_ns = 6;
+ TopicRetention retention = 7;
+}
+
+message GetTopicPublishersRequest {
+ schema_pb.Topic topic = 1;
+}
+message GetTopicPublishersResponse {
+ repeated TopicPublisher publishers = 1;
+}
+
+message GetTopicSubscribersRequest {
+ schema_pb.Topic topic = 1;
+}
+message GetTopicSubscribersResponse {
+ repeated TopicSubscriber subscribers = 1;
+}
+
+message TopicPublisher {
+ string publisher_name = 1;
+ string client_id = 2;
+ schema_pb.Partition partition = 3;
+ int64 connect_time_ns = 4;
+ int64 last_seen_time_ns = 5;
+ string broker = 6;
+ bool is_active = 7;
+ int64 last_published_offset = 8;
+ int64 last_acked_offset = 9;
+}
+
+message TopicSubscriber {
+ string consumer_group = 1;
+ string consumer_id = 2;
+ string client_id = 3;
+ schema_pb.Partition partition = 4;
+ int64 connect_time_ns = 5;
+ int64 last_seen_time_ns = 6;
+ string broker = 7;
+ bool is_active = 8;
+ int64 current_offset = 9; // last acknowledged offset
+ int64 last_received_offset = 10;
+}
message AssignTopicPartitionsRequest {
schema_pb.Topic topic = 1;
diff --git a/weed/pb/mq_pb/mq_broker.pb.go b/weed/pb/mq_pb/mq_broker.pb.go
index 0605828a6..355b02fcb 100644
--- a/weed/pb/mq_pb/mq_broker.pb.go
+++ b/weed/pb/mq_pb/mq_broker.pb.go
@@ -430,18 +430,71 @@ func (*BalanceTopicsResponse) Descriptor() ([]byte, []int) {
}
// ////////////////////////////////////////////////
+type TopicRetention struct {
+ state protoimpl.MessageState `protogen:"open.v1"`
+ RetentionSeconds int64 `protobuf:"varint,1,opt,name=retention_seconds,json=retentionSeconds,proto3" json:"retention_seconds,omitempty"` // retention duration in seconds
+ Enabled bool `protobuf:"varint,2,opt,name=enabled,proto3" json:"enabled,omitempty"` // whether retention is enabled
+ unknownFields protoimpl.UnknownFields
+ sizeCache protoimpl.SizeCache
+}
+
+func (x *TopicRetention) Reset() {
+ *x = TopicRetention{}
+ mi := &file_mq_broker_proto_msgTypes[8]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+}
+
+func (x *TopicRetention) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*TopicRetention) ProtoMessage() {}
+
+func (x *TopicRetention) ProtoReflect() protoreflect.Message {
+ mi := &file_mq_broker_proto_msgTypes[8]
+ if x != nil {
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ if ms.LoadMessageInfo() == nil {
+ ms.StoreMessageInfo(mi)
+ }
+ return ms
+ }
+ return mi.MessageOf(x)
+}
+
+// Deprecated: Use TopicRetention.ProtoReflect.Descriptor instead.
+func (*TopicRetention) Descriptor() ([]byte, []int) {
+ return file_mq_broker_proto_rawDescGZIP(), []int{8}
+}
+
+func (x *TopicRetention) GetRetentionSeconds() int64 {
+ if x != nil {
+ return x.RetentionSeconds
+ }
+ return 0
+}
+
+func (x *TopicRetention) GetEnabled() bool {
+ if x != nil {
+ return x.Enabled
+ }
+ return false
+}
+
type ConfigureTopicRequest struct {
state protoimpl.MessageState `protogen:"open.v1"`
Topic *schema_pb.Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"`
PartitionCount int32 `protobuf:"varint,2,opt,name=partition_count,json=partitionCount,proto3" json:"partition_count,omitempty"`
RecordType *schema_pb.RecordType `protobuf:"bytes,3,opt,name=record_type,json=recordType,proto3" json:"record_type,omitempty"`
+ Retention *TopicRetention `protobuf:"bytes,4,opt,name=retention,proto3" json:"retention,omitempty"`
unknownFields protoimpl.UnknownFields
sizeCache protoimpl.SizeCache
}
func (x *ConfigureTopicRequest) Reset() {
*x = ConfigureTopicRequest{}
- mi := &file_mq_broker_proto_msgTypes[8]
+ mi := &file_mq_broker_proto_msgTypes[9]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -453,7 +506,7 @@ func (x *ConfigureTopicRequest) String() string {
func (*ConfigureTopicRequest) ProtoMessage() {}
func (x *ConfigureTopicRequest) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[8]
+ mi := &file_mq_broker_proto_msgTypes[9]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -466,7 +519,7 @@ func (x *ConfigureTopicRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use ConfigureTopicRequest.ProtoReflect.Descriptor instead.
func (*ConfigureTopicRequest) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{8}
+ return file_mq_broker_proto_rawDescGZIP(), []int{9}
}
func (x *ConfigureTopicRequest) GetTopic() *schema_pb.Topic {
@@ -490,17 +543,25 @@ func (x *ConfigureTopicRequest) GetRecordType() *schema_pb.RecordType {
return nil
}
+func (x *ConfigureTopicRequest) GetRetention() *TopicRetention {
+ if x != nil {
+ return x.Retention
+ }
+ return nil
+}
+
type ConfigureTopicResponse struct {
state protoimpl.MessageState `protogen:"open.v1"`
BrokerPartitionAssignments []*BrokerPartitionAssignment `protobuf:"bytes,2,rep,name=broker_partition_assignments,json=brokerPartitionAssignments,proto3" json:"broker_partition_assignments,omitempty"`
RecordType *schema_pb.RecordType `protobuf:"bytes,3,opt,name=record_type,json=recordType,proto3" json:"record_type,omitempty"`
+ Retention *TopicRetention `protobuf:"bytes,4,opt,name=retention,proto3" json:"retention,omitempty"`
unknownFields protoimpl.UnknownFields
sizeCache protoimpl.SizeCache
}
func (x *ConfigureTopicResponse) Reset() {
*x = ConfigureTopicResponse{}
- mi := &file_mq_broker_proto_msgTypes[9]
+ mi := &file_mq_broker_proto_msgTypes[10]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -512,7 +573,7 @@ func (x *ConfigureTopicResponse) String() string {
func (*ConfigureTopicResponse) ProtoMessage() {}
func (x *ConfigureTopicResponse) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[9]
+ mi := &file_mq_broker_proto_msgTypes[10]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -525,7 +586,7 @@ func (x *ConfigureTopicResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use ConfigureTopicResponse.ProtoReflect.Descriptor instead.
func (*ConfigureTopicResponse) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{9}
+ return file_mq_broker_proto_rawDescGZIP(), []int{10}
}
func (x *ConfigureTopicResponse) GetBrokerPartitionAssignments() []*BrokerPartitionAssignment {
@@ -542,6 +603,13 @@ func (x *ConfigureTopicResponse) GetRecordType() *schema_pb.RecordType {
return nil
}
+func (x *ConfigureTopicResponse) GetRetention() *TopicRetention {
+ if x != nil {
+ return x.Retention
+ }
+ return nil
+}
+
type ListTopicsRequest struct {
state protoimpl.MessageState `protogen:"open.v1"`
unknownFields protoimpl.UnknownFields
@@ -550,7 +618,7 @@ type ListTopicsRequest struct {
func (x *ListTopicsRequest) Reset() {
*x = ListTopicsRequest{}
- mi := &file_mq_broker_proto_msgTypes[10]
+ mi := &file_mq_broker_proto_msgTypes[11]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -562,7 +630,7 @@ func (x *ListTopicsRequest) String() string {
func (*ListTopicsRequest) ProtoMessage() {}
func (x *ListTopicsRequest) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[10]
+ mi := &file_mq_broker_proto_msgTypes[11]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -575,7 +643,7 @@ func (x *ListTopicsRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use ListTopicsRequest.ProtoReflect.Descriptor instead.
func (*ListTopicsRequest) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{10}
+ return file_mq_broker_proto_rawDescGZIP(), []int{11}
}
type ListTopicsResponse struct {
@@ -587,7 +655,7 @@ type ListTopicsResponse struct {
func (x *ListTopicsResponse) Reset() {
*x = ListTopicsResponse{}
- mi := &file_mq_broker_proto_msgTypes[11]
+ mi := &file_mq_broker_proto_msgTypes[12]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -599,7 +667,7 @@ func (x *ListTopicsResponse) String() string {
func (*ListTopicsResponse) ProtoMessage() {}
func (x *ListTopicsResponse) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[11]
+ mi := &file_mq_broker_proto_msgTypes[12]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -612,7 +680,7 @@ func (x *ListTopicsResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use ListTopicsResponse.ProtoReflect.Descriptor instead.
func (*ListTopicsResponse) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{11}
+ return file_mq_broker_proto_rawDescGZIP(), []int{12}
}
func (x *ListTopicsResponse) GetTopics() []*schema_pb.Topic {
@@ -631,7 +699,7 @@ type LookupTopicBrokersRequest struct {
func (x *LookupTopicBrokersRequest) Reset() {
*x = LookupTopicBrokersRequest{}
- mi := &file_mq_broker_proto_msgTypes[12]
+ mi := &file_mq_broker_proto_msgTypes[13]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -643,7 +711,7 @@ func (x *LookupTopicBrokersRequest) String() string {
func (*LookupTopicBrokersRequest) ProtoMessage() {}
func (x *LookupTopicBrokersRequest) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[12]
+ mi := &file_mq_broker_proto_msgTypes[13]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -656,7 +724,7 @@ func (x *LookupTopicBrokersRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use LookupTopicBrokersRequest.ProtoReflect.Descriptor instead.
func (*LookupTopicBrokersRequest) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{12}
+ return file_mq_broker_proto_rawDescGZIP(), []int{13}
}
func (x *LookupTopicBrokersRequest) GetTopic() *schema_pb.Topic {
@@ -676,7 +744,7 @@ type LookupTopicBrokersResponse struct {
func (x *LookupTopicBrokersResponse) Reset() {
*x = LookupTopicBrokersResponse{}
- mi := &file_mq_broker_proto_msgTypes[13]
+ mi := &file_mq_broker_proto_msgTypes[14]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -688,7 +756,7 @@ func (x *LookupTopicBrokersResponse) String() string {
func (*LookupTopicBrokersResponse) ProtoMessage() {}
func (x *LookupTopicBrokersResponse) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[13]
+ mi := &file_mq_broker_proto_msgTypes[14]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -701,7 +769,7 @@ func (x *LookupTopicBrokersResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use LookupTopicBrokersResponse.ProtoReflect.Descriptor instead.
func (*LookupTopicBrokersResponse) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{13}
+ return file_mq_broker_proto_rawDescGZIP(), []int{14}
}
func (x *LookupTopicBrokersResponse) GetTopic() *schema_pb.Topic {
@@ -729,7 +797,7 @@ type BrokerPartitionAssignment struct {
func (x *BrokerPartitionAssignment) Reset() {
*x = BrokerPartitionAssignment{}
- mi := &file_mq_broker_proto_msgTypes[14]
+ mi := &file_mq_broker_proto_msgTypes[15]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -741,7 +809,7 @@ func (x *BrokerPartitionAssignment) String() string {
func (*BrokerPartitionAssignment) ProtoMessage() {}
func (x *BrokerPartitionAssignment) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[14]
+ mi := &file_mq_broker_proto_msgTypes[15]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -754,7 +822,7 @@ func (x *BrokerPartitionAssignment) ProtoReflect() protoreflect.Message {
// Deprecated: Use BrokerPartitionAssignment.ProtoReflect.Descriptor instead.
func (*BrokerPartitionAssignment) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{14}
+ return file_mq_broker_proto_rawDescGZIP(), []int{15}
}
func (x *BrokerPartitionAssignment) GetPartition() *schema_pb.Partition {
@@ -778,6 +846,542 @@ func (x *BrokerPartitionAssignment) GetFollowerBroker() string {
return ""
}
+type GetTopicConfigurationRequest struct {
+ state protoimpl.MessageState `protogen:"open.v1"`
+ Topic *schema_pb.Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"`
+ unknownFields protoimpl.UnknownFields
+ sizeCache protoimpl.SizeCache
+}
+
+func (x *GetTopicConfigurationRequest) Reset() {
+ *x = GetTopicConfigurationRequest{}
+ mi := &file_mq_broker_proto_msgTypes[16]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+}
+
+func (x *GetTopicConfigurationRequest) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GetTopicConfigurationRequest) ProtoMessage() {}
+
+func (x *GetTopicConfigurationRequest) ProtoReflect() protoreflect.Message {
+ mi := &file_mq_broker_proto_msgTypes[16]
+ if x != nil {
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ if ms.LoadMessageInfo() == nil {
+ ms.StoreMessageInfo(mi)
+ }
+ return ms
+ }
+ return mi.MessageOf(x)
+}
+
+// Deprecated: Use GetTopicConfigurationRequest.ProtoReflect.Descriptor instead.
+func (*GetTopicConfigurationRequest) Descriptor() ([]byte, []int) {
+ return file_mq_broker_proto_rawDescGZIP(), []int{16}
+}
+
+func (x *GetTopicConfigurationRequest) GetTopic() *schema_pb.Topic {
+ if x != nil {
+ return x.Topic
+ }
+ return nil
+}
+
+type GetTopicConfigurationResponse struct {
+ state protoimpl.MessageState `protogen:"open.v1"`
+ Topic *schema_pb.Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"`
+ PartitionCount int32 `protobuf:"varint,2,opt,name=partition_count,json=partitionCount,proto3" json:"partition_count,omitempty"`
+ RecordType *schema_pb.RecordType `protobuf:"bytes,3,opt,name=record_type,json=recordType,proto3" json:"record_type,omitempty"`
+ BrokerPartitionAssignments []*BrokerPartitionAssignment `protobuf:"bytes,4,rep,name=broker_partition_assignments,json=brokerPartitionAssignments,proto3" json:"broker_partition_assignments,omitempty"`
+ CreatedAtNs int64 `protobuf:"varint,5,opt,name=created_at_ns,json=createdAtNs,proto3" json:"created_at_ns,omitempty"`
+ LastUpdatedNs int64 `protobuf:"varint,6,opt,name=last_updated_ns,json=lastUpdatedNs,proto3" json:"last_updated_ns,omitempty"`
+ Retention *TopicRetention `protobuf:"bytes,7,opt,name=retention,proto3" json:"retention,omitempty"`
+ unknownFields protoimpl.UnknownFields
+ sizeCache protoimpl.SizeCache
+}
+
+func (x *GetTopicConfigurationResponse) Reset() {
+ *x = GetTopicConfigurationResponse{}
+ mi := &file_mq_broker_proto_msgTypes[17]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+}
+
+func (x *GetTopicConfigurationResponse) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GetTopicConfigurationResponse) ProtoMessage() {}
+
+func (x *GetTopicConfigurationResponse) ProtoReflect() protoreflect.Message {
+ mi := &file_mq_broker_proto_msgTypes[17]
+ if x != nil {
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ if ms.LoadMessageInfo() == nil {
+ ms.StoreMessageInfo(mi)
+ }
+ return ms
+ }
+ return mi.MessageOf(x)
+}
+
+// Deprecated: Use GetTopicConfigurationResponse.ProtoReflect.Descriptor instead.
+func (*GetTopicConfigurationResponse) Descriptor() ([]byte, []int) {
+ return file_mq_broker_proto_rawDescGZIP(), []int{17}
+}
+
+func (x *GetTopicConfigurationResponse) GetTopic() *schema_pb.Topic {
+ if x != nil {
+ return x.Topic
+ }
+ return nil
+}
+
+func (x *GetTopicConfigurationResponse) GetPartitionCount() int32 {
+ if x != nil {
+ return x.PartitionCount
+ }
+ return 0
+}
+
+func (x *GetTopicConfigurationResponse) GetRecordType() *schema_pb.RecordType {
+ if x != nil {
+ return x.RecordType
+ }
+ return nil
+}
+
+func (x *GetTopicConfigurationResponse) GetBrokerPartitionAssignments() []*BrokerPartitionAssignment {
+ if x != nil {
+ return x.BrokerPartitionAssignments
+ }
+ return nil
+}
+
+func (x *GetTopicConfigurationResponse) GetCreatedAtNs() int64 {
+ if x != nil {
+ return x.CreatedAtNs
+ }
+ return 0
+}
+
+func (x *GetTopicConfigurationResponse) GetLastUpdatedNs() int64 {
+ if x != nil {
+ return x.LastUpdatedNs
+ }
+ return 0
+}
+
+func (x *GetTopicConfigurationResponse) GetRetention() *TopicRetention {
+ if x != nil {
+ return x.Retention
+ }
+ return nil
+}
+
+type GetTopicPublishersRequest struct {
+ state protoimpl.MessageState `protogen:"open.v1"`
+ Topic *schema_pb.Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"`
+ unknownFields protoimpl.UnknownFields
+ sizeCache protoimpl.SizeCache
+}
+
+func (x *GetTopicPublishersRequest) Reset() {
+ *x = GetTopicPublishersRequest{}
+ mi := &file_mq_broker_proto_msgTypes[18]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+}
+
+func (x *GetTopicPublishersRequest) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GetTopicPublishersRequest) ProtoMessage() {}
+
+func (x *GetTopicPublishersRequest) ProtoReflect() protoreflect.Message {
+ mi := &file_mq_broker_proto_msgTypes[18]
+ if x != nil {
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ if ms.LoadMessageInfo() == nil {
+ ms.StoreMessageInfo(mi)
+ }
+ return ms
+ }
+ return mi.MessageOf(x)
+}
+
+// Deprecated: Use GetTopicPublishersRequest.ProtoReflect.Descriptor instead.
+func (*GetTopicPublishersRequest) Descriptor() ([]byte, []int) {
+ return file_mq_broker_proto_rawDescGZIP(), []int{18}
+}
+
+func (x *GetTopicPublishersRequest) GetTopic() *schema_pb.Topic {
+ if x != nil {
+ return x.Topic
+ }
+ return nil
+}
+
+type GetTopicPublishersResponse struct {
+ state protoimpl.MessageState `protogen:"open.v1"`
+ Publishers []*TopicPublisher `protobuf:"bytes,1,rep,name=publishers,proto3" json:"publishers,omitempty"`
+ unknownFields protoimpl.UnknownFields
+ sizeCache protoimpl.SizeCache
+}
+
+func (x *GetTopicPublishersResponse) Reset() {
+ *x = GetTopicPublishersResponse{}
+ mi := &file_mq_broker_proto_msgTypes[19]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+}
+
+func (x *GetTopicPublishersResponse) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GetTopicPublishersResponse) ProtoMessage() {}
+
+func (x *GetTopicPublishersResponse) ProtoReflect() protoreflect.Message {
+ mi := &file_mq_broker_proto_msgTypes[19]
+ if x != nil {
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ if ms.LoadMessageInfo() == nil {
+ ms.StoreMessageInfo(mi)
+ }
+ return ms
+ }
+ return mi.MessageOf(x)
+}
+
+// Deprecated: Use GetTopicPublishersResponse.ProtoReflect.Descriptor instead.
+func (*GetTopicPublishersResponse) Descriptor() ([]byte, []int) {
+ return file_mq_broker_proto_rawDescGZIP(), []int{19}
+}
+
+func (x *GetTopicPublishersResponse) GetPublishers() []*TopicPublisher {
+ if x != nil {
+ return x.Publishers
+ }
+ return nil
+}
+
+type GetTopicSubscribersRequest struct {
+ state protoimpl.MessageState `protogen:"open.v1"`
+ Topic *schema_pb.Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"`
+ unknownFields protoimpl.UnknownFields
+ sizeCache protoimpl.SizeCache
+}
+
+func (x *GetTopicSubscribersRequest) Reset() {
+ *x = GetTopicSubscribersRequest{}
+ mi := &file_mq_broker_proto_msgTypes[20]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+}
+
+func (x *GetTopicSubscribersRequest) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GetTopicSubscribersRequest) ProtoMessage() {}
+
+func (x *GetTopicSubscribersRequest) ProtoReflect() protoreflect.Message {
+ mi := &file_mq_broker_proto_msgTypes[20]
+ if x != nil {
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ if ms.LoadMessageInfo() == nil {
+ ms.StoreMessageInfo(mi)
+ }
+ return ms
+ }
+ return mi.MessageOf(x)
+}
+
+// Deprecated: Use GetTopicSubscribersRequest.ProtoReflect.Descriptor instead.
+func (*GetTopicSubscribersRequest) Descriptor() ([]byte, []int) {
+ return file_mq_broker_proto_rawDescGZIP(), []int{20}
+}
+
+func (x *GetTopicSubscribersRequest) GetTopic() *schema_pb.Topic {
+ if x != nil {
+ return x.Topic
+ }
+ return nil
+}
+
+type GetTopicSubscribersResponse struct {
+ state protoimpl.MessageState `protogen:"open.v1"`
+ Subscribers []*TopicSubscriber `protobuf:"bytes,1,rep,name=subscribers,proto3" json:"subscribers,omitempty"`
+ unknownFields protoimpl.UnknownFields
+ sizeCache protoimpl.SizeCache
+}
+
+func (x *GetTopicSubscribersResponse) Reset() {
+ *x = GetTopicSubscribersResponse{}
+ mi := &file_mq_broker_proto_msgTypes[21]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+}
+
+func (x *GetTopicSubscribersResponse) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GetTopicSubscribersResponse) ProtoMessage() {}
+
+func (x *GetTopicSubscribersResponse) ProtoReflect() protoreflect.Message {
+ mi := &file_mq_broker_proto_msgTypes[21]
+ if x != nil {
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ if ms.LoadMessageInfo() == nil {
+ ms.StoreMessageInfo(mi)
+ }
+ return ms
+ }
+ return mi.MessageOf(x)
+}
+
+// Deprecated: Use GetTopicSubscribersResponse.ProtoReflect.Descriptor instead.
+func (*GetTopicSubscribersResponse) Descriptor() ([]byte, []int) {
+ return file_mq_broker_proto_rawDescGZIP(), []int{21}
+}
+
+func (x *GetTopicSubscribersResponse) GetSubscribers() []*TopicSubscriber {
+ if x != nil {
+ return x.Subscribers
+ }
+ return nil
+}
+
+type TopicPublisher struct {
+ state protoimpl.MessageState `protogen:"open.v1"`
+ PublisherName string `protobuf:"bytes,1,opt,name=publisher_name,json=publisherName,proto3" json:"publisher_name,omitempty"`
+ ClientId string `protobuf:"bytes,2,opt,name=client_id,json=clientId,proto3" json:"client_id,omitempty"`
+ Partition *schema_pb.Partition `protobuf:"bytes,3,opt,name=partition,proto3" json:"partition,omitempty"`
+ ConnectTimeNs int64 `protobuf:"varint,4,opt,name=connect_time_ns,json=connectTimeNs,proto3" json:"connect_time_ns,omitempty"`
+ LastSeenTimeNs int64 `protobuf:"varint,5,opt,name=last_seen_time_ns,json=lastSeenTimeNs,proto3" json:"last_seen_time_ns,omitempty"`
+ Broker string `protobuf:"bytes,6,opt,name=broker,proto3" json:"broker,omitempty"`
+ IsActive bool `protobuf:"varint,7,opt,name=is_active,json=isActive,proto3" json:"is_active,omitempty"`
+ LastPublishedOffset int64 `protobuf:"varint,8,opt,name=last_published_offset,json=lastPublishedOffset,proto3" json:"last_published_offset,omitempty"`
+ LastAckedOffset int64 `protobuf:"varint,9,opt,name=last_acked_offset,json=lastAckedOffset,proto3" json:"last_acked_offset,omitempty"`
+ unknownFields protoimpl.UnknownFields
+ sizeCache protoimpl.SizeCache
+}
+
+func (x *TopicPublisher) Reset() {
+ *x = TopicPublisher{}
+ mi := &file_mq_broker_proto_msgTypes[22]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+}
+
+func (x *TopicPublisher) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*TopicPublisher) ProtoMessage() {}
+
+func (x *TopicPublisher) ProtoReflect() protoreflect.Message {
+ mi := &file_mq_broker_proto_msgTypes[22]
+ if x != nil {
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ if ms.LoadMessageInfo() == nil {
+ ms.StoreMessageInfo(mi)
+ }
+ return ms
+ }
+ return mi.MessageOf(x)
+}
+
+// Deprecated: Use TopicPublisher.ProtoReflect.Descriptor instead.
+func (*TopicPublisher) Descriptor() ([]byte, []int) {
+ return file_mq_broker_proto_rawDescGZIP(), []int{22}
+}
+
+func (x *TopicPublisher) GetPublisherName() string {
+ if x != nil {
+ return x.PublisherName
+ }
+ return ""
+}
+
+func (x *TopicPublisher) GetClientId() string {
+ if x != nil {
+ return x.ClientId
+ }
+ return ""
+}
+
+func (x *TopicPublisher) GetPartition() *schema_pb.Partition {
+ if x != nil {
+ return x.Partition
+ }
+ return nil
+}
+
+func (x *TopicPublisher) GetConnectTimeNs() int64 {
+ if x != nil {
+ return x.ConnectTimeNs
+ }
+ return 0
+}
+
+func (x *TopicPublisher) GetLastSeenTimeNs() int64 {
+ if x != nil {
+ return x.LastSeenTimeNs
+ }
+ return 0
+}
+
+func (x *TopicPublisher) GetBroker() string {
+ if x != nil {
+ return x.Broker
+ }
+ return ""
+}
+
+func (x *TopicPublisher) GetIsActive() bool {
+ if x != nil {
+ return x.IsActive
+ }
+ return false
+}
+
+func (x *TopicPublisher) GetLastPublishedOffset() int64 {
+ if x != nil {
+ return x.LastPublishedOffset
+ }
+ return 0
+}
+
+func (x *TopicPublisher) GetLastAckedOffset() int64 {
+ if x != nil {
+ return x.LastAckedOffset
+ }
+ return 0
+}
+
+type TopicSubscriber struct {
+ state protoimpl.MessageState `protogen:"open.v1"`
+ ConsumerGroup string `protobuf:"bytes,1,opt,name=consumer_group,json=consumerGroup,proto3" json:"consumer_group,omitempty"`
+ ConsumerId string `protobuf:"bytes,2,opt,name=consumer_id,json=consumerId,proto3" json:"consumer_id,omitempty"`
+ ClientId string `protobuf:"bytes,3,opt,name=client_id,json=clientId,proto3" json:"client_id,omitempty"`
+ Partition *schema_pb.Partition `protobuf:"bytes,4,opt,name=partition,proto3" json:"partition,omitempty"`
+ ConnectTimeNs int64 `protobuf:"varint,5,opt,name=connect_time_ns,json=connectTimeNs,proto3" json:"connect_time_ns,omitempty"`
+ LastSeenTimeNs int64 `protobuf:"varint,6,opt,name=last_seen_time_ns,json=lastSeenTimeNs,proto3" json:"last_seen_time_ns,omitempty"`
+ Broker string `protobuf:"bytes,7,opt,name=broker,proto3" json:"broker,omitempty"`
+ IsActive bool `protobuf:"varint,8,opt,name=is_active,json=isActive,proto3" json:"is_active,omitempty"`
+ CurrentOffset int64 `protobuf:"varint,9,opt,name=current_offset,json=currentOffset,proto3" json:"current_offset,omitempty"` // last acknowledged offset
+ LastReceivedOffset int64 `protobuf:"varint,10,opt,name=last_received_offset,json=lastReceivedOffset,proto3" json:"last_received_offset,omitempty"`
+ unknownFields protoimpl.UnknownFields
+ sizeCache protoimpl.SizeCache
+}
+
+func (x *TopicSubscriber) Reset() {
+ *x = TopicSubscriber{}
+ mi := &file_mq_broker_proto_msgTypes[23]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+}
+
+func (x *TopicSubscriber) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*TopicSubscriber) ProtoMessage() {}
+
+func (x *TopicSubscriber) ProtoReflect() protoreflect.Message {
+ mi := &file_mq_broker_proto_msgTypes[23]
+ if x != nil {
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ if ms.LoadMessageInfo() == nil {
+ ms.StoreMessageInfo(mi)
+ }
+ return ms
+ }
+ return mi.MessageOf(x)
+}
+
+// Deprecated: Use TopicSubscriber.ProtoReflect.Descriptor instead.
+func (*TopicSubscriber) Descriptor() ([]byte, []int) {
+ return file_mq_broker_proto_rawDescGZIP(), []int{23}
+}
+
+func (x *TopicSubscriber) GetConsumerGroup() string {
+ if x != nil {
+ return x.ConsumerGroup
+ }
+ return ""
+}
+
+func (x *TopicSubscriber) GetConsumerId() string {
+ if x != nil {
+ return x.ConsumerId
+ }
+ return ""
+}
+
+func (x *TopicSubscriber) GetClientId() string {
+ if x != nil {
+ return x.ClientId
+ }
+ return ""
+}
+
+func (x *TopicSubscriber) GetPartition() *schema_pb.Partition {
+ if x != nil {
+ return x.Partition
+ }
+ return nil
+}
+
+func (x *TopicSubscriber) GetConnectTimeNs() int64 {
+ if x != nil {
+ return x.ConnectTimeNs
+ }
+ return 0
+}
+
+func (x *TopicSubscriber) GetLastSeenTimeNs() int64 {
+ if x != nil {
+ return x.LastSeenTimeNs
+ }
+ return 0
+}
+
+func (x *TopicSubscriber) GetBroker() string {
+ if x != nil {
+ return x.Broker
+ }
+ return ""
+}
+
+func (x *TopicSubscriber) GetIsActive() bool {
+ if x != nil {
+ return x.IsActive
+ }
+ return false
+}
+
+func (x *TopicSubscriber) GetCurrentOffset() int64 {
+ if x != nil {
+ return x.CurrentOffset
+ }
+ return 0
+}
+
+func (x *TopicSubscriber) GetLastReceivedOffset() int64 {
+ if x != nil {
+ return x.LastReceivedOffset
+ }
+ return 0
+}
+
type AssignTopicPartitionsRequest struct {
state protoimpl.MessageState `protogen:"open.v1"`
Topic *schema_pb.Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"`
@@ -790,7 +1394,7 @@ type AssignTopicPartitionsRequest struct {
func (x *AssignTopicPartitionsRequest) Reset() {
*x = AssignTopicPartitionsRequest{}
- mi := &file_mq_broker_proto_msgTypes[15]
+ mi := &file_mq_broker_proto_msgTypes[24]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -802,7 +1406,7 @@ func (x *AssignTopicPartitionsRequest) String() string {
func (*AssignTopicPartitionsRequest) ProtoMessage() {}
func (x *AssignTopicPartitionsRequest) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[15]
+ mi := &file_mq_broker_proto_msgTypes[24]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -815,7 +1419,7 @@ func (x *AssignTopicPartitionsRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use AssignTopicPartitionsRequest.ProtoReflect.Descriptor instead.
func (*AssignTopicPartitionsRequest) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{15}
+ return file_mq_broker_proto_rawDescGZIP(), []int{24}
}
func (x *AssignTopicPartitionsRequest) GetTopic() *schema_pb.Topic {
@@ -854,7 +1458,7 @@ type AssignTopicPartitionsResponse struct {
func (x *AssignTopicPartitionsResponse) Reset() {
*x = AssignTopicPartitionsResponse{}
- mi := &file_mq_broker_proto_msgTypes[16]
+ mi := &file_mq_broker_proto_msgTypes[25]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -866,7 +1470,7 @@ func (x *AssignTopicPartitionsResponse) String() string {
func (*AssignTopicPartitionsResponse) ProtoMessage() {}
func (x *AssignTopicPartitionsResponse) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[16]
+ mi := &file_mq_broker_proto_msgTypes[25]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -879,7 +1483,7 @@ func (x *AssignTopicPartitionsResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use AssignTopicPartitionsResponse.ProtoReflect.Descriptor instead.
func (*AssignTopicPartitionsResponse) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{16}
+ return file_mq_broker_proto_rawDescGZIP(), []int{25}
}
type SubscriberToSubCoordinatorRequest struct {
@@ -896,7 +1500,7 @@ type SubscriberToSubCoordinatorRequest struct {
func (x *SubscriberToSubCoordinatorRequest) Reset() {
*x = SubscriberToSubCoordinatorRequest{}
- mi := &file_mq_broker_proto_msgTypes[17]
+ mi := &file_mq_broker_proto_msgTypes[26]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -908,7 +1512,7 @@ func (x *SubscriberToSubCoordinatorRequest) String() string {
func (*SubscriberToSubCoordinatorRequest) ProtoMessage() {}
func (x *SubscriberToSubCoordinatorRequest) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[17]
+ mi := &file_mq_broker_proto_msgTypes[26]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -921,7 +1525,7 @@ func (x *SubscriberToSubCoordinatorRequest) ProtoReflect() protoreflect.Message
// Deprecated: Use SubscriberToSubCoordinatorRequest.ProtoReflect.Descriptor instead.
func (*SubscriberToSubCoordinatorRequest) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{17}
+ return file_mq_broker_proto_rawDescGZIP(), []int{26}
}
func (x *SubscriberToSubCoordinatorRequest) GetMessage() isSubscriberToSubCoordinatorRequest_Message {
@@ -995,7 +1599,7 @@ type SubscriberToSubCoordinatorResponse struct {
func (x *SubscriberToSubCoordinatorResponse) Reset() {
*x = SubscriberToSubCoordinatorResponse{}
- mi := &file_mq_broker_proto_msgTypes[18]
+ mi := &file_mq_broker_proto_msgTypes[27]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1007,7 +1611,7 @@ func (x *SubscriberToSubCoordinatorResponse) String() string {
func (*SubscriberToSubCoordinatorResponse) ProtoMessage() {}
func (x *SubscriberToSubCoordinatorResponse) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[18]
+ mi := &file_mq_broker_proto_msgTypes[27]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1020,7 +1624,7 @@ func (x *SubscriberToSubCoordinatorResponse) ProtoReflect() protoreflect.Message
// Deprecated: Use SubscriberToSubCoordinatorResponse.ProtoReflect.Descriptor instead.
func (*SubscriberToSubCoordinatorResponse) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{18}
+ return file_mq_broker_proto_rawDescGZIP(), []int{27}
}
func (x *SubscriberToSubCoordinatorResponse) GetMessage() isSubscriberToSubCoordinatorResponse_Message {
@@ -1077,7 +1681,7 @@ type ControlMessage struct {
func (x *ControlMessage) Reset() {
*x = ControlMessage{}
- mi := &file_mq_broker_proto_msgTypes[19]
+ mi := &file_mq_broker_proto_msgTypes[28]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1089,7 +1693,7 @@ func (x *ControlMessage) String() string {
func (*ControlMessage) ProtoMessage() {}
func (x *ControlMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[19]
+ mi := &file_mq_broker_proto_msgTypes[28]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1102,7 +1706,7 @@ func (x *ControlMessage) ProtoReflect() protoreflect.Message {
// Deprecated: Use ControlMessage.ProtoReflect.Descriptor instead.
func (*ControlMessage) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{19}
+ return file_mq_broker_proto_rawDescGZIP(), []int{28}
}
func (x *ControlMessage) GetIsClose() bool {
@@ -1131,7 +1735,7 @@ type DataMessage struct {
func (x *DataMessage) Reset() {
*x = DataMessage{}
- mi := &file_mq_broker_proto_msgTypes[20]
+ mi := &file_mq_broker_proto_msgTypes[29]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1143,7 +1747,7 @@ func (x *DataMessage) String() string {
func (*DataMessage) ProtoMessage() {}
func (x *DataMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[20]
+ mi := &file_mq_broker_proto_msgTypes[29]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1156,7 +1760,7 @@ func (x *DataMessage) ProtoReflect() protoreflect.Message {
// Deprecated: Use DataMessage.ProtoReflect.Descriptor instead.
func (*DataMessage) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{20}
+ return file_mq_broker_proto_rawDescGZIP(), []int{29}
}
func (x *DataMessage) GetKey() []byte {
@@ -1200,7 +1804,7 @@ type PublishMessageRequest struct {
func (x *PublishMessageRequest) Reset() {
*x = PublishMessageRequest{}
- mi := &file_mq_broker_proto_msgTypes[21]
+ mi := &file_mq_broker_proto_msgTypes[30]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1212,7 +1816,7 @@ func (x *PublishMessageRequest) String() string {
func (*PublishMessageRequest) ProtoMessage() {}
func (x *PublishMessageRequest) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[21]
+ mi := &file_mq_broker_proto_msgTypes[30]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1225,7 +1829,7 @@ func (x *PublishMessageRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use PublishMessageRequest.ProtoReflect.Descriptor instead.
func (*PublishMessageRequest) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{21}
+ return file_mq_broker_proto_rawDescGZIP(), []int{30}
}
func (x *PublishMessageRequest) GetMessage() isPublishMessageRequest_Message {
@@ -1280,7 +1884,7 @@ type PublishMessageResponse struct {
func (x *PublishMessageResponse) Reset() {
*x = PublishMessageResponse{}
- mi := &file_mq_broker_proto_msgTypes[22]
+ mi := &file_mq_broker_proto_msgTypes[31]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1292,7 +1896,7 @@ func (x *PublishMessageResponse) String() string {
func (*PublishMessageResponse) ProtoMessage() {}
func (x *PublishMessageResponse) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[22]
+ mi := &file_mq_broker_proto_msgTypes[31]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1305,7 +1909,7 @@ func (x *PublishMessageResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use PublishMessageResponse.ProtoReflect.Descriptor instead.
func (*PublishMessageResponse) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{22}
+ return file_mq_broker_proto_rawDescGZIP(), []int{31}
}
func (x *PublishMessageResponse) GetAckSequence() int64 {
@@ -1344,7 +1948,7 @@ type PublishFollowMeRequest struct {
func (x *PublishFollowMeRequest) Reset() {
*x = PublishFollowMeRequest{}
- mi := &file_mq_broker_proto_msgTypes[23]
+ mi := &file_mq_broker_proto_msgTypes[32]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1356,7 +1960,7 @@ func (x *PublishFollowMeRequest) String() string {
func (*PublishFollowMeRequest) ProtoMessage() {}
func (x *PublishFollowMeRequest) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[23]
+ mi := &file_mq_broker_proto_msgTypes[32]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1369,7 +1973,7 @@ func (x *PublishFollowMeRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use PublishFollowMeRequest.ProtoReflect.Descriptor instead.
func (*PublishFollowMeRequest) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{23}
+ return file_mq_broker_proto_rawDescGZIP(), []int{32}
}
func (x *PublishFollowMeRequest) GetMessage() isPublishFollowMeRequest_Message {
@@ -1452,7 +2056,7 @@ type PublishFollowMeResponse struct {
func (x *PublishFollowMeResponse) Reset() {
*x = PublishFollowMeResponse{}
- mi := &file_mq_broker_proto_msgTypes[24]
+ mi := &file_mq_broker_proto_msgTypes[33]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1464,7 +2068,7 @@ func (x *PublishFollowMeResponse) String() string {
func (*PublishFollowMeResponse) ProtoMessage() {}
func (x *PublishFollowMeResponse) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[24]
+ mi := &file_mq_broker_proto_msgTypes[33]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1477,7 +2081,7 @@ func (x *PublishFollowMeResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use PublishFollowMeResponse.ProtoReflect.Descriptor instead.
func (*PublishFollowMeResponse) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{24}
+ return file_mq_broker_proto_rawDescGZIP(), []int{33}
}
func (x *PublishFollowMeResponse) GetAckTsNs() int64 {
@@ -1500,7 +2104,7 @@ type SubscribeMessageRequest struct {
func (x *SubscribeMessageRequest) Reset() {
*x = SubscribeMessageRequest{}
- mi := &file_mq_broker_proto_msgTypes[25]
+ mi := &file_mq_broker_proto_msgTypes[34]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1512,7 +2116,7 @@ func (x *SubscribeMessageRequest) String() string {
func (*SubscribeMessageRequest) ProtoMessage() {}
func (x *SubscribeMessageRequest) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[25]
+ mi := &file_mq_broker_proto_msgTypes[34]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1525,7 +2129,7 @@ func (x *SubscribeMessageRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use SubscribeMessageRequest.ProtoReflect.Descriptor instead.
func (*SubscribeMessageRequest) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{25}
+ return file_mq_broker_proto_rawDescGZIP(), []int{34}
}
func (x *SubscribeMessageRequest) GetMessage() isSubscribeMessageRequest_Message {
@@ -1582,7 +2186,7 @@ type SubscribeMessageResponse struct {
func (x *SubscribeMessageResponse) Reset() {
*x = SubscribeMessageResponse{}
- mi := &file_mq_broker_proto_msgTypes[26]
+ mi := &file_mq_broker_proto_msgTypes[35]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1594,7 +2198,7 @@ func (x *SubscribeMessageResponse) String() string {
func (*SubscribeMessageResponse) ProtoMessage() {}
func (x *SubscribeMessageResponse) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[26]
+ mi := &file_mq_broker_proto_msgTypes[35]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1607,7 +2211,7 @@ func (x *SubscribeMessageResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use SubscribeMessageResponse.ProtoReflect.Descriptor instead.
func (*SubscribeMessageResponse) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{26}
+ return file_mq_broker_proto_rawDescGZIP(), []int{35}
}
func (x *SubscribeMessageResponse) GetMessage() isSubscribeMessageResponse_Message {
@@ -1665,7 +2269,7 @@ type SubscribeFollowMeRequest struct {
func (x *SubscribeFollowMeRequest) Reset() {
*x = SubscribeFollowMeRequest{}
- mi := &file_mq_broker_proto_msgTypes[27]
+ mi := &file_mq_broker_proto_msgTypes[36]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1677,7 +2281,7 @@ func (x *SubscribeFollowMeRequest) String() string {
func (*SubscribeFollowMeRequest) ProtoMessage() {}
func (x *SubscribeFollowMeRequest) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[27]
+ mi := &file_mq_broker_proto_msgTypes[36]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1690,7 +2294,7 @@ func (x *SubscribeFollowMeRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use SubscribeFollowMeRequest.ProtoReflect.Descriptor instead.
func (*SubscribeFollowMeRequest) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{27}
+ return file_mq_broker_proto_rawDescGZIP(), []int{36}
}
func (x *SubscribeFollowMeRequest) GetMessage() isSubscribeFollowMeRequest_Message {
@@ -1758,7 +2362,7 @@ type SubscribeFollowMeResponse struct {
func (x *SubscribeFollowMeResponse) Reset() {
*x = SubscribeFollowMeResponse{}
- mi := &file_mq_broker_proto_msgTypes[28]
+ mi := &file_mq_broker_proto_msgTypes[37]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1770,7 +2374,7 @@ func (x *SubscribeFollowMeResponse) String() string {
func (*SubscribeFollowMeResponse) ProtoMessage() {}
func (x *SubscribeFollowMeResponse) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[28]
+ mi := &file_mq_broker_proto_msgTypes[37]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1783,7 +2387,7 @@ func (x *SubscribeFollowMeResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use SubscribeFollowMeResponse.ProtoReflect.Descriptor instead.
func (*SubscribeFollowMeResponse) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{28}
+ return file_mq_broker_proto_rawDescGZIP(), []int{37}
}
func (x *SubscribeFollowMeResponse) GetAckTsNs() int64 {
@@ -1803,7 +2407,7 @@ type ClosePublishersRequest struct {
func (x *ClosePublishersRequest) Reset() {
*x = ClosePublishersRequest{}
- mi := &file_mq_broker_proto_msgTypes[29]
+ mi := &file_mq_broker_proto_msgTypes[38]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1815,7 +2419,7 @@ func (x *ClosePublishersRequest) String() string {
func (*ClosePublishersRequest) ProtoMessage() {}
func (x *ClosePublishersRequest) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[29]
+ mi := &file_mq_broker_proto_msgTypes[38]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1828,7 +2432,7 @@ func (x *ClosePublishersRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use ClosePublishersRequest.ProtoReflect.Descriptor instead.
func (*ClosePublishersRequest) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{29}
+ return file_mq_broker_proto_rawDescGZIP(), []int{38}
}
func (x *ClosePublishersRequest) GetTopic() *schema_pb.Topic {
@@ -1853,7 +2457,7 @@ type ClosePublishersResponse struct {
func (x *ClosePublishersResponse) Reset() {
*x = ClosePublishersResponse{}
- mi := &file_mq_broker_proto_msgTypes[30]
+ mi := &file_mq_broker_proto_msgTypes[39]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1865,7 +2469,7 @@ func (x *ClosePublishersResponse) String() string {
func (*ClosePublishersResponse) ProtoMessage() {}
func (x *ClosePublishersResponse) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[30]
+ mi := &file_mq_broker_proto_msgTypes[39]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1878,7 +2482,7 @@ func (x *ClosePublishersResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use ClosePublishersResponse.ProtoReflect.Descriptor instead.
func (*ClosePublishersResponse) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{30}
+ return file_mq_broker_proto_rawDescGZIP(), []int{39}
}
type CloseSubscribersRequest struct {
@@ -1891,7 +2495,7 @@ type CloseSubscribersRequest struct {
func (x *CloseSubscribersRequest) Reset() {
*x = CloseSubscribersRequest{}
- mi := &file_mq_broker_proto_msgTypes[31]
+ mi := &file_mq_broker_proto_msgTypes[40]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1903,7 +2507,7 @@ func (x *CloseSubscribersRequest) String() string {
func (*CloseSubscribersRequest) ProtoMessage() {}
func (x *CloseSubscribersRequest) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[31]
+ mi := &file_mq_broker_proto_msgTypes[40]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1916,7 +2520,7 @@ func (x *CloseSubscribersRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use CloseSubscribersRequest.ProtoReflect.Descriptor instead.
func (*CloseSubscribersRequest) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{31}
+ return file_mq_broker_proto_rawDescGZIP(), []int{40}
}
func (x *CloseSubscribersRequest) GetTopic() *schema_pb.Topic {
@@ -1941,7 +2545,7 @@ type CloseSubscribersResponse struct {
func (x *CloseSubscribersResponse) Reset() {
*x = CloseSubscribersResponse{}
- mi := &file_mq_broker_proto_msgTypes[32]
+ mi := &file_mq_broker_proto_msgTypes[41]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1953,7 +2557,7 @@ func (x *CloseSubscribersResponse) String() string {
func (*CloseSubscribersResponse) ProtoMessage() {}
func (x *CloseSubscribersResponse) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[32]
+ mi := &file_mq_broker_proto_msgTypes[41]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1966,7 +2570,7 @@ func (x *CloseSubscribersResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use CloseSubscribersResponse.ProtoReflect.Descriptor instead.
func (*CloseSubscribersResponse) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{32}
+ return file_mq_broker_proto_rawDescGZIP(), []int{41}
}
type PublisherToPubBalancerRequest_InitMessage struct {
@@ -1978,7 +2582,7 @@ type PublisherToPubBalancerRequest_InitMessage struct {
func (x *PublisherToPubBalancerRequest_InitMessage) Reset() {
*x = PublisherToPubBalancerRequest_InitMessage{}
- mi := &file_mq_broker_proto_msgTypes[34]
+ mi := &file_mq_broker_proto_msgTypes[43]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1990,7 +2594,7 @@ func (x *PublisherToPubBalancerRequest_InitMessage) String() string {
func (*PublisherToPubBalancerRequest_InitMessage) ProtoMessage() {}
func (x *PublisherToPubBalancerRequest_InitMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[34]
+ mi := &file_mq_broker_proto_msgTypes[43]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2034,7 +2638,7 @@ type SubscriberToSubCoordinatorRequest_InitMessage struct {
func (x *SubscriberToSubCoordinatorRequest_InitMessage) Reset() {
*x = SubscriberToSubCoordinatorRequest_InitMessage{}
- mi := &file_mq_broker_proto_msgTypes[35]
+ mi := &file_mq_broker_proto_msgTypes[44]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2046,7 +2650,7 @@ func (x *SubscriberToSubCoordinatorRequest_InitMessage) String() string {
func (*SubscriberToSubCoordinatorRequest_InitMessage) ProtoMessage() {}
func (x *SubscriberToSubCoordinatorRequest_InitMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[35]
+ mi := &file_mq_broker_proto_msgTypes[44]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2059,7 +2663,7 @@ func (x *SubscriberToSubCoordinatorRequest_InitMessage) ProtoReflect() protorefl
// Deprecated: Use SubscriberToSubCoordinatorRequest_InitMessage.ProtoReflect.Descriptor instead.
func (*SubscriberToSubCoordinatorRequest_InitMessage) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{17, 0}
+ return file_mq_broker_proto_rawDescGZIP(), []int{26, 0}
}
func (x *SubscriberToSubCoordinatorRequest_InitMessage) GetConsumerGroup() string {
@@ -2106,7 +2710,7 @@ type SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage struct {
func (x *SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) Reset() {
*x = SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage{}
- mi := &file_mq_broker_proto_msgTypes[36]
+ mi := &file_mq_broker_proto_msgTypes[45]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2118,7 +2722,7 @@ func (x *SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) String() stri
func (*SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) ProtoMessage() {}
func (x *SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[36]
+ mi := &file_mq_broker_proto_msgTypes[45]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2131,7 +2735,7 @@ func (x *SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) ProtoReflect(
// Deprecated: Use SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage.ProtoReflect.Descriptor instead.
func (*SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{17, 1}
+ return file_mq_broker_proto_rawDescGZIP(), []int{26, 1}
}
func (x *SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) GetPartition() *schema_pb.Partition {
@@ -2150,7 +2754,7 @@ type SubscriberToSubCoordinatorRequest_AckAssignmentMessage struct {
func (x *SubscriberToSubCoordinatorRequest_AckAssignmentMessage) Reset() {
*x = SubscriberToSubCoordinatorRequest_AckAssignmentMessage{}
- mi := &file_mq_broker_proto_msgTypes[37]
+ mi := &file_mq_broker_proto_msgTypes[46]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2162,7 +2766,7 @@ func (x *SubscriberToSubCoordinatorRequest_AckAssignmentMessage) String() string
func (*SubscriberToSubCoordinatorRequest_AckAssignmentMessage) ProtoMessage() {}
func (x *SubscriberToSubCoordinatorRequest_AckAssignmentMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[37]
+ mi := &file_mq_broker_proto_msgTypes[46]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2175,7 +2779,7 @@ func (x *SubscriberToSubCoordinatorRequest_AckAssignmentMessage) ProtoReflect()
// Deprecated: Use SubscriberToSubCoordinatorRequest_AckAssignmentMessage.ProtoReflect.Descriptor instead.
func (*SubscriberToSubCoordinatorRequest_AckAssignmentMessage) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{17, 2}
+ return file_mq_broker_proto_rawDescGZIP(), []int{26, 2}
}
func (x *SubscriberToSubCoordinatorRequest_AckAssignmentMessage) GetPartition() *schema_pb.Partition {
@@ -2194,7 +2798,7 @@ type SubscriberToSubCoordinatorResponse_Assignment struct {
func (x *SubscriberToSubCoordinatorResponse_Assignment) Reset() {
*x = SubscriberToSubCoordinatorResponse_Assignment{}
- mi := &file_mq_broker_proto_msgTypes[38]
+ mi := &file_mq_broker_proto_msgTypes[47]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2206,7 +2810,7 @@ func (x *SubscriberToSubCoordinatorResponse_Assignment) String() string {
func (*SubscriberToSubCoordinatorResponse_Assignment) ProtoMessage() {}
func (x *SubscriberToSubCoordinatorResponse_Assignment) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[38]
+ mi := &file_mq_broker_proto_msgTypes[47]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2219,7 +2823,7 @@ func (x *SubscriberToSubCoordinatorResponse_Assignment) ProtoReflect() protorefl
// Deprecated: Use SubscriberToSubCoordinatorResponse_Assignment.ProtoReflect.Descriptor instead.
func (*SubscriberToSubCoordinatorResponse_Assignment) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{18, 0}
+ return file_mq_broker_proto_rawDescGZIP(), []int{27, 0}
}
func (x *SubscriberToSubCoordinatorResponse_Assignment) GetPartitionAssignment() *BrokerPartitionAssignment {
@@ -2238,7 +2842,7 @@ type SubscriberToSubCoordinatorResponse_UnAssignment struct {
func (x *SubscriberToSubCoordinatorResponse_UnAssignment) Reset() {
*x = SubscriberToSubCoordinatorResponse_UnAssignment{}
- mi := &file_mq_broker_proto_msgTypes[39]
+ mi := &file_mq_broker_proto_msgTypes[48]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2250,7 +2854,7 @@ func (x *SubscriberToSubCoordinatorResponse_UnAssignment) String() string {
func (*SubscriberToSubCoordinatorResponse_UnAssignment) ProtoMessage() {}
func (x *SubscriberToSubCoordinatorResponse_UnAssignment) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[39]
+ mi := &file_mq_broker_proto_msgTypes[48]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2263,7 +2867,7 @@ func (x *SubscriberToSubCoordinatorResponse_UnAssignment) ProtoReflect() protore
// Deprecated: Use SubscriberToSubCoordinatorResponse_UnAssignment.ProtoReflect.Descriptor instead.
func (*SubscriberToSubCoordinatorResponse_UnAssignment) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{18, 1}
+ return file_mq_broker_proto_rawDescGZIP(), []int{27, 1}
}
func (x *SubscriberToSubCoordinatorResponse_UnAssignment) GetPartition() *schema_pb.Partition {
@@ -2286,7 +2890,7 @@ type PublishMessageRequest_InitMessage struct {
func (x *PublishMessageRequest_InitMessage) Reset() {
*x = PublishMessageRequest_InitMessage{}
- mi := &file_mq_broker_proto_msgTypes[40]
+ mi := &file_mq_broker_proto_msgTypes[49]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2298,7 +2902,7 @@ func (x *PublishMessageRequest_InitMessage) String() string {
func (*PublishMessageRequest_InitMessage) ProtoMessage() {}
func (x *PublishMessageRequest_InitMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[40]
+ mi := &file_mq_broker_proto_msgTypes[49]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2311,7 +2915,7 @@ func (x *PublishMessageRequest_InitMessage) ProtoReflect() protoreflect.Message
// Deprecated: Use PublishMessageRequest_InitMessage.ProtoReflect.Descriptor instead.
func (*PublishMessageRequest_InitMessage) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{21, 0}
+ return file_mq_broker_proto_rawDescGZIP(), []int{30, 0}
}
func (x *PublishMessageRequest_InitMessage) GetTopic() *schema_pb.Topic {
@@ -2359,7 +2963,7 @@ type PublishFollowMeRequest_InitMessage struct {
func (x *PublishFollowMeRequest_InitMessage) Reset() {
*x = PublishFollowMeRequest_InitMessage{}
- mi := &file_mq_broker_proto_msgTypes[41]
+ mi := &file_mq_broker_proto_msgTypes[50]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2371,7 +2975,7 @@ func (x *PublishFollowMeRequest_InitMessage) String() string {
func (*PublishFollowMeRequest_InitMessage) ProtoMessage() {}
func (x *PublishFollowMeRequest_InitMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[41]
+ mi := &file_mq_broker_proto_msgTypes[50]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2384,7 +2988,7 @@ func (x *PublishFollowMeRequest_InitMessage) ProtoReflect() protoreflect.Message
// Deprecated: Use PublishFollowMeRequest_InitMessage.ProtoReflect.Descriptor instead.
func (*PublishFollowMeRequest_InitMessage) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{23, 0}
+ return file_mq_broker_proto_rawDescGZIP(), []int{32, 0}
}
func (x *PublishFollowMeRequest_InitMessage) GetTopic() *schema_pb.Topic {
@@ -2410,7 +3014,7 @@ type PublishFollowMeRequest_FlushMessage struct {
func (x *PublishFollowMeRequest_FlushMessage) Reset() {
*x = PublishFollowMeRequest_FlushMessage{}
- mi := &file_mq_broker_proto_msgTypes[42]
+ mi := &file_mq_broker_proto_msgTypes[51]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2422,7 +3026,7 @@ func (x *PublishFollowMeRequest_FlushMessage) String() string {
func (*PublishFollowMeRequest_FlushMessage) ProtoMessage() {}
func (x *PublishFollowMeRequest_FlushMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[42]
+ mi := &file_mq_broker_proto_msgTypes[51]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2435,7 +3039,7 @@ func (x *PublishFollowMeRequest_FlushMessage) ProtoReflect() protoreflect.Messag
// Deprecated: Use PublishFollowMeRequest_FlushMessage.ProtoReflect.Descriptor instead.
func (*PublishFollowMeRequest_FlushMessage) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{23, 1}
+ return file_mq_broker_proto_rawDescGZIP(), []int{32, 1}
}
func (x *PublishFollowMeRequest_FlushMessage) GetTsNs() int64 {
@@ -2453,7 +3057,7 @@ type PublishFollowMeRequest_CloseMessage struct {
func (x *PublishFollowMeRequest_CloseMessage) Reset() {
*x = PublishFollowMeRequest_CloseMessage{}
- mi := &file_mq_broker_proto_msgTypes[43]
+ mi := &file_mq_broker_proto_msgTypes[52]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2465,7 +3069,7 @@ func (x *PublishFollowMeRequest_CloseMessage) String() string {
func (*PublishFollowMeRequest_CloseMessage) ProtoMessage() {}
func (x *PublishFollowMeRequest_CloseMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[43]
+ mi := &file_mq_broker_proto_msgTypes[52]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2478,7 +3082,7 @@ func (x *PublishFollowMeRequest_CloseMessage) ProtoReflect() protoreflect.Messag
// Deprecated: Use PublishFollowMeRequest_CloseMessage.ProtoReflect.Descriptor instead.
func (*PublishFollowMeRequest_CloseMessage) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{23, 2}
+ return file_mq_broker_proto_rawDescGZIP(), []int{32, 2}
}
type SubscribeMessageRequest_InitMessage struct {
@@ -2498,7 +3102,7 @@ type SubscribeMessageRequest_InitMessage struct {
func (x *SubscribeMessageRequest_InitMessage) Reset() {
*x = SubscribeMessageRequest_InitMessage{}
- mi := &file_mq_broker_proto_msgTypes[44]
+ mi := &file_mq_broker_proto_msgTypes[53]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2510,7 +3114,7 @@ func (x *SubscribeMessageRequest_InitMessage) String() string {
func (*SubscribeMessageRequest_InitMessage) ProtoMessage() {}
func (x *SubscribeMessageRequest_InitMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[44]
+ mi := &file_mq_broker_proto_msgTypes[53]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2523,7 +3127,7 @@ func (x *SubscribeMessageRequest_InitMessage) ProtoReflect() protoreflect.Messag
// Deprecated: Use SubscribeMessageRequest_InitMessage.ProtoReflect.Descriptor instead.
func (*SubscribeMessageRequest_InitMessage) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{25, 0}
+ return file_mq_broker_proto_rawDescGZIP(), []int{34, 0}
}
func (x *SubscribeMessageRequest_InitMessage) GetConsumerGroup() string {
@@ -2599,7 +3203,7 @@ type SubscribeMessageRequest_AckMessage struct {
func (x *SubscribeMessageRequest_AckMessage) Reset() {
*x = SubscribeMessageRequest_AckMessage{}
- mi := &file_mq_broker_proto_msgTypes[45]
+ mi := &file_mq_broker_proto_msgTypes[54]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2611,7 +3215,7 @@ func (x *SubscribeMessageRequest_AckMessage) String() string {
func (*SubscribeMessageRequest_AckMessage) ProtoMessage() {}
func (x *SubscribeMessageRequest_AckMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[45]
+ mi := &file_mq_broker_proto_msgTypes[54]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2624,7 +3228,7 @@ func (x *SubscribeMessageRequest_AckMessage) ProtoReflect() protoreflect.Message
// Deprecated: Use SubscribeMessageRequest_AckMessage.ProtoReflect.Descriptor instead.
func (*SubscribeMessageRequest_AckMessage) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{25, 1}
+ return file_mq_broker_proto_rawDescGZIP(), []int{34, 1}
}
func (x *SubscribeMessageRequest_AckMessage) GetSequence() int64 {
@@ -2652,7 +3256,7 @@ type SubscribeMessageResponse_SubscribeCtrlMessage struct {
func (x *SubscribeMessageResponse_SubscribeCtrlMessage) Reset() {
*x = SubscribeMessageResponse_SubscribeCtrlMessage{}
- mi := &file_mq_broker_proto_msgTypes[46]
+ mi := &file_mq_broker_proto_msgTypes[55]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2664,7 +3268,7 @@ func (x *SubscribeMessageResponse_SubscribeCtrlMessage) String() string {
func (*SubscribeMessageResponse_SubscribeCtrlMessage) ProtoMessage() {}
func (x *SubscribeMessageResponse_SubscribeCtrlMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[46]
+ mi := &file_mq_broker_proto_msgTypes[55]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2677,7 +3281,7 @@ func (x *SubscribeMessageResponse_SubscribeCtrlMessage) ProtoReflect() protorefl
// Deprecated: Use SubscribeMessageResponse_SubscribeCtrlMessage.ProtoReflect.Descriptor instead.
func (*SubscribeMessageResponse_SubscribeCtrlMessage) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{26, 0}
+ return file_mq_broker_proto_rawDescGZIP(), []int{35, 0}
}
func (x *SubscribeMessageResponse_SubscribeCtrlMessage) GetError() string {
@@ -2712,7 +3316,7 @@ type SubscribeFollowMeRequest_InitMessage struct {
func (x *SubscribeFollowMeRequest_InitMessage) Reset() {
*x = SubscribeFollowMeRequest_InitMessage{}
- mi := &file_mq_broker_proto_msgTypes[47]
+ mi := &file_mq_broker_proto_msgTypes[56]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2724,7 +3328,7 @@ func (x *SubscribeFollowMeRequest_InitMessage) String() string {
func (*SubscribeFollowMeRequest_InitMessage) ProtoMessage() {}
func (x *SubscribeFollowMeRequest_InitMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[47]
+ mi := &file_mq_broker_proto_msgTypes[56]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2737,7 +3341,7 @@ func (x *SubscribeFollowMeRequest_InitMessage) ProtoReflect() protoreflect.Messa
// Deprecated: Use SubscribeFollowMeRequest_InitMessage.ProtoReflect.Descriptor instead.
func (*SubscribeFollowMeRequest_InitMessage) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{27, 0}
+ return file_mq_broker_proto_rawDescGZIP(), []int{36, 0}
}
func (x *SubscribeFollowMeRequest_InitMessage) GetTopic() *schema_pb.Topic {
@@ -2770,7 +3374,7 @@ type SubscribeFollowMeRequest_AckMessage struct {
func (x *SubscribeFollowMeRequest_AckMessage) Reset() {
*x = SubscribeFollowMeRequest_AckMessage{}
- mi := &file_mq_broker_proto_msgTypes[48]
+ mi := &file_mq_broker_proto_msgTypes[57]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2782,7 +3386,7 @@ func (x *SubscribeFollowMeRequest_AckMessage) String() string {
func (*SubscribeFollowMeRequest_AckMessage) ProtoMessage() {}
func (x *SubscribeFollowMeRequest_AckMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[48]
+ mi := &file_mq_broker_proto_msgTypes[57]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2795,7 +3399,7 @@ func (x *SubscribeFollowMeRequest_AckMessage) ProtoReflect() protoreflect.Messag
// Deprecated: Use SubscribeFollowMeRequest_AckMessage.ProtoReflect.Descriptor instead.
func (*SubscribeFollowMeRequest_AckMessage) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{27, 1}
+ return file_mq_broker_proto_rawDescGZIP(), []int{36, 1}
}
func (x *SubscribeFollowMeRequest_AckMessage) GetTsNs() int64 {
@@ -2813,7 +3417,7 @@ type SubscribeFollowMeRequest_CloseMessage struct {
func (x *SubscribeFollowMeRequest_CloseMessage) Reset() {
*x = SubscribeFollowMeRequest_CloseMessage{}
- mi := &file_mq_broker_proto_msgTypes[49]
+ mi := &file_mq_broker_proto_msgTypes[58]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2825,7 +3429,7 @@ func (x *SubscribeFollowMeRequest_CloseMessage) String() string {
func (*SubscribeFollowMeRequest_CloseMessage) ProtoMessage() {}
func (x *SubscribeFollowMeRequest_CloseMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_broker_proto_msgTypes[49]
+ mi := &file_mq_broker_proto_msgTypes[58]
if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2838,7 +3442,7 @@ func (x *SubscribeFollowMeRequest_CloseMessage) ProtoReflect() protoreflect.Mess
// Deprecated: Use SubscribeFollowMeRequest_CloseMessage.ProtoReflect.Descriptor instead.
func (*SubscribeFollowMeRequest_CloseMessage) Descriptor() ([]byte, []int) {
- return file_mq_broker_proto_rawDescGZIP(), []int{27, 2}
+ return file_mq_broker_proto_rawDescGZIP(), []int{36, 2}
}
var File_mq_broker_proto protoreflect.FileDescriptor
@@ -2872,16 +3476,21 @@ const file_mq_broker_proto_rawDesc = "" +
"\amessage\" \n" +
"\x1ePublisherToPubBalancerResponse\"\x16\n" +
"\x14BalanceTopicsRequest\"\x17\n" +
- "\x15BalanceTopicsResponse\"\xa0\x01\n" +
+ "\x15BalanceTopicsResponse\"W\n" +
+ "\x0eTopicRetention\x12+\n" +
+ "\x11retention_seconds\x18\x01 \x01(\x03R\x10retentionSeconds\x12\x18\n" +
+ "\aenabled\x18\x02 \x01(\bR\aenabled\"\xdc\x01\n" +
"\x15ConfigureTopicRequest\x12&\n" +
"\x05topic\x18\x01 \x01(\v2\x10.schema_pb.TopicR\x05topic\x12'\n" +
"\x0fpartition_count\x18\x02 \x01(\x05R\x0epartitionCount\x126\n" +
"\vrecord_type\x18\x03 \x01(\v2\x15.schema_pb.RecordTypeR\n" +
- "recordType\"\xbb\x01\n" +
+ "recordType\x12:\n" +
+ "\tretention\x18\x04 \x01(\v2\x1c.messaging_pb.TopicRetentionR\tretention\"\xf7\x01\n" +
"\x16ConfigureTopicResponse\x12i\n" +
"\x1cbroker_partition_assignments\x18\x02 \x03(\v2'.messaging_pb.BrokerPartitionAssignmentR\x1abrokerPartitionAssignments\x126\n" +
"\vrecord_type\x18\x03 \x01(\v2\x15.schema_pb.RecordTypeR\n" +
- "recordType\"\x13\n" +
+ "recordType\x12:\n" +
+ "\tretention\x18\x04 \x01(\v2\x1c.messaging_pb.TopicRetentionR\tretention\"\x13\n" +
"\x11ListTopicsRequest\">\n" +
"\x12ListTopicsResponse\x12(\n" +
"\x06topics\x18\x01 \x03(\v2\x10.schema_pb.TopicR\x06topics\"C\n" +
@@ -2893,7 +3502,51 @@ const file_mq_broker_proto_rawDesc = "" +
"\x19BrokerPartitionAssignment\x122\n" +
"\tpartition\x18\x01 \x01(\v2\x14.schema_pb.PartitionR\tpartition\x12#\n" +
"\rleader_broker\x18\x02 \x01(\tR\fleaderBroker\x12'\n" +
- "\x0ffollower_broker\x18\x03 \x01(\tR\x0efollowerBroker\"\xef\x01\n" +
+ "\x0ffollower_broker\x18\x03 \x01(\tR\x0efollowerBroker\"F\n" +
+ "\x1cGetTopicConfigurationRequest\x12&\n" +
+ "\x05topic\x18\x01 \x01(\v2\x10.schema_pb.TopicR\x05topic\"\x9b\x03\n" +
+ "\x1dGetTopicConfigurationResponse\x12&\n" +
+ "\x05topic\x18\x01 \x01(\v2\x10.schema_pb.TopicR\x05topic\x12'\n" +
+ "\x0fpartition_count\x18\x02 \x01(\x05R\x0epartitionCount\x126\n" +
+ "\vrecord_type\x18\x03 \x01(\v2\x15.schema_pb.RecordTypeR\n" +
+ "recordType\x12i\n" +
+ "\x1cbroker_partition_assignments\x18\x04 \x03(\v2'.messaging_pb.BrokerPartitionAssignmentR\x1abrokerPartitionAssignments\x12\"\n" +
+ "\rcreated_at_ns\x18\x05 \x01(\x03R\vcreatedAtNs\x12&\n" +
+ "\x0flast_updated_ns\x18\x06 \x01(\x03R\rlastUpdatedNs\x12:\n" +
+ "\tretention\x18\a \x01(\v2\x1c.messaging_pb.TopicRetentionR\tretention\"C\n" +
+ "\x19GetTopicPublishersRequest\x12&\n" +
+ "\x05topic\x18\x01 \x01(\v2\x10.schema_pb.TopicR\x05topic\"Z\n" +
+ "\x1aGetTopicPublishersResponse\x12<\n" +
+ "\n" +
+ "publishers\x18\x01 \x03(\v2\x1c.messaging_pb.TopicPublisherR\n" +
+ "publishers\"D\n" +
+ "\x1aGetTopicSubscribersRequest\x12&\n" +
+ "\x05topic\x18\x01 \x01(\v2\x10.schema_pb.TopicR\x05topic\"^\n" +
+ "\x1bGetTopicSubscribersResponse\x12?\n" +
+ "\vsubscribers\x18\x01 \x03(\v2\x1d.messaging_pb.TopicSubscriberR\vsubscribers\"\xf0\x02\n" +
+ "\x0eTopicPublisher\x12%\n" +
+ "\x0epublisher_name\x18\x01 \x01(\tR\rpublisherName\x12\x1b\n" +
+ "\tclient_id\x18\x02 \x01(\tR\bclientId\x122\n" +
+ "\tpartition\x18\x03 \x01(\v2\x14.schema_pb.PartitionR\tpartition\x12&\n" +
+ "\x0fconnect_time_ns\x18\x04 \x01(\x03R\rconnectTimeNs\x12)\n" +
+ "\x11last_seen_time_ns\x18\x05 \x01(\x03R\x0elastSeenTimeNs\x12\x16\n" +
+ "\x06broker\x18\x06 \x01(\tR\x06broker\x12\x1b\n" +
+ "\tis_active\x18\a \x01(\bR\bisActive\x122\n" +
+ "\x15last_published_offset\x18\b \x01(\x03R\x13lastPublishedOffset\x12*\n" +
+ "\x11last_acked_offset\x18\t \x01(\x03R\x0flastAckedOffset\"\x8b\x03\n" +
+ "\x0fTopicSubscriber\x12%\n" +
+ "\x0econsumer_group\x18\x01 \x01(\tR\rconsumerGroup\x12\x1f\n" +
+ "\vconsumer_id\x18\x02 \x01(\tR\n" +
+ "consumerId\x12\x1b\n" +
+ "\tclient_id\x18\x03 \x01(\tR\bclientId\x122\n" +
+ "\tpartition\x18\x04 \x01(\v2\x14.schema_pb.PartitionR\tpartition\x12&\n" +
+ "\x0fconnect_time_ns\x18\x05 \x01(\x03R\rconnectTimeNs\x12)\n" +
+ "\x11last_seen_time_ns\x18\x06 \x01(\x03R\x0elastSeenTimeNs\x12\x16\n" +
+ "\x06broker\x18\a \x01(\tR\x06broker\x12\x1b\n" +
+ "\tis_active\x18\b \x01(\bR\bisActive\x12%\n" +
+ "\x0ecurrent_offset\x18\t \x01(\x03R\rcurrentOffset\x120\n" +
+ "\x14last_received_offset\x18\n" +
+ " \x01(\x03R\x12lastReceivedOffset\"\xef\x01\n" +
"\x1cAssignTopicPartitionsRequest\x12&\n" +
"\x05topic\x18\x01 \x01(\v2\x10.schema_pb.TopicR\x05topic\x12i\n" +
"\x1cbroker_partition_assignments\x18\x02 \x03(\v2'.messaging_pb.BrokerPartitionAssignmentR\x1abrokerPartitionAssignments\x12\x1b\n" +
@@ -3016,7 +3669,7 @@ const file_mq_broker_proto_rawDesc = "" +
"\x05topic\x18\x01 \x01(\v2\x10.schema_pb.TopicR\x05topic\x12 \n" +
"\funix_time_ns\x18\x02 \x01(\x03R\n" +
"unixTimeNs\"\x1a\n" +
- "\x18CloseSubscribersResponse2\xca\v\n" +
+ "\x18CloseSubscribersResponse2\x97\x0e\n" +
"\x10SeaweedMessaging\x12c\n" +
"\x10FindBrokerLeader\x12%.messaging_pb.FindBrokerLeaderRequest\x1a&.messaging_pb.FindBrokerLeaderResponse\"\x00\x12y\n" +
"\x16PublisherToPubBalancer\x12+.messaging_pb.PublisherToPubBalancerRequest\x1a,.messaging_pb.PublisherToPubBalancerResponse\"\x00(\x010\x01\x12Z\n" +
@@ -3025,6 +3678,9 @@ const file_mq_broker_proto_rawDesc = "" +
"ListTopics\x12\x1f.messaging_pb.ListTopicsRequest\x1a .messaging_pb.ListTopicsResponse\"\x00\x12]\n" +
"\x0eConfigureTopic\x12#.messaging_pb.ConfigureTopicRequest\x1a$.messaging_pb.ConfigureTopicResponse\"\x00\x12i\n" +
"\x12LookupTopicBrokers\x12'.messaging_pb.LookupTopicBrokersRequest\x1a(.messaging_pb.LookupTopicBrokersResponse\"\x00\x12r\n" +
+ "\x15GetTopicConfiguration\x12*.messaging_pb.GetTopicConfigurationRequest\x1a+.messaging_pb.GetTopicConfigurationResponse\"\x00\x12i\n" +
+ "\x12GetTopicPublishers\x12'.messaging_pb.GetTopicPublishersRequest\x1a(.messaging_pb.GetTopicPublishersResponse\"\x00\x12l\n" +
+ "\x13GetTopicSubscribers\x12(.messaging_pb.GetTopicSubscribersRequest\x1a).messaging_pb.GetTopicSubscribersResponse\"\x00\x12r\n" +
"\x15AssignTopicPartitions\x12*.messaging_pb.AssignTopicPartitionsRequest\x1a+.messaging_pb.AssignTopicPartitionsResponse\"\x00\x12`\n" +
"\x0fClosePublishers\x12$.messaging_pb.ClosePublishersRequest\x1a%.messaging_pb.ClosePublishersResponse\"\x00\x12c\n" +
"\x10CloseSubscribers\x12%.messaging_pb.CloseSubscribersRequest\x1a&.messaging_pb.CloseSubscribersResponse\"\x00\x12\x85\x01\n" +
@@ -3047,7 +3703,7 @@ func file_mq_broker_proto_rawDescGZIP() []byte {
return file_mq_broker_proto_rawDescData
}
-var file_mq_broker_proto_msgTypes = make([]protoimpl.MessageInfo, 50)
+var file_mq_broker_proto_msgTypes = make([]protoimpl.MessageInfo, 59)
var file_mq_broker_proto_goTypes = []any{
(*FindBrokerLeaderRequest)(nil), // 0: messaging_pb.FindBrokerLeaderRequest
(*FindBrokerLeaderResponse)(nil), // 1: messaging_pb.FindBrokerLeaderResponse
@@ -3057,140 +3713,168 @@ var file_mq_broker_proto_goTypes = []any{
(*PublisherToPubBalancerResponse)(nil), // 5: messaging_pb.PublisherToPubBalancerResponse
(*BalanceTopicsRequest)(nil), // 6: messaging_pb.BalanceTopicsRequest
(*BalanceTopicsResponse)(nil), // 7: messaging_pb.BalanceTopicsResponse
- (*ConfigureTopicRequest)(nil), // 8: messaging_pb.ConfigureTopicRequest
- (*ConfigureTopicResponse)(nil), // 9: messaging_pb.ConfigureTopicResponse
- (*ListTopicsRequest)(nil), // 10: messaging_pb.ListTopicsRequest
- (*ListTopicsResponse)(nil), // 11: messaging_pb.ListTopicsResponse
- (*LookupTopicBrokersRequest)(nil), // 12: messaging_pb.LookupTopicBrokersRequest
- (*LookupTopicBrokersResponse)(nil), // 13: messaging_pb.LookupTopicBrokersResponse
- (*BrokerPartitionAssignment)(nil), // 14: messaging_pb.BrokerPartitionAssignment
- (*AssignTopicPartitionsRequest)(nil), // 15: messaging_pb.AssignTopicPartitionsRequest
- (*AssignTopicPartitionsResponse)(nil), // 16: messaging_pb.AssignTopicPartitionsResponse
- (*SubscriberToSubCoordinatorRequest)(nil), // 17: messaging_pb.SubscriberToSubCoordinatorRequest
- (*SubscriberToSubCoordinatorResponse)(nil), // 18: messaging_pb.SubscriberToSubCoordinatorResponse
- (*ControlMessage)(nil), // 19: messaging_pb.ControlMessage
- (*DataMessage)(nil), // 20: messaging_pb.DataMessage
- (*PublishMessageRequest)(nil), // 21: messaging_pb.PublishMessageRequest
- (*PublishMessageResponse)(nil), // 22: messaging_pb.PublishMessageResponse
- (*PublishFollowMeRequest)(nil), // 23: messaging_pb.PublishFollowMeRequest
- (*PublishFollowMeResponse)(nil), // 24: messaging_pb.PublishFollowMeResponse
- (*SubscribeMessageRequest)(nil), // 25: messaging_pb.SubscribeMessageRequest
- (*SubscribeMessageResponse)(nil), // 26: messaging_pb.SubscribeMessageResponse
- (*SubscribeFollowMeRequest)(nil), // 27: messaging_pb.SubscribeFollowMeRequest
- (*SubscribeFollowMeResponse)(nil), // 28: messaging_pb.SubscribeFollowMeResponse
- (*ClosePublishersRequest)(nil), // 29: messaging_pb.ClosePublishersRequest
- (*ClosePublishersResponse)(nil), // 30: messaging_pb.ClosePublishersResponse
- (*CloseSubscribersRequest)(nil), // 31: messaging_pb.CloseSubscribersRequest
- (*CloseSubscribersResponse)(nil), // 32: messaging_pb.CloseSubscribersResponse
- nil, // 33: messaging_pb.BrokerStats.StatsEntry
- (*PublisherToPubBalancerRequest_InitMessage)(nil), // 34: messaging_pb.PublisherToPubBalancerRequest.InitMessage
- (*SubscriberToSubCoordinatorRequest_InitMessage)(nil), // 35: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage
- (*SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage)(nil), // 36: messaging_pb.SubscriberToSubCoordinatorRequest.AckUnAssignmentMessage
- (*SubscriberToSubCoordinatorRequest_AckAssignmentMessage)(nil), // 37: messaging_pb.SubscriberToSubCoordinatorRequest.AckAssignmentMessage
- (*SubscriberToSubCoordinatorResponse_Assignment)(nil), // 38: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment
- (*SubscriberToSubCoordinatorResponse_UnAssignment)(nil), // 39: messaging_pb.SubscriberToSubCoordinatorResponse.UnAssignment
- (*PublishMessageRequest_InitMessage)(nil), // 40: messaging_pb.PublishMessageRequest.InitMessage
- (*PublishFollowMeRequest_InitMessage)(nil), // 41: messaging_pb.PublishFollowMeRequest.InitMessage
- (*PublishFollowMeRequest_FlushMessage)(nil), // 42: messaging_pb.PublishFollowMeRequest.FlushMessage
- (*PublishFollowMeRequest_CloseMessage)(nil), // 43: messaging_pb.PublishFollowMeRequest.CloseMessage
- (*SubscribeMessageRequest_InitMessage)(nil), // 44: messaging_pb.SubscribeMessageRequest.InitMessage
- (*SubscribeMessageRequest_AckMessage)(nil), // 45: messaging_pb.SubscribeMessageRequest.AckMessage
- (*SubscribeMessageResponse_SubscribeCtrlMessage)(nil), // 46: messaging_pb.SubscribeMessageResponse.SubscribeCtrlMessage
- (*SubscribeFollowMeRequest_InitMessage)(nil), // 47: messaging_pb.SubscribeFollowMeRequest.InitMessage
- (*SubscribeFollowMeRequest_AckMessage)(nil), // 48: messaging_pb.SubscribeFollowMeRequest.AckMessage
- (*SubscribeFollowMeRequest_CloseMessage)(nil), // 49: messaging_pb.SubscribeFollowMeRequest.CloseMessage
- (*schema_pb.Topic)(nil), // 50: schema_pb.Topic
- (*schema_pb.Partition)(nil), // 51: schema_pb.Partition
- (*schema_pb.RecordType)(nil), // 52: schema_pb.RecordType
- (*schema_pb.PartitionOffset)(nil), // 53: schema_pb.PartitionOffset
- (schema_pb.OffsetType)(0), // 54: schema_pb.OffsetType
+ (*TopicRetention)(nil), // 8: messaging_pb.TopicRetention
+ (*ConfigureTopicRequest)(nil), // 9: messaging_pb.ConfigureTopicRequest
+ (*ConfigureTopicResponse)(nil), // 10: messaging_pb.ConfigureTopicResponse
+ (*ListTopicsRequest)(nil), // 11: messaging_pb.ListTopicsRequest
+ (*ListTopicsResponse)(nil), // 12: messaging_pb.ListTopicsResponse
+ (*LookupTopicBrokersRequest)(nil), // 13: messaging_pb.LookupTopicBrokersRequest
+ (*LookupTopicBrokersResponse)(nil), // 14: messaging_pb.LookupTopicBrokersResponse
+ (*BrokerPartitionAssignment)(nil), // 15: messaging_pb.BrokerPartitionAssignment
+ (*GetTopicConfigurationRequest)(nil), // 16: messaging_pb.GetTopicConfigurationRequest
+ (*GetTopicConfigurationResponse)(nil), // 17: messaging_pb.GetTopicConfigurationResponse
+ (*GetTopicPublishersRequest)(nil), // 18: messaging_pb.GetTopicPublishersRequest
+ (*GetTopicPublishersResponse)(nil), // 19: messaging_pb.GetTopicPublishersResponse
+ (*GetTopicSubscribersRequest)(nil), // 20: messaging_pb.GetTopicSubscribersRequest
+ (*GetTopicSubscribersResponse)(nil), // 21: messaging_pb.GetTopicSubscribersResponse
+ (*TopicPublisher)(nil), // 22: messaging_pb.TopicPublisher
+ (*TopicSubscriber)(nil), // 23: messaging_pb.TopicSubscriber
+ (*AssignTopicPartitionsRequest)(nil), // 24: messaging_pb.AssignTopicPartitionsRequest
+ (*AssignTopicPartitionsResponse)(nil), // 25: messaging_pb.AssignTopicPartitionsResponse
+ (*SubscriberToSubCoordinatorRequest)(nil), // 26: messaging_pb.SubscriberToSubCoordinatorRequest
+ (*SubscriberToSubCoordinatorResponse)(nil), // 27: messaging_pb.SubscriberToSubCoordinatorResponse
+ (*ControlMessage)(nil), // 28: messaging_pb.ControlMessage
+ (*DataMessage)(nil), // 29: messaging_pb.DataMessage
+ (*PublishMessageRequest)(nil), // 30: messaging_pb.PublishMessageRequest
+ (*PublishMessageResponse)(nil), // 31: messaging_pb.PublishMessageResponse
+ (*PublishFollowMeRequest)(nil), // 32: messaging_pb.PublishFollowMeRequest
+ (*PublishFollowMeResponse)(nil), // 33: messaging_pb.PublishFollowMeResponse
+ (*SubscribeMessageRequest)(nil), // 34: messaging_pb.SubscribeMessageRequest
+ (*SubscribeMessageResponse)(nil), // 35: messaging_pb.SubscribeMessageResponse
+ (*SubscribeFollowMeRequest)(nil), // 36: messaging_pb.SubscribeFollowMeRequest
+ (*SubscribeFollowMeResponse)(nil), // 37: messaging_pb.SubscribeFollowMeResponse
+ (*ClosePublishersRequest)(nil), // 38: messaging_pb.ClosePublishersRequest
+ (*ClosePublishersResponse)(nil), // 39: messaging_pb.ClosePublishersResponse
+ (*CloseSubscribersRequest)(nil), // 40: messaging_pb.CloseSubscribersRequest
+ (*CloseSubscribersResponse)(nil), // 41: messaging_pb.CloseSubscribersResponse
+ nil, // 42: messaging_pb.BrokerStats.StatsEntry
+ (*PublisherToPubBalancerRequest_InitMessage)(nil), // 43: messaging_pb.PublisherToPubBalancerRequest.InitMessage
+ (*SubscriberToSubCoordinatorRequest_InitMessage)(nil), // 44: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage
+ (*SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage)(nil), // 45: messaging_pb.SubscriberToSubCoordinatorRequest.AckUnAssignmentMessage
+ (*SubscriberToSubCoordinatorRequest_AckAssignmentMessage)(nil), // 46: messaging_pb.SubscriberToSubCoordinatorRequest.AckAssignmentMessage
+ (*SubscriberToSubCoordinatorResponse_Assignment)(nil), // 47: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment
+ (*SubscriberToSubCoordinatorResponse_UnAssignment)(nil), // 48: messaging_pb.SubscriberToSubCoordinatorResponse.UnAssignment
+ (*PublishMessageRequest_InitMessage)(nil), // 49: messaging_pb.PublishMessageRequest.InitMessage
+ (*PublishFollowMeRequest_InitMessage)(nil), // 50: messaging_pb.PublishFollowMeRequest.InitMessage
+ (*PublishFollowMeRequest_FlushMessage)(nil), // 51: messaging_pb.PublishFollowMeRequest.FlushMessage
+ (*PublishFollowMeRequest_CloseMessage)(nil), // 52: messaging_pb.PublishFollowMeRequest.CloseMessage
+ (*SubscribeMessageRequest_InitMessage)(nil), // 53: messaging_pb.SubscribeMessageRequest.InitMessage
+ (*SubscribeMessageRequest_AckMessage)(nil), // 54: messaging_pb.SubscribeMessageRequest.AckMessage
+ (*SubscribeMessageResponse_SubscribeCtrlMessage)(nil), // 55: messaging_pb.SubscribeMessageResponse.SubscribeCtrlMessage
+ (*SubscribeFollowMeRequest_InitMessage)(nil), // 56: messaging_pb.SubscribeFollowMeRequest.InitMessage
+ (*SubscribeFollowMeRequest_AckMessage)(nil), // 57: messaging_pb.SubscribeFollowMeRequest.AckMessage
+ (*SubscribeFollowMeRequest_CloseMessage)(nil), // 58: messaging_pb.SubscribeFollowMeRequest.CloseMessage
+ (*schema_pb.Topic)(nil), // 59: schema_pb.Topic
+ (*schema_pb.Partition)(nil), // 60: schema_pb.Partition
+ (*schema_pb.RecordType)(nil), // 61: schema_pb.RecordType
+ (*schema_pb.PartitionOffset)(nil), // 62: schema_pb.PartitionOffset
+ (schema_pb.OffsetType)(0), // 63: schema_pb.OffsetType
}
var file_mq_broker_proto_depIdxs = []int32{
- 33, // 0: messaging_pb.BrokerStats.stats:type_name -> messaging_pb.BrokerStats.StatsEntry
- 50, // 1: messaging_pb.TopicPartitionStats.topic:type_name -> schema_pb.Topic
- 51, // 2: messaging_pb.TopicPartitionStats.partition:type_name -> schema_pb.Partition
- 34, // 3: messaging_pb.PublisherToPubBalancerRequest.init:type_name -> messaging_pb.PublisherToPubBalancerRequest.InitMessage
+ 42, // 0: messaging_pb.BrokerStats.stats:type_name -> messaging_pb.BrokerStats.StatsEntry
+ 59, // 1: messaging_pb.TopicPartitionStats.topic:type_name -> schema_pb.Topic
+ 60, // 2: messaging_pb.TopicPartitionStats.partition:type_name -> schema_pb.Partition
+ 43, // 3: messaging_pb.PublisherToPubBalancerRequest.init:type_name -> messaging_pb.PublisherToPubBalancerRequest.InitMessage
2, // 4: messaging_pb.PublisherToPubBalancerRequest.stats:type_name -> messaging_pb.BrokerStats
- 50, // 5: messaging_pb.ConfigureTopicRequest.topic:type_name -> schema_pb.Topic
- 52, // 6: messaging_pb.ConfigureTopicRequest.record_type:type_name -> schema_pb.RecordType
- 14, // 7: messaging_pb.ConfigureTopicResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment
- 52, // 8: messaging_pb.ConfigureTopicResponse.record_type:type_name -> schema_pb.RecordType
- 50, // 9: messaging_pb.ListTopicsResponse.topics:type_name -> schema_pb.Topic
- 50, // 10: messaging_pb.LookupTopicBrokersRequest.topic:type_name -> schema_pb.Topic
- 50, // 11: messaging_pb.LookupTopicBrokersResponse.topic:type_name -> schema_pb.Topic
- 14, // 12: messaging_pb.LookupTopicBrokersResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment
- 51, // 13: messaging_pb.BrokerPartitionAssignment.partition:type_name -> schema_pb.Partition
- 50, // 14: messaging_pb.AssignTopicPartitionsRequest.topic:type_name -> schema_pb.Topic
- 14, // 15: messaging_pb.AssignTopicPartitionsRequest.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment
- 35, // 16: messaging_pb.SubscriberToSubCoordinatorRequest.init:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage
- 37, // 17: messaging_pb.SubscriberToSubCoordinatorRequest.ack_assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.AckAssignmentMessage
- 36, // 18: messaging_pb.SubscriberToSubCoordinatorRequest.ack_un_assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.AckUnAssignmentMessage
- 38, // 19: messaging_pb.SubscriberToSubCoordinatorResponse.assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.Assignment
- 39, // 20: messaging_pb.SubscriberToSubCoordinatorResponse.un_assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.UnAssignment
- 19, // 21: messaging_pb.DataMessage.ctrl:type_name -> messaging_pb.ControlMessage
- 40, // 22: messaging_pb.PublishMessageRequest.init:type_name -> messaging_pb.PublishMessageRequest.InitMessage
- 20, // 23: messaging_pb.PublishMessageRequest.data:type_name -> messaging_pb.DataMessage
- 41, // 24: messaging_pb.PublishFollowMeRequest.init:type_name -> messaging_pb.PublishFollowMeRequest.InitMessage
- 20, // 25: messaging_pb.PublishFollowMeRequest.data:type_name -> messaging_pb.DataMessage
- 42, // 26: messaging_pb.PublishFollowMeRequest.flush:type_name -> messaging_pb.PublishFollowMeRequest.FlushMessage
- 43, // 27: messaging_pb.PublishFollowMeRequest.close:type_name -> messaging_pb.PublishFollowMeRequest.CloseMessage
- 44, // 28: messaging_pb.SubscribeMessageRequest.init:type_name -> messaging_pb.SubscribeMessageRequest.InitMessage
- 45, // 29: messaging_pb.SubscribeMessageRequest.ack:type_name -> messaging_pb.SubscribeMessageRequest.AckMessage
- 46, // 30: messaging_pb.SubscribeMessageResponse.ctrl:type_name -> messaging_pb.SubscribeMessageResponse.SubscribeCtrlMessage
- 20, // 31: messaging_pb.SubscribeMessageResponse.data:type_name -> messaging_pb.DataMessage
- 47, // 32: messaging_pb.SubscribeFollowMeRequest.init:type_name -> messaging_pb.SubscribeFollowMeRequest.InitMessage
- 48, // 33: messaging_pb.SubscribeFollowMeRequest.ack:type_name -> messaging_pb.SubscribeFollowMeRequest.AckMessage
- 49, // 34: messaging_pb.SubscribeFollowMeRequest.close:type_name -> messaging_pb.SubscribeFollowMeRequest.CloseMessage
- 50, // 35: messaging_pb.ClosePublishersRequest.topic:type_name -> schema_pb.Topic
- 50, // 36: messaging_pb.CloseSubscribersRequest.topic:type_name -> schema_pb.Topic
- 3, // 37: messaging_pb.BrokerStats.StatsEntry.value:type_name -> messaging_pb.TopicPartitionStats
- 50, // 38: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage.topic:type_name -> schema_pb.Topic
- 51, // 39: messaging_pb.SubscriberToSubCoordinatorRequest.AckUnAssignmentMessage.partition:type_name -> schema_pb.Partition
- 51, // 40: messaging_pb.SubscriberToSubCoordinatorRequest.AckAssignmentMessage.partition:type_name -> schema_pb.Partition
- 14, // 41: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment.partition_assignment:type_name -> messaging_pb.BrokerPartitionAssignment
- 51, // 42: messaging_pb.SubscriberToSubCoordinatorResponse.UnAssignment.partition:type_name -> schema_pb.Partition
- 50, // 43: messaging_pb.PublishMessageRequest.InitMessage.topic:type_name -> schema_pb.Topic
- 51, // 44: messaging_pb.PublishMessageRequest.InitMessage.partition:type_name -> schema_pb.Partition
- 50, // 45: messaging_pb.PublishFollowMeRequest.InitMessage.topic:type_name -> schema_pb.Topic
- 51, // 46: messaging_pb.PublishFollowMeRequest.InitMessage.partition:type_name -> schema_pb.Partition
- 50, // 47: messaging_pb.SubscribeMessageRequest.InitMessage.topic:type_name -> schema_pb.Topic
- 53, // 48: messaging_pb.SubscribeMessageRequest.InitMessage.partition_offset:type_name -> schema_pb.PartitionOffset
- 54, // 49: messaging_pb.SubscribeMessageRequest.InitMessage.offset_type:type_name -> schema_pb.OffsetType
- 50, // 50: messaging_pb.SubscribeFollowMeRequest.InitMessage.topic:type_name -> schema_pb.Topic
- 51, // 51: messaging_pb.SubscribeFollowMeRequest.InitMessage.partition:type_name -> schema_pb.Partition
- 0, // 52: messaging_pb.SeaweedMessaging.FindBrokerLeader:input_type -> messaging_pb.FindBrokerLeaderRequest
- 4, // 53: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:input_type -> messaging_pb.PublisherToPubBalancerRequest
- 6, // 54: messaging_pb.SeaweedMessaging.BalanceTopics:input_type -> messaging_pb.BalanceTopicsRequest
- 10, // 55: messaging_pb.SeaweedMessaging.ListTopics:input_type -> messaging_pb.ListTopicsRequest
- 8, // 56: messaging_pb.SeaweedMessaging.ConfigureTopic:input_type -> messaging_pb.ConfigureTopicRequest
- 12, // 57: messaging_pb.SeaweedMessaging.LookupTopicBrokers:input_type -> messaging_pb.LookupTopicBrokersRequest
- 15, // 58: messaging_pb.SeaweedMessaging.AssignTopicPartitions:input_type -> messaging_pb.AssignTopicPartitionsRequest
- 29, // 59: messaging_pb.SeaweedMessaging.ClosePublishers:input_type -> messaging_pb.ClosePublishersRequest
- 31, // 60: messaging_pb.SeaweedMessaging.CloseSubscribers:input_type -> messaging_pb.CloseSubscribersRequest
- 17, // 61: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:input_type -> messaging_pb.SubscriberToSubCoordinatorRequest
- 21, // 62: messaging_pb.SeaweedMessaging.PublishMessage:input_type -> messaging_pb.PublishMessageRequest
- 25, // 63: messaging_pb.SeaweedMessaging.SubscribeMessage:input_type -> messaging_pb.SubscribeMessageRequest
- 23, // 64: messaging_pb.SeaweedMessaging.PublishFollowMe:input_type -> messaging_pb.PublishFollowMeRequest
- 27, // 65: messaging_pb.SeaweedMessaging.SubscribeFollowMe:input_type -> messaging_pb.SubscribeFollowMeRequest
- 1, // 66: messaging_pb.SeaweedMessaging.FindBrokerLeader:output_type -> messaging_pb.FindBrokerLeaderResponse
- 5, // 67: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:output_type -> messaging_pb.PublisherToPubBalancerResponse
- 7, // 68: messaging_pb.SeaweedMessaging.BalanceTopics:output_type -> messaging_pb.BalanceTopicsResponse
- 11, // 69: messaging_pb.SeaweedMessaging.ListTopics:output_type -> messaging_pb.ListTopicsResponse
- 9, // 70: messaging_pb.SeaweedMessaging.ConfigureTopic:output_type -> messaging_pb.ConfigureTopicResponse
- 13, // 71: messaging_pb.SeaweedMessaging.LookupTopicBrokers:output_type -> messaging_pb.LookupTopicBrokersResponse
- 16, // 72: messaging_pb.SeaweedMessaging.AssignTopicPartitions:output_type -> messaging_pb.AssignTopicPartitionsResponse
- 30, // 73: messaging_pb.SeaweedMessaging.ClosePublishers:output_type -> messaging_pb.ClosePublishersResponse
- 32, // 74: messaging_pb.SeaweedMessaging.CloseSubscribers:output_type -> messaging_pb.CloseSubscribersResponse
- 18, // 75: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:output_type -> messaging_pb.SubscriberToSubCoordinatorResponse
- 22, // 76: messaging_pb.SeaweedMessaging.PublishMessage:output_type -> messaging_pb.PublishMessageResponse
- 26, // 77: messaging_pb.SeaweedMessaging.SubscribeMessage:output_type -> messaging_pb.SubscribeMessageResponse
- 24, // 78: messaging_pb.SeaweedMessaging.PublishFollowMe:output_type -> messaging_pb.PublishFollowMeResponse
- 28, // 79: messaging_pb.SeaweedMessaging.SubscribeFollowMe:output_type -> messaging_pb.SubscribeFollowMeResponse
- 66, // [66:80] is the sub-list for method output_type
- 52, // [52:66] is the sub-list for method input_type
- 52, // [52:52] is the sub-list for extension type_name
- 52, // [52:52] is the sub-list for extension extendee
- 0, // [0:52] is the sub-list for field type_name
+ 59, // 5: messaging_pb.ConfigureTopicRequest.topic:type_name -> schema_pb.Topic
+ 61, // 6: messaging_pb.ConfigureTopicRequest.record_type:type_name -> schema_pb.RecordType
+ 8, // 7: messaging_pb.ConfigureTopicRequest.retention:type_name -> messaging_pb.TopicRetention
+ 15, // 8: messaging_pb.ConfigureTopicResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment
+ 61, // 9: messaging_pb.ConfigureTopicResponse.record_type:type_name -> schema_pb.RecordType
+ 8, // 10: messaging_pb.ConfigureTopicResponse.retention:type_name -> messaging_pb.TopicRetention
+ 59, // 11: messaging_pb.ListTopicsResponse.topics:type_name -> schema_pb.Topic
+ 59, // 12: messaging_pb.LookupTopicBrokersRequest.topic:type_name -> schema_pb.Topic
+ 59, // 13: messaging_pb.LookupTopicBrokersResponse.topic:type_name -> schema_pb.Topic
+ 15, // 14: messaging_pb.LookupTopicBrokersResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment
+ 60, // 15: messaging_pb.BrokerPartitionAssignment.partition:type_name -> schema_pb.Partition
+ 59, // 16: messaging_pb.GetTopicConfigurationRequest.topic:type_name -> schema_pb.Topic
+ 59, // 17: messaging_pb.GetTopicConfigurationResponse.topic:type_name -> schema_pb.Topic
+ 61, // 18: messaging_pb.GetTopicConfigurationResponse.record_type:type_name -> schema_pb.RecordType
+ 15, // 19: messaging_pb.GetTopicConfigurationResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment
+ 8, // 20: messaging_pb.GetTopicConfigurationResponse.retention:type_name -> messaging_pb.TopicRetention
+ 59, // 21: messaging_pb.GetTopicPublishersRequest.topic:type_name -> schema_pb.Topic
+ 22, // 22: messaging_pb.GetTopicPublishersResponse.publishers:type_name -> messaging_pb.TopicPublisher
+ 59, // 23: messaging_pb.GetTopicSubscribersRequest.topic:type_name -> schema_pb.Topic
+ 23, // 24: messaging_pb.GetTopicSubscribersResponse.subscribers:type_name -> messaging_pb.TopicSubscriber
+ 60, // 25: messaging_pb.TopicPublisher.partition:type_name -> schema_pb.Partition
+ 60, // 26: messaging_pb.TopicSubscriber.partition:type_name -> schema_pb.Partition
+ 59, // 27: messaging_pb.AssignTopicPartitionsRequest.topic:type_name -> schema_pb.Topic
+ 15, // 28: messaging_pb.AssignTopicPartitionsRequest.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment
+ 44, // 29: messaging_pb.SubscriberToSubCoordinatorRequest.init:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage
+ 46, // 30: messaging_pb.SubscriberToSubCoordinatorRequest.ack_assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.AckAssignmentMessage
+ 45, // 31: messaging_pb.SubscriberToSubCoordinatorRequest.ack_un_assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.AckUnAssignmentMessage
+ 47, // 32: messaging_pb.SubscriberToSubCoordinatorResponse.assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.Assignment
+ 48, // 33: messaging_pb.SubscriberToSubCoordinatorResponse.un_assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.UnAssignment
+ 28, // 34: messaging_pb.DataMessage.ctrl:type_name -> messaging_pb.ControlMessage
+ 49, // 35: messaging_pb.PublishMessageRequest.init:type_name -> messaging_pb.PublishMessageRequest.InitMessage
+ 29, // 36: messaging_pb.PublishMessageRequest.data:type_name -> messaging_pb.DataMessage
+ 50, // 37: messaging_pb.PublishFollowMeRequest.init:type_name -> messaging_pb.PublishFollowMeRequest.InitMessage
+ 29, // 38: messaging_pb.PublishFollowMeRequest.data:type_name -> messaging_pb.DataMessage
+ 51, // 39: messaging_pb.PublishFollowMeRequest.flush:type_name -> messaging_pb.PublishFollowMeRequest.FlushMessage
+ 52, // 40: messaging_pb.PublishFollowMeRequest.close:type_name -> messaging_pb.PublishFollowMeRequest.CloseMessage
+ 53, // 41: messaging_pb.SubscribeMessageRequest.init:type_name -> messaging_pb.SubscribeMessageRequest.InitMessage
+ 54, // 42: messaging_pb.SubscribeMessageRequest.ack:type_name -> messaging_pb.SubscribeMessageRequest.AckMessage
+ 55, // 43: messaging_pb.SubscribeMessageResponse.ctrl:type_name -> messaging_pb.SubscribeMessageResponse.SubscribeCtrlMessage
+ 29, // 44: messaging_pb.SubscribeMessageResponse.data:type_name -> messaging_pb.DataMessage
+ 56, // 45: messaging_pb.SubscribeFollowMeRequest.init:type_name -> messaging_pb.SubscribeFollowMeRequest.InitMessage
+ 57, // 46: messaging_pb.SubscribeFollowMeRequest.ack:type_name -> messaging_pb.SubscribeFollowMeRequest.AckMessage
+ 58, // 47: messaging_pb.SubscribeFollowMeRequest.close:type_name -> messaging_pb.SubscribeFollowMeRequest.CloseMessage
+ 59, // 48: messaging_pb.ClosePublishersRequest.topic:type_name -> schema_pb.Topic
+ 59, // 49: messaging_pb.CloseSubscribersRequest.topic:type_name -> schema_pb.Topic
+ 3, // 50: messaging_pb.BrokerStats.StatsEntry.value:type_name -> messaging_pb.TopicPartitionStats
+ 59, // 51: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage.topic:type_name -> schema_pb.Topic
+ 60, // 52: messaging_pb.SubscriberToSubCoordinatorRequest.AckUnAssignmentMessage.partition:type_name -> schema_pb.Partition
+ 60, // 53: messaging_pb.SubscriberToSubCoordinatorRequest.AckAssignmentMessage.partition:type_name -> schema_pb.Partition
+ 15, // 54: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment.partition_assignment:type_name -> messaging_pb.BrokerPartitionAssignment
+ 60, // 55: messaging_pb.SubscriberToSubCoordinatorResponse.UnAssignment.partition:type_name -> schema_pb.Partition
+ 59, // 56: messaging_pb.PublishMessageRequest.InitMessage.topic:type_name -> schema_pb.Topic
+ 60, // 57: messaging_pb.PublishMessageRequest.InitMessage.partition:type_name -> schema_pb.Partition
+ 59, // 58: messaging_pb.PublishFollowMeRequest.InitMessage.topic:type_name -> schema_pb.Topic
+ 60, // 59: messaging_pb.PublishFollowMeRequest.InitMessage.partition:type_name -> schema_pb.Partition
+ 59, // 60: messaging_pb.SubscribeMessageRequest.InitMessage.topic:type_name -> schema_pb.Topic
+ 62, // 61: messaging_pb.SubscribeMessageRequest.InitMessage.partition_offset:type_name -> schema_pb.PartitionOffset
+ 63, // 62: messaging_pb.SubscribeMessageRequest.InitMessage.offset_type:type_name -> schema_pb.OffsetType
+ 59, // 63: messaging_pb.SubscribeFollowMeRequest.InitMessage.topic:type_name -> schema_pb.Topic
+ 60, // 64: messaging_pb.SubscribeFollowMeRequest.InitMessage.partition:type_name -> schema_pb.Partition
+ 0, // 65: messaging_pb.SeaweedMessaging.FindBrokerLeader:input_type -> messaging_pb.FindBrokerLeaderRequest
+ 4, // 66: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:input_type -> messaging_pb.PublisherToPubBalancerRequest
+ 6, // 67: messaging_pb.SeaweedMessaging.BalanceTopics:input_type -> messaging_pb.BalanceTopicsRequest
+ 11, // 68: messaging_pb.SeaweedMessaging.ListTopics:input_type -> messaging_pb.ListTopicsRequest
+ 9, // 69: messaging_pb.SeaweedMessaging.ConfigureTopic:input_type -> messaging_pb.ConfigureTopicRequest
+ 13, // 70: messaging_pb.SeaweedMessaging.LookupTopicBrokers:input_type -> messaging_pb.LookupTopicBrokersRequest
+ 16, // 71: messaging_pb.SeaweedMessaging.GetTopicConfiguration:input_type -> messaging_pb.GetTopicConfigurationRequest
+ 18, // 72: messaging_pb.SeaweedMessaging.GetTopicPublishers:input_type -> messaging_pb.GetTopicPublishersRequest
+ 20, // 73: messaging_pb.SeaweedMessaging.GetTopicSubscribers:input_type -> messaging_pb.GetTopicSubscribersRequest
+ 24, // 74: messaging_pb.SeaweedMessaging.AssignTopicPartitions:input_type -> messaging_pb.AssignTopicPartitionsRequest
+ 38, // 75: messaging_pb.SeaweedMessaging.ClosePublishers:input_type -> messaging_pb.ClosePublishersRequest
+ 40, // 76: messaging_pb.SeaweedMessaging.CloseSubscribers:input_type -> messaging_pb.CloseSubscribersRequest
+ 26, // 77: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:input_type -> messaging_pb.SubscriberToSubCoordinatorRequest
+ 30, // 78: messaging_pb.SeaweedMessaging.PublishMessage:input_type -> messaging_pb.PublishMessageRequest
+ 34, // 79: messaging_pb.SeaweedMessaging.SubscribeMessage:input_type -> messaging_pb.SubscribeMessageRequest
+ 32, // 80: messaging_pb.SeaweedMessaging.PublishFollowMe:input_type -> messaging_pb.PublishFollowMeRequest
+ 36, // 81: messaging_pb.SeaweedMessaging.SubscribeFollowMe:input_type -> messaging_pb.SubscribeFollowMeRequest
+ 1, // 82: messaging_pb.SeaweedMessaging.FindBrokerLeader:output_type -> messaging_pb.FindBrokerLeaderResponse
+ 5, // 83: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:output_type -> messaging_pb.PublisherToPubBalancerResponse
+ 7, // 84: messaging_pb.SeaweedMessaging.BalanceTopics:output_type -> messaging_pb.BalanceTopicsResponse
+ 12, // 85: messaging_pb.SeaweedMessaging.ListTopics:output_type -> messaging_pb.ListTopicsResponse
+ 10, // 86: messaging_pb.SeaweedMessaging.ConfigureTopic:output_type -> messaging_pb.ConfigureTopicResponse
+ 14, // 87: messaging_pb.SeaweedMessaging.LookupTopicBrokers:output_type -> messaging_pb.LookupTopicBrokersResponse
+ 17, // 88: messaging_pb.SeaweedMessaging.GetTopicConfiguration:output_type -> messaging_pb.GetTopicConfigurationResponse
+ 19, // 89: messaging_pb.SeaweedMessaging.GetTopicPublishers:output_type -> messaging_pb.GetTopicPublishersResponse
+ 21, // 90: messaging_pb.SeaweedMessaging.GetTopicSubscribers:output_type -> messaging_pb.GetTopicSubscribersResponse
+ 25, // 91: messaging_pb.SeaweedMessaging.AssignTopicPartitions:output_type -> messaging_pb.AssignTopicPartitionsResponse
+ 39, // 92: messaging_pb.SeaweedMessaging.ClosePublishers:output_type -> messaging_pb.ClosePublishersResponse
+ 41, // 93: messaging_pb.SeaweedMessaging.CloseSubscribers:output_type -> messaging_pb.CloseSubscribersResponse
+ 27, // 94: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:output_type -> messaging_pb.SubscriberToSubCoordinatorResponse
+ 31, // 95: messaging_pb.SeaweedMessaging.PublishMessage:output_type -> messaging_pb.PublishMessageResponse
+ 35, // 96: messaging_pb.SeaweedMessaging.SubscribeMessage:output_type -> messaging_pb.SubscribeMessageResponse
+ 33, // 97: messaging_pb.SeaweedMessaging.PublishFollowMe:output_type -> messaging_pb.PublishFollowMeResponse
+ 37, // 98: messaging_pb.SeaweedMessaging.SubscribeFollowMe:output_type -> messaging_pb.SubscribeFollowMeResponse
+ 82, // [82:99] is the sub-list for method output_type
+ 65, // [65:82] is the sub-list for method input_type
+ 65, // [65:65] is the sub-list for extension type_name
+ 65, // [65:65] is the sub-list for extension extendee
+ 0, // [0:65] is the sub-list for field type_name
}
func init() { file_mq_broker_proto_init() }
@@ -3202,34 +3886,34 @@ func file_mq_broker_proto_init() {
(*PublisherToPubBalancerRequest_Init)(nil),
(*PublisherToPubBalancerRequest_Stats)(nil),
}
- file_mq_broker_proto_msgTypes[17].OneofWrappers = []any{
+ file_mq_broker_proto_msgTypes[26].OneofWrappers = []any{
(*SubscriberToSubCoordinatorRequest_Init)(nil),
(*SubscriberToSubCoordinatorRequest_AckAssignment)(nil),
(*SubscriberToSubCoordinatorRequest_AckUnAssignment)(nil),
}
- file_mq_broker_proto_msgTypes[18].OneofWrappers = []any{
+ file_mq_broker_proto_msgTypes[27].OneofWrappers = []any{
(*SubscriberToSubCoordinatorResponse_Assignment_)(nil),
(*SubscriberToSubCoordinatorResponse_UnAssignment_)(nil),
}
- file_mq_broker_proto_msgTypes[21].OneofWrappers = []any{
+ file_mq_broker_proto_msgTypes[30].OneofWrappers = []any{
(*PublishMessageRequest_Init)(nil),
(*PublishMessageRequest_Data)(nil),
}
- file_mq_broker_proto_msgTypes[23].OneofWrappers = []any{
+ file_mq_broker_proto_msgTypes[32].OneofWrappers = []any{
(*PublishFollowMeRequest_Init)(nil),
(*PublishFollowMeRequest_Data)(nil),
(*PublishFollowMeRequest_Flush)(nil),
(*PublishFollowMeRequest_Close)(nil),
}
- file_mq_broker_proto_msgTypes[25].OneofWrappers = []any{
+ file_mq_broker_proto_msgTypes[34].OneofWrappers = []any{
(*SubscribeMessageRequest_Init)(nil),
(*SubscribeMessageRequest_Ack)(nil),
}
- file_mq_broker_proto_msgTypes[26].OneofWrappers = []any{
+ file_mq_broker_proto_msgTypes[35].OneofWrappers = []any{
(*SubscribeMessageResponse_Ctrl)(nil),
(*SubscribeMessageResponse_Data)(nil),
}
- file_mq_broker_proto_msgTypes[27].OneofWrappers = []any{
+ file_mq_broker_proto_msgTypes[36].OneofWrappers = []any{
(*SubscribeFollowMeRequest_Init)(nil),
(*SubscribeFollowMeRequest_Ack)(nil),
(*SubscribeFollowMeRequest_Close)(nil),
@@ -3240,7 +3924,7 @@ func file_mq_broker_proto_init() {
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: unsafe.Slice(unsafe.StringData(file_mq_broker_proto_rawDesc), len(file_mq_broker_proto_rawDesc)),
NumEnums: 0,
- NumMessages: 50,
+ NumMessages: 59,
NumExtensions: 0,
NumServices: 1,
},
diff --git a/weed/pb/mq_pb/mq_broker_grpc.pb.go b/weed/pb/mq_pb/mq_broker_grpc.pb.go
index d577c6c05..5241861bc 100644
--- a/weed/pb/mq_pb/mq_broker_grpc.pb.go
+++ b/weed/pb/mq_pb/mq_broker_grpc.pb.go
@@ -25,6 +25,9 @@ const (
SeaweedMessaging_ListTopics_FullMethodName = "/messaging_pb.SeaweedMessaging/ListTopics"
SeaweedMessaging_ConfigureTopic_FullMethodName = "/messaging_pb.SeaweedMessaging/ConfigureTopic"
SeaweedMessaging_LookupTopicBrokers_FullMethodName = "/messaging_pb.SeaweedMessaging/LookupTopicBrokers"
+ SeaweedMessaging_GetTopicConfiguration_FullMethodName = "/messaging_pb.SeaweedMessaging/GetTopicConfiguration"
+ SeaweedMessaging_GetTopicPublishers_FullMethodName = "/messaging_pb.SeaweedMessaging/GetTopicPublishers"
+ SeaweedMessaging_GetTopicSubscribers_FullMethodName = "/messaging_pb.SeaweedMessaging/GetTopicSubscribers"
SeaweedMessaging_AssignTopicPartitions_FullMethodName = "/messaging_pb.SeaweedMessaging/AssignTopicPartitions"
SeaweedMessaging_ClosePublishers_FullMethodName = "/messaging_pb.SeaweedMessaging/ClosePublishers"
SeaweedMessaging_CloseSubscribers_FullMethodName = "/messaging_pb.SeaweedMessaging/CloseSubscribers"
@@ -48,6 +51,9 @@ type SeaweedMessagingClient interface {
ListTopics(ctx context.Context, in *ListTopicsRequest, opts ...grpc.CallOption) (*ListTopicsResponse, error)
ConfigureTopic(ctx context.Context, in *ConfigureTopicRequest, opts ...grpc.CallOption) (*ConfigureTopicResponse, error)
LookupTopicBrokers(ctx context.Context, in *LookupTopicBrokersRequest, opts ...grpc.CallOption) (*LookupTopicBrokersResponse, error)
+ GetTopicConfiguration(ctx context.Context, in *GetTopicConfigurationRequest, opts ...grpc.CallOption) (*GetTopicConfigurationResponse, error)
+ GetTopicPublishers(ctx context.Context, in *GetTopicPublishersRequest, opts ...grpc.CallOption) (*GetTopicPublishersResponse, error)
+ GetTopicSubscribers(ctx context.Context, in *GetTopicSubscribersRequest, opts ...grpc.CallOption) (*GetTopicSubscribersResponse, error)
// invoked by the balancer, running on each broker
AssignTopicPartitions(ctx context.Context, in *AssignTopicPartitionsRequest, opts ...grpc.CallOption) (*AssignTopicPartitionsResponse, error)
ClosePublishers(ctx context.Context, in *ClosePublishersRequest, opts ...grpc.CallOption) (*ClosePublishersResponse, error)
@@ -133,6 +139,36 @@ func (c *seaweedMessagingClient) LookupTopicBrokers(ctx context.Context, in *Loo
return out, nil
}
+func (c *seaweedMessagingClient) GetTopicConfiguration(ctx context.Context, in *GetTopicConfigurationRequest, opts ...grpc.CallOption) (*GetTopicConfigurationResponse, error) {
+ cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
+ out := new(GetTopicConfigurationResponse)
+ err := c.cc.Invoke(ctx, SeaweedMessaging_GetTopicConfiguration_FullMethodName, in, out, cOpts...)
+ if err != nil {
+ return nil, err
+ }
+ return out, nil
+}
+
+func (c *seaweedMessagingClient) GetTopicPublishers(ctx context.Context, in *GetTopicPublishersRequest, opts ...grpc.CallOption) (*GetTopicPublishersResponse, error) {
+ cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
+ out := new(GetTopicPublishersResponse)
+ err := c.cc.Invoke(ctx, SeaweedMessaging_GetTopicPublishers_FullMethodName, in, out, cOpts...)
+ if err != nil {
+ return nil, err
+ }
+ return out, nil
+}
+
+func (c *seaweedMessagingClient) GetTopicSubscribers(ctx context.Context, in *GetTopicSubscribersRequest, opts ...grpc.CallOption) (*GetTopicSubscribersResponse, error) {
+ cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
+ out := new(GetTopicSubscribersResponse)
+ err := c.cc.Invoke(ctx, SeaweedMessaging_GetTopicSubscribers_FullMethodName, in, out, cOpts...)
+ if err != nil {
+ return nil, err
+ }
+ return out, nil
+}
+
func (c *seaweedMessagingClient) AssignTopicPartitions(ctx context.Context, in *AssignTopicPartitionsRequest, opts ...grpc.CallOption) (*AssignTopicPartitionsResponse, error) {
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...)
out := new(AssignTopicPartitionsResponse)
@@ -241,6 +277,9 @@ type SeaweedMessagingServer interface {
ListTopics(context.Context, *ListTopicsRequest) (*ListTopicsResponse, error)
ConfigureTopic(context.Context, *ConfigureTopicRequest) (*ConfigureTopicResponse, error)
LookupTopicBrokers(context.Context, *LookupTopicBrokersRequest) (*LookupTopicBrokersResponse, error)
+ GetTopicConfiguration(context.Context, *GetTopicConfigurationRequest) (*GetTopicConfigurationResponse, error)
+ GetTopicPublishers(context.Context, *GetTopicPublishersRequest) (*GetTopicPublishersResponse, error)
+ GetTopicSubscribers(context.Context, *GetTopicSubscribersRequest) (*GetTopicSubscribersResponse, error)
// invoked by the balancer, running on each broker
AssignTopicPartitions(context.Context, *AssignTopicPartitionsRequest) (*AssignTopicPartitionsResponse, error)
ClosePublishers(context.Context, *ClosePublishersRequest) (*ClosePublishersResponse, error)
@@ -281,6 +320,15 @@ func (UnimplementedSeaweedMessagingServer) ConfigureTopic(context.Context, *Conf
func (UnimplementedSeaweedMessagingServer) LookupTopicBrokers(context.Context, *LookupTopicBrokersRequest) (*LookupTopicBrokersResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method LookupTopicBrokers not implemented")
}
+func (UnimplementedSeaweedMessagingServer) GetTopicConfiguration(context.Context, *GetTopicConfigurationRequest) (*GetTopicConfigurationResponse, error) {
+ return nil, status.Errorf(codes.Unimplemented, "method GetTopicConfiguration not implemented")
+}
+func (UnimplementedSeaweedMessagingServer) GetTopicPublishers(context.Context, *GetTopicPublishersRequest) (*GetTopicPublishersResponse, error) {
+ return nil, status.Errorf(codes.Unimplemented, "method GetTopicPublishers not implemented")
+}
+func (UnimplementedSeaweedMessagingServer) GetTopicSubscribers(context.Context, *GetTopicSubscribersRequest) (*GetTopicSubscribersResponse, error) {
+ return nil, status.Errorf(codes.Unimplemented, "method GetTopicSubscribers not implemented")
+}
func (UnimplementedSeaweedMessagingServer) AssignTopicPartitions(context.Context, *AssignTopicPartitionsRequest) (*AssignTopicPartitionsResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method AssignTopicPartitions not implemented")
}
@@ -423,6 +471,60 @@ func _SeaweedMessaging_LookupTopicBrokers_Handler(srv interface{}, ctx context.C
return interceptor(ctx, in, info, handler)
}
+func _SeaweedMessaging_GetTopicConfiguration_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+ in := new(GetTopicConfigurationRequest)
+ if err := dec(in); err != nil {
+ return nil, err
+ }
+ if interceptor == nil {
+ return srv.(SeaweedMessagingServer).GetTopicConfiguration(ctx, in)
+ }
+ info := &grpc.UnaryServerInfo{
+ Server: srv,
+ FullMethod: SeaweedMessaging_GetTopicConfiguration_FullMethodName,
+ }
+ handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+ return srv.(SeaweedMessagingServer).GetTopicConfiguration(ctx, req.(*GetTopicConfigurationRequest))
+ }
+ return interceptor(ctx, in, info, handler)
+}
+
+func _SeaweedMessaging_GetTopicPublishers_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+ in := new(GetTopicPublishersRequest)
+ if err := dec(in); err != nil {
+ return nil, err
+ }
+ if interceptor == nil {
+ return srv.(SeaweedMessagingServer).GetTopicPublishers(ctx, in)
+ }
+ info := &grpc.UnaryServerInfo{
+ Server: srv,
+ FullMethod: SeaweedMessaging_GetTopicPublishers_FullMethodName,
+ }
+ handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+ return srv.(SeaweedMessagingServer).GetTopicPublishers(ctx, req.(*GetTopicPublishersRequest))
+ }
+ return interceptor(ctx, in, info, handler)
+}
+
+func _SeaweedMessaging_GetTopicSubscribers_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+ in := new(GetTopicSubscribersRequest)
+ if err := dec(in); err != nil {
+ return nil, err
+ }
+ if interceptor == nil {
+ return srv.(SeaweedMessagingServer).GetTopicSubscribers(ctx, in)
+ }
+ info := &grpc.UnaryServerInfo{
+ Server: srv,
+ FullMethod: SeaweedMessaging_GetTopicSubscribers_FullMethodName,
+ }
+ handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+ return srv.(SeaweedMessagingServer).GetTopicSubscribers(ctx, req.(*GetTopicSubscribersRequest))
+ }
+ return interceptor(ctx, in, info, handler)
+}
+
func _SeaweedMessaging_AssignTopicPartitions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(AssignTopicPartitionsRequest)
if err := dec(in); err != nil {
@@ -540,6 +642,18 @@ var SeaweedMessaging_ServiceDesc = grpc.ServiceDesc{
Handler: _SeaweedMessaging_LookupTopicBrokers_Handler,
},
{
+ MethodName: "GetTopicConfiguration",
+ Handler: _SeaweedMessaging_GetTopicConfiguration_Handler,
+ },
+ {
+ MethodName: "GetTopicPublishers",
+ Handler: _SeaweedMessaging_GetTopicPublishers_Handler,
+ },
+ {
+ MethodName: "GetTopicSubscribers",
+ Handler: _SeaweedMessaging_GetTopicSubscribers_Handler,
+ },
+ {
MethodName: "AssignTopicPartitions",
Handler: _SeaweedMessaging_AssignTopicPartitions_Handler,
},
diff --git a/weed/pb/worker_pb/worker.pb.go b/weed/pb/worker_pb/worker.pb.go
index 6f47e04f0..00188937b 100644
--- a/weed/pb/worker_pb/worker.pb.go
+++ b/weed/pb/worker_pb/worker.pb.go
@@ -1,6 +1,6 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
-// protoc-gen-go v1.34.2
+// protoc-gen-go v1.36.6
// protoc v5.29.3
// source: worker.proto
@@ -11,6 +11,7 @@ import (
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
reflect "reflect"
sync "sync"
+ unsafe "unsafe"
)
const (
@@ -22,13 +23,10 @@ const (
// WorkerMessage represents messages from worker to admin
type WorkerMessage struct {
- state protoimpl.MessageState
- sizeCache protoimpl.SizeCache
- unknownFields protoimpl.UnknownFields
-
- WorkerId string `protobuf:"bytes,1,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"`
- Timestamp int64 `protobuf:"varint,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
- // Types that are assignable to Message:
+ state protoimpl.MessageState `protogen:"open.v1"`
+ WorkerId string `protobuf:"bytes,1,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"`
+ Timestamp int64 `protobuf:"varint,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
+ // Types that are valid to be assigned to Message:
//
// *WorkerMessage_Registration
// *WorkerMessage_Heartbeat
@@ -36,16 +34,16 @@ type WorkerMessage struct {
// *WorkerMessage_TaskUpdate
// *WorkerMessage_TaskComplete
// *WorkerMessage_Shutdown
- Message isWorkerMessage_Message `protobuf_oneof:"message"`
+ Message isWorkerMessage_Message `protobuf_oneof:"message"`
+ unknownFields protoimpl.UnknownFields
+ sizeCache protoimpl.SizeCache
}
func (x *WorkerMessage) Reset() {
*x = WorkerMessage{}
- if protoimpl.UnsafeEnabled {
- mi := &file_worker_proto_msgTypes[0]
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- ms.StoreMessageInfo(mi)
- }
+ mi := &file_worker_proto_msgTypes[0]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
}
func (x *WorkerMessage) String() string {
@@ -56,7 +54,7 @@ func (*WorkerMessage) ProtoMessage() {}
func (x *WorkerMessage) ProtoReflect() protoreflect.Message {
mi := &file_worker_proto_msgTypes[0]
- if protoimpl.UnsafeEnabled && x != nil {
+ if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
@@ -85,51 +83,63 @@ func (x *WorkerMessage) GetTimestamp() int64 {
return 0
}
-func (m *WorkerMessage) GetMessage() isWorkerMessage_Message {
- if m != nil {
- return m.Message
+func (x *WorkerMessage) GetMessage() isWorkerMessage_Message {
+ if x != nil {
+ return x.Message
}
return nil
}
func (x *WorkerMessage) GetRegistration() *WorkerRegistration {
- if x, ok := x.GetMessage().(*WorkerMessage_Registration); ok {
- return x.Registration
+ if x != nil {
+ if x, ok := x.Message.(*WorkerMessage_Registration); ok {
+ return x.Registration
+ }
}
return nil
}
func (x *WorkerMessage) GetHeartbeat() *WorkerHeartbeat {
- if x, ok := x.GetMessage().(*WorkerMessage_Heartbeat); ok {
- return x.Heartbeat
+ if x != nil {
+ if x, ok := x.Message.(*WorkerMessage_Heartbeat); ok {
+ return x.Heartbeat
+ }
}
return nil
}
func (x *WorkerMessage) GetTaskRequest() *TaskRequest {
- if x, ok := x.GetMessage().(*WorkerMessage_TaskRequest); ok {
- return x.TaskRequest
+ if x != nil {
+ if x, ok := x.Message.(*WorkerMessage_TaskRequest); ok {
+ return x.TaskRequest
+ }
}
return nil
}
func (x *WorkerMessage) GetTaskUpdate() *TaskUpdate {
- if x, ok := x.GetMessage().(*WorkerMessage_TaskUpdate); ok {
- return x.TaskUpdate
+ if x != nil {
+ if x, ok := x.Message.(*WorkerMessage_TaskUpdate); ok {
+ return x.TaskUpdate
+ }
}
return nil
}
func (x *WorkerMessage) GetTaskComplete() *TaskComplete {
- if x, ok := x.GetMessage().(*WorkerMessage_TaskComplete); ok {
- return x.TaskComplete
+ if x != nil {
+ if x, ok := x.Message.(*WorkerMessage_TaskComplete); ok {
+ return x.TaskComplete
+ }
}
return nil
}
func (x *WorkerMessage) GetShutdown() *WorkerShutdown {
- if x, ok := x.GetMessage().(*WorkerMessage_Shutdown); ok {
- return x.Shutdown
+ if x != nil {
+ if x, ok := x.Message.(*WorkerMessage_Shutdown); ok {
+ return x.Shutdown
+ }
}
return nil
}
@@ -176,29 +186,26 @@ func (*WorkerMessage_Shutdown) isWorkerMessage_Message() {}
// AdminMessage represents messages from admin to worker
type AdminMessage struct {
- state protoimpl.MessageState
- sizeCache protoimpl.SizeCache
- unknownFields protoimpl.UnknownFields
-
- AdminId string `protobuf:"bytes,1,opt,name=admin_id,json=adminId,proto3" json:"admin_id,omitempty"`
- Timestamp int64 `protobuf:"varint,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
- // Types that are assignable to Message:
+ state protoimpl.MessageState `protogen:"open.v1"`
+ AdminId string `protobuf:"bytes,1,opt,name=admin_id,json=adminId,proto3" json:"admin_id,omitempty"`
+ Timestamp int64 `protobuf:"varint,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
+ // Types that are valid to be assigned to Message:
//
// *AdminMessage_RegistrationResponse
// *AdminMessage_HeartbeatResponse
// *AdminMessage_TaskAssignment
// *AdminMessage_TaskCancellation
// *AdminMessage_AdminShutdown
- Message isAdminMessage_Message `protobuf_oneof:"message"`
+ Message isAdminMessage_Message `protobuf_oneof:"message"`
+ unknownFields protoimpl.UnknownFields
+ sizeCache protoimpl.SizeCache
}
func (x *AdminMessage) Reset() {
*x = AdminMessage{}
- if protoimpl.UnsafeEnabled {
- mi := &file_worker_proto_msgTypes[1]
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- ms.StoreMessageInfo(mi)
- }
+ mi := &file_worker_proto_msgTypes[1]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
}
func (x *AdminMessage) String() string {
@@ -209,7 +216,7 @@ func (*AdminMessage) ProtoMessage() {}
func (x *AdminMessage) ProtoReflect() protoreflect.Message {
mi := &file_worker_proto_msgTypes[1]
- if protoimpl.UnsafeEnabled && x != nil {
+ if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
@@ -238,44 +245,54 @@ func (x *AdminMessage) GetTimestamp() int64 {
return 0
}
-func (m *AdminMessage) GetMessage() isAdminMessage_Message {
- if m != nil {
- return m.Message
+func (x *AdminMessage) GetMessage() isAdminMessage_Message {
+ if x != nil {
+ return x.Message
}
return nil
}
func (x *AdminMessage) GetRegistrationResponse() *RegistrationResponse {
- if x, ok := x.GetMessage().(*AdminMessage_RegistrationResponse); ok {
- return x.RegistrationResponse
+ if x != nil {
+ if x, ok := x.Message.(*AdminMessage_RegistrationResponse); ok {
+ return x.RegistrationResponse
+ }
}
return nil
}
func (x *AdminMessage) GetHeartbeatResponse() *HeartbeatResponse {
- if x, ok := x.GetMessage().(*AdminMessage_HeartbeatResponse); ok {
- return x.HeartbeatResponse
+ if x != nil {
+ if x, ok := x.Message.(*AdminMessage_HeartbeatResponse); ok {
+ return x.HeartbeatResponse
+ }
}
return nil
}
func (x *AdminMessage) GetTaskAssignment() *TaskAssignment {
- if x, ok := x.GetMessage().(*AdminMessage_TaskAssignment); ok {
- return x.TaskAssignment
+ if x != nil {
+ if x, ok := x.Message.(*AdminMessage_TaskAssignment); ok {
+ return x.TaskAssignment
+ }
}
return nil
}
func (x *AdminMessage) GetTaskCancellation() *TaskCancellation {
- if x, ok := x.GetMessage().(*AdminMessage_TaskCancellation); ok {
- return x.TaskCancellation
+ if x != nil {
+ if x, ok := x.Message.(*AdminMessage_TaskCancellation); ok {
+ return x.TaskCancellation
+ }
}
return nil
}
func (x *AdminMessage) GetAdminShutdown() *AdminShutdown {
- if x, ok := x.GetMessage().(*AdminMessage_AdminShutdown); ok {
- return x.AdminShutdown
+ if x != nil {
+ if x, ok := x.Message.(*AdminMessage_AdminShutdown); ok {
+ return x.AdminShutdown
+ }
}
return nil
}
@@ -316,24 +333,21 @@ func (*AdminMessage_AdminShutdown) isAdminMessage_Message() {}
// WorkerRegistration message when worker connects
type WorkerRegistration struct {
- state protoimpl.MessageState
- sizeCache protoimpl.SizeCache
+ state protoimpl.MessageState `protogen:"open.v1"`
+ WorkerId string `protobuf:"bytes,1,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"`
+ Address string `protobuf:"bytes,2,opt,name=address,proto3" json:"address,omitempty"`
+ Capabilities []string `protobuf:"bytes,3,rep,name=capabilities,proto3" json:"capabilities,omitempty"`
+ MaxConcurrent int32 `protobuf:"varint,4,opt,name=max_concurrent,json=maxConcurrent,proto3" json:"max_concurrent,omitempty"`
+ Metadata map[string]string `protobuf:"bytes,5,rep,name=metadata,proto3" json:"metadata,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
unknownFields protoimpl.UnknownFields
-
- WorkerId string `protobuf:"bytes,1,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"`
- Address string `protobuf:"bytes,2,opt,name=address,proto3" json:"address,omitempty"`
- Capabilities []string `protobuf:"bytes,3,rep,name=capabilities,proto3" json:"capabilities,omitempty"`
- MaxConcurrent int32 `protobuf:"varint,4,opt,name=max_concurrent,json=maxConcurrent,proto3" json:"max_concurrent,omitempty"`
- Metadata map[string]string `protobuf:"bytes,5,rep,name=metadata,proto3" json:"metadata,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
+ sizeCache protoimpl.SizeCache
}
func (x *WorkerRegistration) Reset() {
*x = WorkerRegistration{}
- if protoimpl.UnsafeEnabled {
- mi := &file_worker_proto_msgTypes[2]
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- ms.StoreMessageInfo(mi)
- }
+ mi := &file_worker_proto_msgTypes[2]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
}
func (x *WorkerRegistration) String() string {
@@ -344,7 +358,7 @@ func (*WorkerRegistration) ProtoMessage() {}
func (x *WorkerRegistration) ProtoReflect() protoreflect.Message {
mi := &file_worker_proto_msgTypes[2]
- if protoimpl.UnsafeEnabled && x != nil {
+ if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
@@ -396,22 +410,19 @@ func (x *WorkerRegistration) GetMetadata() map[string]string {
// RegistrationResponse confirms worker registration
type RegistrationResponse struct {
- state protoimpl.MessageState
- sizeCache protoimpl.SizeCache
- unknownFields protoimpl.UnknownFields
-
- Success bool `protobuf:"varint,1,opt,name=success,proto3" json:"success,omitempty"`
- Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"`
- AssignedWorkerId string `protobuf:"bytes,3,opt,name=assigned_worker_id,json=assignedWorkerId,proto3" json:"assigned_worker_id,omitempty"`
+ state protoimpl.MessageState `protogen:"open.v1"`
+ Success bool `protobuf:"varint,1,opt,name=success,proto3" json:"success,omitempty"`
+ Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"`
+ AssignedWorkerId string `protobuf:"bytes,3,opt,name=assigned_worker_id,json=assignedWorkerId,proto3" json:"assigned_worker_id,omitempty"`
+ unknownFields protoimpl.UnknownFields
+ sizeCache protoimpl.SizeCache
}
func (x *RegistrationResponse) Reset() {
*x = RegistrationResponse{}
- if protoimpl.UnsafeEnabled {
- mi := &file_worker_proto_msgTypes[3]
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- ms.StoreMessageInfo(mi)
- }
+ mi := &file_worker_proto_msgTypes[3]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
}
func (x *RegistrationResponse) String() string {
@@ -422,7 +433,7 @@ func (*RegistrationResponse) ProtoMessage() {}
func (x *RegistrationResponse) ProtoReflect() protoreflect.Message {
mi := &file_worker_proto_msgTypes[3]
- if protoimpl.UnsafeEnabled && x != nil {
+ if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
@@ -460,27 +471,24 @@ func (x *RegistrationResponse) GetAssignedWorkerId() string {
// WorkerHeartbeat sent periodically by worker
type WorkerHeartbeat struct {
- state protoimpl.MessageState
- sizeCache protoimpl.SizeCache
- unknownFields protoimpl.UnknownFields
-
- WorkerId string `protobuf:"bytes,1,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"`
- Status string `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"`
- CurrentLoad int32 `protobuf:"varint,3,opt,name=current_load,json=currentLoad,proto3" json:"current_load,omitempty"`
- MaxConcurrent int32 `protobuf:"varint,4,opt,name=max_concurrent,json=maxConcurrent,proto3" json:"max_concurrent,omitempty"`
- CurrentTaskIds []string `protobuf:"bytes,5,rep,name=current_task_ids,json=currentTaskIds,proto3" json:"current_task_ids,omitempty"`
- TasksCompleted int32 `protobuf:"varint,6,opt,name=tasks_completed,json=tasksCompleted,proto3" json:"tasks_completed,omitempty"`
- TasksFailed int32 `protobuf:"varint,7,opt,name=tasks_failed,json=tasksFailed,proto3" json:"tasks_failed,omitempty"`
- UptimeSeconds int64 `protobuf:"varint,8,opt,name=uptime_seconds,json=uptimeSeconds,proto3" json:"uptime_seconds,omitempty"`
+ state protoimpl.MessageState `protogen:"open.v1"`
+ WorkerId string `protobuf:"bytes,1,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"`
+ Status string `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"`
+ CurrentLoad int32 `protobuf:"varint,3,opt,name=current_load,json=currentLoad,proto3" json:"current_load,omitempty"`
+ MaxConcurrent int32 `protobuf:"varint,4,opt,name=max_concurrent,json=maxConcurrent,proto3" json:"max_concurrent,omitempty"`
+ CurrentTaskIds []string `protobuf:"bytes,5,rep,name=current_task_ids,json=currentTaskIds,proto3" json:"current_task_ids,omitempty"`
+ TasksCompleted int32 `protobuf:"varint,6,opt,name=tasks_completed,json=tasksCompleted,proto3" json:"tasks_completed,omitempty"`
+ TasksFailed int32 `protobuf:"varint,7,opt,name=tasks_failed,json=tasksFailed,proto3" json:"tasks_failed,omitempty"`
+ UptimeSeconds int64 `protobuf:"varint,8,opt,name=uptime_seconds,json=uptimeSeconds,proto3" json:"uptime_seconds,omitempty"`
+ unknownFields protoimpl.UnknownFields
+ sizeCache protoimpl.SizeCache
}
func (x *WorkerHeartbeat) Reset() {
*x = WorkerHeartbeat{}
- if protoimpl.UnsafeEnabled {
- mi := &file_worker_proto_msgTypes[4]
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- ms.StoreMessageInfo(mi)
- }
+ mi := &file_worker_proto_msgTypes[4]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
}
func (x *WorkerHeartbeat) String() string {
@@ -491,7 +499,7 @@ func (*WorkerHeartbeat) ProtoMessage() {}
func (x *WorkerHeartbeat) ProtoReflect() protoreflect.Message {
mi := &file_worker_proto_msgTypes[4]
- if protoimpl.UnsafeEnabled && x != nil {
+ if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
@@ -564,21 +572,18 @@ func (x *WorkerHeartbeat) GetUptimeSeconds() int64 {
// HeartbeatResponse acknowledges heartbeat
type HeartbeatResponse struct {
- state protoimpl.MessageState
- sizeCache protoimpl.SizeCache
+ state protoimpl.MessageState `protogen:"open.v1"`
+ Success bool `protobuf:"varint,1,opt,name=success,proto3" json:"success,omitempty"`
+ Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"`
unknownFields protoimpl.UnknownFields
-
- Success bool `protobuf:"varint,1,opt,name=success,proto3" json:"success,omitempty"`
- Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"`
+ sizeCache protoimpl.SizeCache
}
func (x *HeartbeatResponse) Reset() {
*x = HeartbeatResponse{}
- if protoimpl.UnsafeEnabled {
- mi := &file_worker_proto_msgTypes[5]
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- ms.StoreMessageInfo(mi)
- }
+ mi := &file_worker_proto_msgTypes[5]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
}
func (x *HeartbeatResponse) String() string {
@@ -589,7 +594,7 @@ func (*HeartbeatResponse) ProtoMessage() {}
func (x *HeartbeatResponse) ProtoReflect() protoreflect.Message {
mi := &file_worker_proto_msgTypes[5]
- if protoimpl.UnsafeEnabled && x != nil {
+ if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
@@ -620,22 +625,19 @@ func (x *HeartbeatResponse) GetMessage() string {
// TaskRequest from worker asking for new tasks
type TaskRequest struct {
- state protoimpl.MessageState
- sizeCache protoimpl.SizeCache
- unknownFields protoimpl.UnknownFields
-
- WorkerId string `protobuf:"bytes,1,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"`
- Capabilities []string `protobuf:"bytes,2,rep,name=capabilities,proto3" json:"capabilities,omitempty"`
- AvailableSlots int32 `protobuf:"varint,3,opt,name=available_slots,json=availableSlots,proto3" json:"available_slots,omitempty"`
+ state protoimpl.MessageState `protogen:"open.v1"`
+ WorkerId string `protobuf:"bytes,1,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"`
+ Capabilities []string `protobuf:"bytes,2,rep,name=capabilities,proto3" json:"capabilities,omitempty"`
+ AvailableSlots int32 `protobuf:"varint,3,opt,name=available_slots,json=availableSlots,proto3" json:"available_slots,omitempty"`
+ unknownFields protoimpl.UnknownFields
+ sizeCache protoimpl.SizeCache
}
func (x *TaskRequest) Reset() {
*x = TaskRequest{}
- if protoimpl.UnsafeEnabled {
- mi := &file_worker_proto_msgTypes[6]
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- ms.StoreMessageInfo(mi)
- }
+ mi := &file_worker_proto_msgTypes[6]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
}
func (x *TaskRequest) String() string {
@@ -646,7 +648,7 @@ func (*TaskRequest) ProtoMessage() {}
func (x *TaskRequest) ProtoReflect() protoreflect.Message {
mi := &file_worker_proto_msgTypes[6]
- if protoimpl.UnsafeEnabled && x != nil {
+ if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
@@ -684,25 +686,22 @@ func (x *TaskRequest) GetAvailableSlots() int32 {
// TaskAssignment from admin to worker
type TaskAssignment struct {
- state protoimpl.MessageState
- sizeCache protoimpl.SizeCache
+ state protoimpl.MessageState `protogen:"open.v1"`
+ TaskId string `protobuf:"bytes,1,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
+ TaskType string `protobuf:"bytes,2,opt,name=task_type,json=taskType,proto3" json:"task_type,omitempty"`
+ Params *TaskParams `protobuf:"bytes,3,opt,name=params,proto3" json:"params,omitempty"`
+ Priority int32 `protobuf:"varint,4,opt,name=priority,proto3" json:"priority,omitempty"`
+ CreatedTime int64 `protobuf:"varint,5,opt,name=created_time,json=createdTime,proto3" json:"created_time,omitempty"`
+ Metadata map[string]string `protobuf:"bytes,6,rep,name=metadata,proto3" json:"metadata,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
unknownFields protoimpl.UnknownFields
-
- TaskId string `protobuf:"bytes,1,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
- TaskType string `protobuf:"bytes,2,opt,name=task_type,json=taskType,proto3" json:"task_type,omitempty"`
- Params *TaskParams `protobuf:"bytes,3,opt,name=params,proto3" json:"params,omitempty"`
- Priority int32 `protobuf:"varint,4,opt,name=priority,proto3" json:"priority,omitempty"`
- CreatedTime int64 `protobuf:"varint,5,opt,name=created_time,json=createdTime,proto3" json:"created_time,omitempty"`
- Metadata map[string]string `protobuf:"bytes,6,rep,name=metadata,proto3" json:"metadata,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
+ sizeCache protoimpl.SizeCache
}
func (x *TaskAssignment) Reset() {
*x = TaskAssignment{}
- if protoimpl.UnsafeEnabled {
- mi := &file_worker_proto_msgTypes[7]
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- ms.StoreMessageInfo(mi)
- }
+ mi := &file_worker_proto_msgTypes[7]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
}
func (x *TaskAssignment) String() string {
@@ -713,7 +712,7 @@ func (*TaskAssignment) ProtoMessage() {}
func (x *TaskAssignment) ProtoReflect() protoreflect.Message {
mi := &file_worker_proto_msgTypes[7]
- if protoimpl.UnsafeEnabled && x != nil {
+ if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
@@ -772,26 +771,23 @@ func (x *TaskAssignment) GetMetadata() map[string]string {
// TaskParams contains task-specific parameters
type TaskParams struct {
- state protoimpl.MessageState
- sizeCache protoimpl.SizeCache
+ state protoimpl.MessageState `protogen:"open.v1"`
+ VolumeId uint32 `protobuf:"varint,1,opt,name=volume_id,json=volumeId,proto3" json:"volume_id,omitempty"`
+ Server string `protobuf:"bytes,2,opt,name=server,proto3" json:"server,omitempty"`
+ Collection string `protobuf:"bytes,3,opt,name=collection,proto3" json:"collection,omitempty"`
+ DataCenter string `protobuf:"bytes,4,opt,name=data_center,json=dataCenter,proto3" json:"data_center,omitempty"`
+ Rack string `protobuf:"bytes,5,opt,name=rack,proto3" json:"rack,omitempty"`
+ Replicas []string `protobuf:"bytes,6,rep,name=replicas,proto3" json:"replicas,omitempty"`
+ Parameters map[string]string `protobuf:"bytes,7,rep,name=parameters,proto3" json:"parameters,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
unknownFields protoimpl.UnknownFields
-
- VolumeId uint32 `protobuf:"varint,1,opt,name=volume_id,json=volumeId,proto3" json:"volume_id,omitempty"`
- Server string `protobuf:"bytes,2,opt,name=server,proto3" json:"server,omitempty"`
- Collection string `protobuf:"bytes,3,opt,name=collection,proto3" json:"collection,omitempty"`
- DataCenter string `protobuf:"bytes,4,opt,name=data_center,json=dataCenter,proto3" json:"data_center,omitempty"`
- Rack string `protobuf:"bytes,5,opt,name=rack,proto3" json:"rack,omitempty"`
- Replicas []string `protobuf:"bytes,6,rep,name=replicas,proto3" json:"replicas,omitempty"`
- Parameters map[string]string `protobuf:"bytes,7,rep,name=parameters,proto3" json:"parameters,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
+ sizeCache protoimpl.SizeCache
}
func (x *TaskParams) Reset() {
*x = TaskParams{}
- if protoimpl.UnsafeEnabled {
- mi := &file_worker_proto_msgTypes[8]
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- ms.StoreMessageInfo(mi)
- }
+ mi := &file_worker_proto_msgTypes[8]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
}
func (x *TaskParams) String() string {
@@ -802,7 +798,7 @@ func (*TaskParams) ProtoMessage() {}
func (x *TaskParams) ProtoReflect() protoreflect.Message {
mi := &file_worker_proto_msgTypes[8]
- if protoimpl.UnsafeEnabled && x != nil {
+ if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
@@ -868,25 +864,22 @@ func (x *TaskParams) GetParameters() map[string]string {
// TaskUpdate reports task progress
type TaskUpdate struct {
- state protoimpl.MessageState
- sizeCache protoimpl.SizeCache
+ state protoimpl.MessageState `protogen:"open.v1"`
+ TaskId string `protobuf:"bytes,1,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
+ WorkerId string `protobuf:"bytes,2,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"`
+ Status string `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
+ Progress float32 `protobuf:"fixed32,4,opt,name=progress,proto3" json:"progress,omitempty"`
+ Message string `protobuf:"bytes,5,opt,name=message,proto3" json:"message,omitempty"`
+ Metadata map[string]string `protobuf:"bytes,6,rep,name=metadata,proto3" json:"metadata,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
unknownFields protoimpl.UnknownFields
-
- TaskId string `protobuf:"bytes,1,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
- WorkerId string `protobuf:"bytes,2,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"`
- Status string `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
- Progress float32 `protobuf:"fixed32,4,opt,name=progress,proto3" json:"progress,omitempty"`
- Message string `protobuf:"bytes,5,opt,name=message,proto3" json:"message,omitempty"`
- Metadata map[string]string `protobuf:"bytes,6,rep,name=metadata,proto3" json:"metadata,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
+ sizeCache protoimpl.SizeCache
}
func (x *TaskUpdate) Reset() {
*x = TaskUpdate{}
- if protoimpl.UnsafeEnabled {
- mi := &file_worker_proto_msgTypes[9]
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- ms.StoreMessageInfo(mi)
- }
+ mi := &file_worker_proto_msgTypes[9]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
}
func (x *TaskUpdate) String() string {
@@ -897,7 +890,7 @@ func (*TaskUpdate) ProtoMessage() {}
func (x *TaskUpdate) ProtoReflect() protoreflect.Message {
mi := &file_worker_proto_msgTypes[9]
- if protoimpl.UnsafeEnabled && x != nil {
+ if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
@@ -956,25 +949,22 @@ func (x *TaskUpdate) GetMetadata() map[string]string {
// TaskComplete reports task completion
type TaskComplete struct {
- state protoimpl.MessageState
- sizeCache protoimpl.SizeCache
- unknownFields protoimpl.UnknownFields
-
- TaskId string `protobuf:"bytes,1,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
- WorkerId string `protobuf:"bytes,2,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"`
- Success bool `protobuf:"varint,3,opt,name=success,proto3" json:"success,omitempty"`
- ErrorMessage string `protobuf:"bytes,4,opt,name=error_message,json=errorMessage,proto3" json:"error_message,omitempty"`
- CompletionTime int64 `protobuf:"varint,5,opt,name=completion_time,json=completionTime,proto3" json:"completion_time,omitempty"`
- ResultMetadata map[string]string `protobuf:"bytes,6,rep,name=result_metadata,json=resultMetadata,proto3" json:"result_metadata,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
+ state protoimpl.MessageState `protogen:"open.v1"`
+ TaskId string `protobuf:"bytes,1,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
+ WorkerId string `protobuf:"bytes,2,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"`
+ Success bool `protobuf:"varint,3,opt,name=success,proto3" json:"success,omitempty"`
+ ErrorMessage string `protobuf:"bytes,4,opt,name=error_message,json=errorMessage,proto3" json:"error_message,omitempty"`
+ CompletionTime int64 `protobuf:"varint,5,opt,name=completion_time,json=completionTime,proto3" json:"completion_time,omitempty"`
+ ResultMetadata map[string]string `protobuf:"bytes,6,rep,name=result_metadata,json=resultMetadata,proto3" json:"result_metadata,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+ unknownFields protoimpl.UnknownFields
+ sizeCache protoimpl.SizeCache
}
func (x *TaskComplete) Reset() {
*x = TaskComplete{}
- if protoimpl.UnsafeEnabled {
- mi := &file_worker_proto_msgTypes[10]
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- ms.StoreMessageInfo(mi)
- }
+ mi := &file_worker_proto_msgTypes[10]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
}
func (x *TaskComplete) String() string {
@@ -985,7 +975,7 @@ func (*TaskComplete) ProtoMessage() {}
func (x *TaskComplete) ProtoReflect() protoreflect.Message {
mi := &file_worker_proto_msgTypes[10]
- if protoimpl.UnsafeEnabled && x != nil {
+ if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
@@ -1044,22 +1034,19 @@ func (x *TaskComplete) GetResultMetadata() map[string]string {
// TaskCancellation from admin to cancel a task
type TaskCancellation struct {
- state protoimpl.MessageState
- sizeCache protoimpl.SizeCache
+ state protoimpl.MessageState `protogen:"open.v1"`
+ TaskId string `protobuf:"bytes,1,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
+ Reason string `protobuf:"bytes,2,opt,name=reason,proto3" json:"reason,omitempty"`
+ Force bool `protobuf:"varint,3,opt,name=force,proto3" json:"force,omitempty"`
unknownFields protoimpl.UnknownFields
-
- TaskId string `protobuf:"bytes,1,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
- Reason string `protobuf:"bytes,2,opt,name=reason,proto3" json:"reason,omitempty"`
- Force bool `protobuf:"varint,3,opt,name=force,proto3" json:"force,omitempty"`
+ sizeCache protoimpl.SizeCache
}
func (x *TaskCancellation) Reset() {
*x = TaskCancellation{}
- if protoimpl.UnsafeEnabled {
- mi := &file_worker_proto_msgTypes[11]
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- ms.StoreMessageInfo(mi)
- }
+ mi := &file_worker_proto_msgTypes[11]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
}
func (x *TaskCancellation) String() string {
@@ -1070,7 +1057,7 @@ func (*TaskCancellation) ProtoMessage() {}
func (x *TaskCancellation) ProtoReflect() protoreflect.Message {
mi := &file_worker_proto_msgTypes[11]
- if protoimpl.UnsafeEnabled && x != nil {
+ if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
@@ -1108,22 +1095,19 @@ func (x *TaskCancellation) GetForce() bool {
// WorkerShutdown notifies admin that worker is shutting down
type WorkerShutdown struct {
- state protoimpl.MessageState
- sizeCache protoimpl.SizeCache
- unknownFields protoimpl.UnknownFields
-
- WorkerId string `protobuf:"bytes,1,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"`
- Reason string `protobuf:"bytes,2,opt,name=reason,proto3" json:"reason,omitempty"`
- PendingTaskIds []string `protobuf:"bytes,3,rep,name=pending_task_ids,json=pendingTaskIds,proto3" json:"pending_task_ids,omitempty"`
+ state protoimpl.MessageState `protogen:"open.v1"`
+ WorkerId string `protobuf:"bytes,1,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"`
+ Reason string `protobuf:"bytes,2,opt,name=reason,proto3" json:"reason,omitempty"`
+ PendingTaskIds []string `protobuf:"bytes,3,rep,name=pending_task_ids,json=pendingTaskIds,proto3" json:"pending_task_ids,omitempty"`
+ unknownFields protoimpl.UnknownFields
+ sizeCache protoimpl.SizeCache
}
func (x *WorkerShutdown) Reset() {
*x = WorkerShutdown{}
- if protoimpl.UnsafeEnabled {
- mi := &file_worker_proto_msgTypes[12]
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- ms.StoreMessageInfo(mi)
- }
+ mi := &file_worker_proto_msgTypes[12]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
}
func (x *WorkerShutdown) String() string {
@@ -1134,7 +1118,7 @@ func (*WorkerShutdown) ProtoMessage() {}
func (x *WorkerShutdown) ProtoReflect() protoreflect.Message {
mi := &file_worker_proto_msgTypes[12]
- if protoimpl.UnsafeEnabled && x != nil {
+ if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
@@ -1172,21 +1156,18 @@ func (x *WorkerShutdown) GetPendingTaskIds() []string {
// AdminShutdown notifies worker that admin is shutting down
type AdminShutdown struct {
- state protoimpl.MessageState
- sizeCache protoimpl.SizeCache
- unknownFields protoimpl.UnknownFields
-
- Reason string `protobuf:"bytes,1,opt,name=reason,proto3" json:"reason,omitempty"`
- GracefulShutdownSeconds int32 `protobuf:"varint,2,opt,name=graceful_shutdown_seconds,json=gracefulShutdownSeconds,proto3" json:"graceful_shutdown_seconds,omitempty"`
+ state protoimpl.MessageState `protogen:"open.v1"`
+ Reason string `protobuf:"bytes,1,opt,name=reason,proto3" json:"reason,omitempty"`
+ GracefulShutdownSeconds int32 `protobuf:"varint,2,opt,name=graceful_shutdown_seconds,json=gracefulShutdownSeconds,proto3" json:"graceful_shutdown_seconds,omitempty"`
+ unknownFields protoimpl.UnknownFields
+ sizeCache protoimpl.SizeCache
}
func (x *AdminShutdown) Reset() {
*x = AdminShutdown{}
- if protoimpl.UnsafeEnabled {
- mi := &file_worker_proto_msgTypes[13]
- ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
- ms.StoreMessageInfo(mi)
- }
+ mi := &file_worker_proto_msgTypes[13]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
}
func (x *AdminShutdown) String() string {
@@ -1197,7 +1178,7 @@ func (*AdminShutdown) ProtoMessage() {}
func (x *AdminShutdown) ProtoReflect() protoreflect.Message {
mi := &file_worker_proto_msgTypes[13]
- if protoimpl.UnsafeEnabled && x != nil {
+ if x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
@@ -1228,238 +1209,128 @@ func (x *AdminShutdown) GetGracefulShutdownSeconds() int32 {
var File_worker_proto protoreflect.FileDescriptor
-var file_worker_proto_rawDesc = []byte{
- 0x0a, 0x0c, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x09,
- 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x22, 0xc6, 0x03, 0x0a, 0x0d, 0x57, 0x6f,
- 0x72, 0x6b, 0x65, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x77,
- 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08,
- 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65,
- 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x69, 0x6d,
- 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x43, 0x0a, 0x0c, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74,
- 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x77,
- 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52,
- 0x65, 0x67, 0x69, 0x73, 0x74, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x0c, 0x72,
- 0x65, 0x67, 0x69, 0x73, 0x74, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3a, 0x0a, 0x09, 0x68,
- 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a,
- 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65,
- 0x72, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x48, 0x00, 0x52, 0x09, 0x68, 0x65,
- 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x12, 0x3b, 0x0a, 0x0c, 0x74, 0x61, 0x73, 0x6b, 0x5f,
- 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e,
- 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65,
- 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0b, 0x74, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x71,
- 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0b, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x75, 0x70, 0x64,
- 0x61, 0x74, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x77, 0x6f, 0x72, 0x6b,
- 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65,
- 0x48, 0x00, 0x52, 0x0a, 0x74, 0x61, 0x73, 0x6b, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x12, 0x3e,
- 0x0a, 0x0d, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x18,
- 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x70,
- 0x62, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x48, 0x00,
- 0x52, 0x0c, 0x74, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x37,
- 0x0a, 0x08, 0x73, 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b,
- 0x32, 0x19, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72,
- 0x6b, 0x65, 0x72, 0x53, 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x48, 0x00, 0x52, 0x08, 0x73,
- 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61,
- 0x67, 0x65, 0x22, 0xce, 0x03, 0x0a, 0x0c, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x4d, 0x65, 0x73, 0x73,
- 0x61, 0x67, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18,
- 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x49, 0x64, 0x12, 0x1c,
- 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28,
- 0x03, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x56, 0x0a, 0x15,
- 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x73,
- 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x77, 0x6f,
- 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x72, 0x61,
- 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x14,
- 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70,
- 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4d, 0x0a, 0x12, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61,
- 0x74, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b,
- 0x32, 0x1c, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x48, 0x65, 0x61,
- 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00,
- 0x52, 0x11, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f,
- 0x6e, 0x73, 0x65, 0x12, 0x44, 0x0a, 0x0f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x61, 0x73, 0x73, 0x69,
- 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x77,
- 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x41, 0x73, 0x73,
- 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x48, 0x00, 0x52, 0x0e, 0x74, 0x61, 0x73, 0x6b, 0x41,
- 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x4a, 0x0a, 0x11, 0x74, 0x61, 0x73,
- 0x6b, 0x5f, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x06,
- 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x62,
- 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x69, 0x6f,
- 0x6e, 0x48, 0x00, 0x52, 0x10, 0x74, 0x61, 0x73, 0x6b, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x6c,
- 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x41, 0x0a, 0x0e, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x5f, 0x73,
- 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e,
- 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x53,
- 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x48, 0x00, 0x52, 0x0d, 0x61, 0x64, 0x6d, 0x69, 0x6e,
- 0x53, 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73,
- 0x61, 0x67, 0x65, 0x22, 0x9c, 0x02, 0x0a, 0x12, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65,
- 0x67, 0x69, 0x73, 0x74, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1b, 0x0a, 0x09, 0x77, 0x6f,
- 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77,
- 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65,
- 0x73, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73,
- 0x73, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65,
- 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c,
- 0x69, 0x74, 0x69, 0x65, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x6d, 0x61, 0x78, 0x5f, 0x63, 0x6f, 0x6e,
- 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x6d,
- 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x12, 0x47, 0x0a, 0x08,
- 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2b,
- 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65,
- 0x72, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x4d, 0x65,
- 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x6d, 0x65, 0x74,
- 0x61, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x3b, 0x0a, 0x0d, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74,
- 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20,
- 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75,
- 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02,
- 0x38, 0x01, 0x22, 0x78, 0x0a, 0x14, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x72, 0x61, 0x74, 0x69,
- 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75,
- 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63,
- 0x63, 0x65, 0x73, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18,
- 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x2c,
- 0x0a, 0x12, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65,
- 0x72, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x61, 0x73, 0x73, 0x69,
- 0x67, 0x6e, 0x65, 0x64, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 0x22, 0xad, 0x02, 0x0a,
- 0x0f, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74,
- 0x12, 0x1b, 0x0a, 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20,
- 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 0x12, 0x16, 0x0a,
- 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73,
- 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74,
- 0x5f, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x63, 0x75, 0x72,
- 0x72, 0x65, 0x6e, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x6d, 0x61, 0x78, 0x5f,
- 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05,
- 0x52, 0x0d, 0x6d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x12,
- 0x28, 0x0a, 0x10, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f,
- 0x69, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x75, 0x72, 0x72, 0x65,
- 0x6e, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x74, 0x61, 0x73,
- 0x6b, 0x73, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x18, 0x06, 0x20, 0x01,
- 0x28, 0x05, 0x52, 0x0e, 0x74, 0x61, 0x73, 0x6b, 0x73, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74,
- 0x65, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x61, 0x73, 0x6b, 0x73, 0x5f, 0x66, 0x61, 0x69, 0x6c,
- 0x65, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x74, 0x61, 0x73, 0x6b, 0x73, 0x46,
- 0x61, 0x69, 0x6c, 0x65, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x75, 0x70, 0x74, 0x69, 0x6d, 0x65, 0x5f,
- 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x75,
- 0x70, 0x74, 0x69, 0x6d, 0x65, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22, 0x47, 0x0a, 0x11,
- 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
- 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01,
- 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x6d,
- 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65,
- 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x77, 0x0a, 0x0b, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x71,
- 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x69,
- 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49,
- 0x64, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65,
- 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c,
- 0x69, 0x74, 0x69, 0x65, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62,
- 0x6c, 0x65, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0e,
- 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x6c, 0x6f, 0x74, 0x73, 0x22, 0xb6,
- 0x02, 0x0a, 0x0e, 0x54, 0x61, 0x73, 0x6b, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e,
- 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01,
- 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x61,
- 0x73, 0x6b, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74,
- 0x61, 0x73, 0x6b, 0x54, 0x79, 0x70, 0x65, 0x12, 0x2d, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d,
- 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72,
- 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x52, 0x06,
- 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69,
- 0x74, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69,
- 0x74, 0x79, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69,
- 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65,
- 0x64, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x43, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74,
- 0x61, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72,
- 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65,
- 0x6e, 0x74, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79,
- 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x3b, 0x0a, 0x0d, 0x4d, 0x65,
- 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b,
- 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a,
- 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61,
- 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xb8, 0x02, 0x0a, 0x0a, 0x54, 0x61, 0x73, 0x6b,
- 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65,
- 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x76, 0x6f, 0x6c, 0x75, 0x6d,
- 0x65, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x18, 0x02, 0x20,
- 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x12, 0x1e, 0x0a, 0x0a, 0x63,
- 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52,
- 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x64,
- 0x61, 0x74, 0x61, 0x5f, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09,
- 0x52, 0x0a, 0x64, 0x61, 0x74, 0x61, 0x43, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04,
- 0x72, 0x61, 0x63, 0x6b, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x72, 0x61, 0x63, 0x6b,
- 0x12, 0x1a, 0x0a, 0x08, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x18, 0x06, 0x20, 0x03,
- 0x28, 0x09, 0x52, 0x08, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x12, 0x45, 0x0a, 0x0a,
- 0x70, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b,
- 0x32, 0x25, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x61, 0x73,
- 0x6b, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x2e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65,
- 0x72, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74,
- 0x65, 0x72, 0x73, 0x1a, 0x3d, 0x0a, 0x0f, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72,
- 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20,
- 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75,
- 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02,
- 0x38, 0x01, 0x22, 0x8e, 0x02, 0x0a, 0x0a, 0x54, 0x61, 0x73, 0x6b, 0x55, 0x70, 0x64, 0x61, 0x74,
- 0x65, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01,
- 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x77, 0x6f,
- 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77,
- 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75,
- 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12,
- 0x1a, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28,
- 0x02, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x6d,
- 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65,
- 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x3f, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74,
- 0x61, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72,
- 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x2e, 0x4d,
- 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x6d, 0x65,
- 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x3b, 0x0a, 0x0d, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61,
- 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01,
- 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c,
- 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a,
- 0x02, 0x38, 0x01, 0x22, 0xc5, 0x02, 0x0a, 0x0c, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70,
- 0x6c, 0x65, 0x74, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18,
- 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x1b, 0x0a,
- 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09,
- 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75,
- 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x75, 0x63,
- 0x63, 0x65, 0x73, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, 0x65,
- 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, 0x72,
- 0x6f, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6d,
- 0x70, 0x6c, 0x65, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01,
- 0x28, 0x03, 0x52, 0x0e, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69,
- 0x6d, 0x65, 0x12, 0x54, 0x0a, 0x0f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x6d, 0x65, 0x74,
- 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x77, 0x6f,
- 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x43, 0x6f, 0x6d, 0x70,
- 0x6c, 0x65, 0x74, 0x65, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64,
- 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74,
- 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x41, 0x0a, 0x13, 0x52, 0x65, 0x73, 0x75,
- 0x6c, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12,
- 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65,
- 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09,
- 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x59, 0x0a, 0x10, 0x54,
- 0x61, 0x73, 0x6b, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12,
- 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
- 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73,
- 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e,
- 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52,
- 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x22, 0x6f, 0x0a, 0x0e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72,
- 0x53, 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x12, 0x1b, 0x0a, 0x09, 0x77, 0x6f, 0x72, 0x6b,
- 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72,
- 0x6b, 0x65, 0x72, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18,
- 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x28, 0x0a,
- 0x10, 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64,
- 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x70, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67,
- 0x54, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x73, 0x22, 0x63, 0x0a, 0x0d, 0x41, 0x64, 0x6d, 0x69, 0x6e,
- 0x53, 0x68, 0x75, 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73,
- 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e,
- 0x12, 0x3a, 0x0a, 0x19, 0x67, 0x72, 0x61, 0x63, 0x65, 0x66, 0x75, 0x6c, 0x5f, 0x73, 0x68, 0x75,
- 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x02, 0x20,
- 0x01, 0x28, 0x05, 0x52, 0x17, 0x67, 0x72, 0x61, 0x63, 0x65, 0x66, 0x75, 0x6c, 0x53, 0x68, 0x75,
- 0x74, 0x64, 0x6f, 0x77, 0x6e, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x32, 0x56, 0x0a, 0x0d,
- 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x45, 0x0a,
- 0x0c, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x18, 0x2e,
- 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72,
- 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x1a, 0x17, 0x2e, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72,
- 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
- 0x28, 0x01, 0x30, 0x01, 0x42, 0x32, 0x5a, 0x30, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63,
- 0x6f, 0x6d, 0x2f, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2f, 0x73, 0x65, 0x61,
- 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2f, 0x77, 0x65, 0x65, 0x64, 0x2f, 0x70, 0x62, 0x2f, 0x77,
- 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
-}
+const file_worker_proto_rawDesc = "" +
+ "\n" +
+ "\fworker.proto\x12\tworker_pb\"\xc6\x03\n" +
+ "\rWorkerMessage\x12\x1b\n" +
+ "\tworker_id\x18\x01 \x01(\tR\bworkerId\x12\x1c\n" +
+ "\ttimestamp\x18\x02 \x01(\x03R\ttimestamp\x12C\n" +
+ "\fregistration\x18\x03 \x01(\v2\x1d.worker_pb.WorkerRegistrationH\x00R\fregistration\x12:\n" +
+ "\theartbeat\x18\x04 \x01(\v2\x1a.worker_pb.WorkerHeartbeatH\x00R\theartbeat\x12;\n" +
+ "\ftask_request\x18\x05 \x01(\v2\x16.worker_pb.TaskRequestH\x00R\vtaskRequest\x128\n" +
+ "\vtask_update\x18\x06 \x01(\v2\x15.worker_pb.TaskUpdateH\x00R\n" +
+ "taskUpdate\x12>\n" +
+ "\rtask_complete\x18\a \x01(\v2\x17.worker_pb.TaskCompleteH\x00R\ftaskComplete\x127\n" +
+ "\bshutdown\x18\b \x01(\v2\x19.worker_pb.WorkerShutdownH\x00R\bshutdownB\t\n" +
+ "\amessage\"\xce\x03\n" +
+ "\fAdminMessage\x12\x19\n" +
+ "\badmin_id\x18\x01 \x01(\tR\aadminId\x12\x1c\n" +
+ "\ttimestamp\x18\x02 \x01(\x03R\ttimestamp\x12V\n" +
+ "\x15registration_response\x18\x03 \x01(\v2\x1f.worker_pb.RegistrationResponseH\x00R\x14registrationResponse\x12M\n" +
+ "\x12heartbeat_response\x18\x04 \x01(\v2\x1c.worker_pb.HeartbeatResponseH\x00R\x11heartbeatResponse\x12D\n" +
+ "\x0ftask_assignment\x18\x05 \x01(\v2\x19.worker_pb.TaskAssignmentH\x00R\x0etaskAssignment\x12J\n" +
+ "\x11task_cancellation\x18\x06 \x01(\v2\x1b.worker_pb.TaskCancellationH\x00R\x10taskCancellation\x12A\n" +
+ "\x0eadmin_shutdown\x18\a \x01(\v2\x18.worker_pb.AdminShutdownH\x00R\radminShutdownB\t\n" +
+ "\amessage\"\x9c\x02\n" +
+ "\x12WorkerRegistration\x12\x1b\n" +
+ "\tworker_id\x18\x01 \x01(\tR\bworkerId\x12\x18\n" +
+ "\aaddress\x18\x02 \x01(\tR\aaddress\x12\"\n" +
+ "\fcapabilities\x18\x03 \x03(\tR\fcapabilities\x12%\n" +
+ "\x0emax_concurrent\x18\x04 \x01(\x05R\rmaxConcurrent\x12G\n" +
+ "\bmetadata\x18\x05 \x03(\v2+.worker_pb.WorkerRegistration.MetadataEntryR\bmetadata\x1a;\n" +
+ "\rMetadataEntry\x12\x10\n" +
+ "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" +
+ "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"x\n" +
+ "\x14RegistrationResponse\x12\x18\n" +
+ "\asuccess\x18\x01 \x01(\bR\asuccess\x12\x18\n" +
+ "\amessage\x18\x02 \x01(\tR\amessage\x12,\n" +
+ "\x12assigned_worker_id\x18\x03 \x01(\tR\x10assignedWorkerId\"\xad\x02\n" +
+ "\x0fWorkerHeartbeat\x12\x1b\n" +
+ "\tworker_id\x18\x01 \x01(\tR\bworkerId\x12\x16\n" +
+ "\x06status\x18\x02 \x01(\tR\x06status\x12!\n" +
+ "\fcurrent_load\x18\x03 \x01(\x05R\vcurrentLoad\x12%\n" +
+ "\x0emax_concurrent\x18\x04 \x01(\x05R\rmaxConcurrent\x12(\n" +
+ "\x10current_task_ids\x18\x05 \x03(\tR\x0ecurrentTaskIds\x12'\n" +
+ "\x0ftasks_completed\x18\x06 \x01(\x05R\x0etasksCompleted\x12!\n" +
+ "\ftasks_failed\x18\a \x01(\x05R\vtasksFailed\x12%\n" +
+ "\x0euptime_seconds\x18\b \x01(\x03R\ruptimeSeconds\"G\n" +
+ "\x11HeartbeatResponse\x12\x18\n" +
+ "\asuccess\x18\x01 \x01(\bR\asuccess\x12\x18\n" +
+ "\amessage\x18\x02 \x01(\tR\amessage\"w\n" +
+ "\vTaskRequest\x12\x1b\n" +
+ "\tworker_id\x18\x01 \x01(\tR\bworkerId\x12\"\n" +
+ "\fcapabilities\x18\x02 \x03(\tR\fcapabilities\x12'\n" +
+ "\x0favailable_slots\x18\x03 \x01(\x05R\x0eavailableSlots\"\xb6\x02\n" +
+ "\x0eTaskAssignment\x12\x17\n" +
+ "\atask_id\x18\x01 \x01(\tR\x06taskId\x12\x1b\n" +
+ "\ttask_type\x18\x02 \x01(\tR\btaskType\x12-\n" +
+ "\x06params\x18\x03 \x01(\v2\x15.worker_pb.TaskParamsR\x06params\x12\x1a\n" +
+ "\bpriority\x18\x04 \x01(\x05R\bpriority\x12!\n" +
+ "\fcreated_time\x18\x05 \x01(\x03R\vcreatedTime\x12C\n" +
+ "\bmetadata\x18\x06 \x03(\v2'.worker_pb.TaskAssignment.MetadataEntryR\bmetadata\x1a;\n" +
+ "\rMetadataEntry\x12\x10\n" +
+ "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" +
+ "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"\xb8\x02\n" +
+ "\n" +
+ "TaskParams\x12\x1b\n" +
+ "\tvolume_id\x18\x01 \x01(\rR\bvolumeId\x12\x16\n" +
+ "\x06server\x18\x02 \x01(\tR\x06server\x12\x1e\n" +
+ "\n" +
+ "collection\x18\x03 \x01(\tR\n" +
+ "collection\x12\x1f\n" +
+ "\vdata_center\x18\x04 \x01(\tR\n" +
+ "dataCenter\x12\x12\n" +
+ "\x04rack\x18\x05 \x01(\tR\x04rack\x12\x1a\n" +
+ "\breplicas\x18\x06 \x03(\tR\breplicas\x12E\n" +
+ "\n" +
+ "parameters\x18\a \x03(\v2%.worker_pb.TaskParams.ParametersEntryR\n" +
+ "parameters\x1a=\n" +
+ "\x0fParametersEntry\x12\x10\n" +
+ "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" +
+ "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"\x8e\x02\n" +
+ "\n" +
+ "TaskUpdate\x12\x17\n" +
+ "\atask_id\x18\x01 \x01(\tR\x06taskId\x12\x1b\n" +
+ "\tworker_id\x18\x02 \x01(\tR\bworkerId\x12\x16\n" +
+ "\x06status\x18\x03 \x01(\tR\x06status\x12\x1a\n" +
+ "\bprogress\x18\x04 \x01(\x02R\bprogress\x12\x18\n" +
+ "\amessage\x18\x05 \x01(\tR\amessage\x12?\n" +
+ "\bmetadata\x18\x06 \x03(\v2#.worker_pb.TaskUpdate.MetadataEntryR\bmetadata\x1a;\n" +
+ "\rMetadataEntry\x12\x10\n" +
+ "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" +
+ "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"\xc5\x02\n" +
+ "\fTaskComplete\x12\x17\n" +
+ "\atask_id\x18\x01 \x01(\tR\x06taskId\x12\x1b\n" +
+ "\tworker_id\x18\x02 \x01(\tR\bworkerId\x12\x18\n" +
+ "\asuccess\x18\x03 \x01(\bR\asuccess\x12#\n" +
+ "\rerror_message\x18\x04 \x01(\tR\ferrorMessage\x12'\n" +
+ "\x0fcompletion_time\x18\x05 \x01(\x03R\x0ecompletionTime\x12T\n" +
+ "\x0fresult_metadata\x18\x06 \x03(\v2+.worker_pb.TaskComplete.ResultMetadataEntryR\x0eresultMetadata\x1aA\n" +
+ "\x13ResultMetadataEntry\x12\x10\n" +
+ "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" +
+ "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"Y\n" +
+ "\x10TaskCancellation\x12\x17\n" +
+ "\atask_id\x18\x01 \x01(\tR\x06taskId\x12\x16\n" +
+ "\x06reason\x18\x02 \x01(\tR\x06reason\x12\x14\n" +
+ "\x05force\x18\x03 \x01(\bR\x05force\"o\n" +
+ "\x0eWorkerShutdown\x12\x1b\n" +
+ "\tworker_id\x18\x01 \x01(\tR\bworkerId\x12\x16\n" +
+ "\x06reason\x18\x02 \x01(\tR\x06reason\x12(\n" +
+ "\x10pending_task_ids\x18\x03 \x03(\tR\x0ependingTaskIds\"c\n" +
+ "\rAdminShutdown\x12\x16\n" +
+ "\x06reason\x18\x01 \x01(\tR\x06reason\x12:\n" +
+ "\x19graceful_shutdown_seconds\x18\x02 \x01(\x05R\x17gracefulShutdownSeconds2V\n" +
+ "\rWorkerService\x12E\n" +
+ "\fWorkerStream\x12\x18.worker_pb.WorkerMessage\x1a\x17.worker_pb.AdminMessage(\x010\x01B2Z0github.com/seaweedfs/seaweedfs/weed/pb/worker_pbb\x06proto3"
var (
file_worker_proto_rawDescOnce sync.Once
- file_worker_proto_rawDescData = file_worker_proto_rawDesc
+ file_worker_proto_rawDescData []byte
)
func file_worker_proto_rawDescGZIP() []byte {
file_worker_proto_rawDescOnce.Do(func() {
- file_worker_proto_rawDescData = protoimpl.X.CompressGZIP(file_worker_proto_rawDescData)
+ file_worker_proto_rawDescData = protoimpl.X.CompressGZIP(unsafe.Slice(unsafe.StringData(file_worker_proto_rawDesc), len(file_worker_proto_rawDesc)))
})
return file_worker_proto_rawDescData
}
@@ -1518,176 +1389,6 @@ func file_worker_proto_init() {
if File_worker_proto != nil {
return
}
- if !protoimpl.UnsafeEnabled {
- file_worker_proto_msgTypes[0].Exporter = func(v any, i int) any {
- switch v := v.(*WorkerMessage); i {
- case 0:
- return &v.state
- case 1:
- return &v.sizeCache
- case 2:
- return &v.unknownFields
- default:
- return nil
- }
- }
- file_worker_proto_msgTypes[1].Exporter = func(v any, i int) any {
- switch v := v.(*AdminMessage); i {
- case 0:
- return &v.state
- case 1:
- return &v.sizeCache
- case 2:
- return &v.unknownFields
- default:
- return nil
- }
- }
- file_worker_proto_msgTypes[2].Exporter = func(v any, i int) any {
- switch v := v.(*WorkerRegistration); i {
- case 0:
- return &v.state
- case 1:
- return &v.sizeCache
- case 2:
- return &v.unknownFields
- default:
- return nil
- }
- }
- file_worker_proto_msgTypes[3].Exporter = func(v any, i int) any {
- switch v := v.(*RegistrationResponse); i {
- case 0:
- return &v.state
- case 1:
- return &v.sizeCache
- case 2:
- return &v.unknownFields
- default:
- return nil
- }
- }
- file_worker_proto_msgTypes[4].Exporter = func(v any, i int) any {
- switch v := v.(*WorkerHeartbeat); i {
- case 0:
- return &v.state
- case 1:
- return &v.sizeCache
- case 2:
- return &v.unknownFields
- default:
- return nil
- }
- }
- file_worker_proto_msgTypes[5].Exporter = func(v any, i int) any {
- switch v := v.(*HeartbeatResponse); i {
- case 0:
- return &v.state
- case 1:
- return &v.sizeCache
- case 2:
- return &v.unknownFields
- default:
- return nil
- }
- }
- file_worker_proto_msgTypes[6].Exporter = func(v any, i int) any {
- switch v := v.(*TaskRequest); i {
- case 0:
- return &v.state
- case 1:
- return &v.sizeCache
- case 2:
- return &v.unknownFields
- default:
- return nil
- }
- }
- file_worker_proto_msgTypes[7].Exporter = func(v any, i int) any {
- switch v := v.(*TaskAssignment); i {
- case 0:
- return &v.state
- case 1:
- return &v.sizeCache
- case 2:
- return &v.unknownFields
- default:
- return nil
- }
- }
- file_worker_proto_msgTypes[8].Exporter = func(v any, i int) any {
- switch v := v.(*TaskParams); i {
- case 0:
- return &v.state
- case 1:
- return &v.sizeCache
- case 2:
- return &v.unknownFields
- default:
- return nil
- }
- }
- file_worker_proto_msgTypes[9].Exporter = func(v any, i int) any {
- switch v := v.(*TaskUpdate); i {
- case 0:
- return &v.state
- case 1:
- return &v.sizeCache
- case 2:
- return &v.unknownFields
- default:
- return nil
- }
- }
- file_worker_proto_msgTypes[10].Exporter = func(v any, i int) any {
- switch v := v.(*TaskComplete); i {
- case 0:
- return &v.state
- case 1:
- return &v.sizeCache
- case 2:
- return &v.unknownFields
- default:
- return nil
- }
- }
- file_worker_proto_msgTypes[11].Exporter = func(v any, i int) any {
- switch v := v.(*TaskCancellation); i {
- case 0:
- return &v.state
- case 1:
- return &v.sizeCache
- case 2:
- return &v.unknownFields
- default:
- return nil
- }
- }
- file_worker_proto_msgTypes[12].Exporter = func(v any, i int) any {
- switch v := v.(*WorkerShutdown); i {
- case 0:
- return &v.state
- case 1:
- return &v.sizeCache
- case 2:
- return &v.unknownFields
- default:
- return nil
- }
- }
- file_worker_proto_msgTypes[13].Exporter = func(v any, i int) any {
- switch v := v.(*AdminShutdown); i {
- case 0:
- return &v.state
- case 1:
- return &v.sizeCache
- case 2:
- return &v.unknownFields
- default:
- return nil
- }
- }
- }
file_worker_proto_msgTypes[0].OneofWrappers = []any{
(*WorkerMessage_Registration)(nil),
(*WorkerMessage_Heartbeat)(nil),
@@ -1707,7 +1408,7 @@ func file_worker_proto_init() {
out := protoimpl.TypeBuilder{
File: protoimpl.DescBuilder{
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
- RawDescriptor: file_worker_proto_rawDesc,
+ RawDescriptor: unsafe.Slice(unsafe.StringData(file_worker_proto_rawDesc), len(file_worker_proto_rawDesc)),
NumEnums: 0,
NumMessages: 19,
NumExtensions: 0,
@@ -1718,7 +1419,6 @@ func file_worker_proto_init() {
MessageInfos: file_worker_proto_msgTypes,
}.Build()
File_worker_proto = out.File
- file_worker_proto_rawDesc = nil
file_worker_proto_goTypes = nil
file_worker_proto_depIdxs = nil
}