mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2025-10-08 04:11:34 +08:00
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
This commit is contained in:
@@ -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)
|
||||
}
|
||||
}
|
||||
|
@@ -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)
|
||||
}
|
||||
}
|
||||
|
@@ -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)
|
||||
|
Reference in New Issue
Block a user