Browse Source

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
pull/6973/head
Chris Lu 5 months ago
committed by GitHub
parent
commit
51543bbb87
No known key found for this signature in database GPG Key ID: B5690EEEBB952194
  1. 5
      .gitignore
  2. 228
      test/mq/Makefile
  3. 244
      test/mq/README.md
  4. 192
      test/mq/consumer/main.go
  5. 172
      test/mq/producer/main.go
  6. 83
      weed/admin/dash/admin_data.go
  7. 293
      weed/admin/dash/admin_server.go
  8. 91
      weed/admin/dash/bucket_management.go
  9. 615
      weed/admin/dash/mq_management.go
  10. 296
      weed/admin/dash/topic_retention.go
  11. 144
      weed/admin/dash/types.go
  12. 31
      weed/admin/handlers/admin_handlers.go
  13. 27
      weed/admin/handlers/cluster_handlers.go
  14. 238
      weed/admin/handlers/mq_handlers.go
  15. 14
      weed/admin/static_embed.go
  16. 12
      weed/admin/view/app/admin.templ
  17. 165
      weed/admin/view/app/admin_templ.go
  18. 144
      weed/admin/view/app/cluster_brokers.templ
  19. 168
      weed/admin/view/app/cluster_brokers_templ.go
  20. 272
      weed/admin/view/app/s3_buckets.templ
  21. 138
      weed/admin/view/app/s3_buckets_templ.go
  22. 151
      weed/admin/view/app/subscribers.templ
  23. 246
      weed/admin/view/app/subscribers_templ.go
  24. 677
      weed/admin/view/app/topic_details.templ
  25. 949
      weed/admin/view/app/topic_details_templ.go
  26. 511
      weed/admin/view/app/topics.templ
  27. 230
      weed/admin/view/app/topics_templ.go
  28. 61
      weed/admin/view/layout/layout.templ
  29. 161
      weed/admin/view/layout/layout_templ.go
  30. 11
      weed/command/admin.go
  31. 20
      weed/command/mq_agent.go
  32. 19
      weed/command/mq_broker.go
  33. 15
      weed/filer_client/filer_client_accessor.go
  34. 1
      weed/mq/broker/broker_grpc_configure.go
  35. 260
      weed/mq/broker/broker_grpc_lookup.go
  36. 23
      weed/mq/broker/broker_grpc_pub.go
  37. 14
      weed/mq/broker/broker_grpc_sub.go
  38. 67
      weed/mq/topic/local_partition_publishers.go
  39. 74
      weed/mq/topic/local_partition_subscribers.go
  40. 34
      weed/mq/topic/topic.go
  41. 64
      weed/pb/mq_broker.proto
  42. 1332
      weed/pb/mq_pb/mq_broker.pb.go
  43. 114
      weed/pb/mq_pb/mq_broker_grpc.pb.go
  44. 944
      weed/pb/worker_pb/worker.pb.go

5
.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

228
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"

244
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

192
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)"
}
}

172
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
}

83
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
}

293
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...")

91
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 {

615
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,
}
}

296
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
}

144
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"`

31
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)
}
}
}
}

27
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()

238
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,
})
}

14
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")
}

12
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>

165
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
}

144
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>
}

168
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

272
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>
@ -151,6 +153,33 @@ templ S3Buckets(data dash.S3BucketsData) {
<span class="text-muted">No quota</span>
}
</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))}
@ -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

138
weed/admin/view/app/s3_buckets_templ.go
File diff suppressed because it is too large
View File

151
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>
}

246
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

677
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>
}

949
weed/admin/view/app/topic_details_templ.go
File diff suppressed because it is too large
View File

511
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>
}

230
weed/admin/view/app/topics_templ.go
File diff suppressed because it is too large
View File

61
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">
@ -147,6 +150,64 @@ templ Layout(c *gin.Context, content templ.Component) {
</ul>
</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

161
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
}

11
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

20
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

19
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

15
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
}

1
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 {

260
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()
}

23
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)

14
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
})

67
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)
}
}

74
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)
}
}

34
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)

64
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;

1332
weed/pb/mq_pb/mq_broker.pb.go
File diff suppressed because it is too large
View File

114
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 {
@ -539,6 +641,18 @@ var SeaweedMessaging_ServiceDesc = grpc.ServiceDesc{
MethodName: "LookupTopicBrokers",
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,

944
weed/pb/worker_pb/worker.pb.go
File diff suppressed because it is too large
View File

Loading…
Cancel
Save