Browse Source
Admin UI add maintenance menu (#6944)
Admin UI add maintenance menu (#6944)
* add ui for maintenance * valid config loading. fix workers page. * refactor * grpc between admin and workers * add a long-running bidirectional grpc call between admin and worker * use the grpc call to heartbeat * use the grpc call to communicate * worker can remove the http client * admin uses http port + 10000 as its default grpc port * one task one package * handles connection failures gracefully with exponential backoff * grpc with insecure tls * grpc with optional tls * fix detecting tls * change time config from nano seconds to seconds * add tasks with 3 interfaces * compiles reducing hard coded * remove a couple of tasks * remove hard coded references * reduce hard coded values * remove hard coded values * remove hard coded from templ * refactor maintenance package * fix import cycle * simplify * simplify * auto register * auto register factory * auto register task types * self register types * refactor * simplify * remove one task * register ui * lazy init executor factories * use registered task types * DefaultWorkerConfig remove hard coded task types * remove more hard coded * implement get maintenance task * dynamic task configuration * "System Settings" should only have system level settings * adjust menu for tasks * ensure menu not collapsed * render job configuration well * use templ for ui of task configuration * fix ordering * fix bugs * saving duration in seconds * use value and unit for duration * Delete WORKER_REFACTORING_PLAN.md * Delete maintenance.json * Delete custom_worker_example.go * remove address from workers * remove old code from ec task * remove creating collection button * reconnect with exponential backoff * worker use security.toml * start admin server with tls info from security.toml * fix "weed admin" cli descriptionpull/6948/head
committed by
GitHub
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
76 changed files with 18218 additions and 206 deletions
-
632weed/admin/dash/admin_server.go
-
270weed/admin/dash/config_persistence.go
-
49weed/admin/dash/types.go
-
461weed/admin/dash/worker_grpc_server.go
-
53weed/admin/handlers/admin_handlers.go
-
388weed/admin/handlers/maintenance_handlers.go
-
409weed/admin/maintenance/maintenance_integration.go
-
407weed/admin/maintenance/maintenance_manager.go
-
140weed/admin/maintenance/maintenance_manager_test.go
-
500weed/admin/maintenance/maintenance_queue.go
-
163weed/admin/maintenance/maintenance_scanner.go
-
560weed/admin/maintenance/maintenance_types.go
-
413weed/admin/maintenance/maintenance_worker.go
-
94weed/admin/static/js/admin.js
-
67weed/admin/view/app/cluster_collections.templ
-
28weed/admin/view/app/cluster_collections_templ.go
-
244weed/admin/view/app/maintenance_config.templ
-
280weed/admin/view/app/maintenance_config_templ.go
-
289weed/admin/view/app/maintenance_queue.templ
-
585weed/admin/view/app/maintenance_queue_templ.go
-
340weed/admin/view/app/maintenance_workers.templ
-
431weed/admin/view/app/maintenance_workers_templ.go
-
160weed/admin/view/app/task_config.templ
-
174weed/admin/view/app/task_config_templ.go
-
160weed/admin/view/app/task_config_templ.templ
-
112weed/admin/view/app/task_config_templ_templ.go
-
83weed/admin/view/components/config_sections.templ
-
257weed/admin/view/components/config_sections_templ.go
-
306weed/admin/view/components/form_fields.templ
-
1104weed/admin/view/components/form_fields_templ.go
-
75weed/admin/view/layout/layout.templ
-
301weed/admin/view/layout/layout_templ.go
-
47weed/admin/view/layout/menu_helper.go
-
190weed/command/admin.go
-
1weed/command/command.go
-
20weed/command/scaffold/security.toml
-
182weed/command/worker.go
-
1weed/pb/Makefile
-
8weed/pb/grpc_client_server.go
-
142weed/pb/worker.proto
-
1724weed/pb/worker_pb/worker.pb.go
-
121weed/pb/worker_pb/worker_grpc.pb.go
-
761weed/worker/client.go
-
111weed/worker/client_test.go
-
146weed/worker/client_tls_test.go
-
348weed/worker/registry.go
-
82weed/worker/tasks/balance/balance.go
-
171weed/worker/tasks/balance/balance_detector.go
-
81weed/worker/tasks/balance/balance_register.go
-
197weed/worker/tasks/balance/balance_scheduler.go
-
361weed/worker/tasks/balance/ui.go
-
369weed/worker/tasks/balance/ui_templ.go
-
79weed/worker/tasks/erasure_coding/ec.go
-
139weed/worker/tasks/erasure_coding/ec_detector.go
-
81weed/worker/tasks/erasure_coding/ec_register.go
-
114weed/worker/tasks/erasure_coding/ec_scheduler.go
-
309weed/worker/tasks/erasure_coding/ui.go
-
319weed/worker/tasks/erasure_coding/ui_templ.go
-
110weed/worker/tasks/registry.go
-
252weed/worker/tasks/task.go
-
314weed/worker/tasks/vacuum/ui.go
-
330weed/worker/tasks/vacuum/ui_templ.go
-
79weed/worker/tasks/vacuum/vacuum.go
-
132weed/worker/tasks/vacuum/vacuum_detector.go
-
81weed/worker/tasks/vacuum/vacuum_register.go
-
111weed/worker/tasks/vacuum/vacuum_scheduler.go
-
268weed/worker/types/config_types.go
-
40weed/worker/types/data_types.go
-
28weed/worker/types/task_detector.go
-
54weed/worker/types/task_registry.go
-
32weed/worker/types/task_scheduler.go
-
89weed/worker/types/task_types.go
-
281weed/worker/types/task_ui.go
-
63weed/worker/types/task_ui_templ.go
-
111weed/worker/types/worker_types.go
-
410weed/worker/worker.go
@ -0,0 +1,270 @@ |
|||
package dash |
|||
|
|||
import ( |
|||
"encoding/json" |
|||
"fmt" |
|||
"os" |
|||
"path/filepath" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
) |
|||
|
|||
const ( |
|||
// Configuration file names
|
|||
MaintenanceConfigFile = "maintenance.json" |
|||
AdminConfigFile = "admin.json" |
|||
ConfigDirPermissions = 0755 |
|||
ConfigFilePermissions = 0644 |
|||
) |
|||
|
|||
// ConfigPersistence handles saving and loading configuration files
|
|||
type ConfigPersistence struct { |
|||
dataDir string |
|||
} |
|||
|
|||
// NewConfigPersistence creates a new configuration persistence manager
|
|||
func NewConfigPersistence(dataDir string) *ConfigPersistence { |
|||
return &ConfigPersistence{ |
|||
dataDir: dataDir, |
|||
} |
|||
} |
|||
|
|||
// SaveMaintenanceConfig saves maintenance configuration to JSON file
|
|||
func (cp *ConfigPersistence) SaveMaintenanceConfig(config *MaintenanceConfig) error { |
|||
if cp.dataDir == "" { |
|||
return fmt.Errorf("no data directory specified, cannot save configuration") |
|||
} |
|||
|
|||
configPath := filepath.Join(cp.dataDir, MaintenanceConfigFile) |
|||
|
|||
// Create directory if it doesn't exist
|
|||
if err := os.MkdirAll(cp.dataDir, ConfigDirPermissions); err != nil { |
|||
return fmt.Errorf("failed to create config directory: %v", err) |
|||
} |
|||
|
|||
// Marshal configuration to JSON
|
|||
configData, err := json.MarshalIndent(config, "", " ") |
|||
if err != nil { |
|||
return fmt.Errorf("failed to marshal maintenance config: %v", err) |
|||
} |
|||
|
|||
// Write to file
|
|||
if err := os.WriteFile(configPath, configData, ConfigFilePermissions); err != nil { |
|||
return fmt.Errorf("failed to write maintenance config file: %v", err) |
|||
} |
|||
|
|||
glog.V(1).Infof("Saved maintenance configuration to %s", configPath) |
|||
return nil |
|||
} |
|||
|
|||
// LoadMaintenanceConfig loads maintenance configuration from JSON file
|
|||
func (cp *ConfigPersistence) LoadMaintenanceConfig() (*MaintenanceConfig, error) { |
|||
if cp.dataDir == "" { |
|||
glog.V(1).Infof("No data directory specified, using default maintenance configuration") |
|||
return DefaultMaintenanceConfig(), nil |
|||
} |
|||
|
|||
configPath := filepath.Join(cp.dataDir, MaintenanceConfigFile) |
|||
|
|||
// Check if file exists
|
|||
if _, err := os.Stat(configPath); os.IsNotExist(err) { |
|||
glog.V(1).Infof("Maintenance config file does not exist, using defaults: %s", configPath) |
|||
return DefaultMaintenanceConfig(), nil |
|||
} |
|||
|
|||
// Read file
|
|||
configData, err := os.ReadFile(configPath) |
|||
if err != nil { |
|||
return nil, fmt.Errorf("failed to read maintenance config file: %v", err) |
|||
} |
|||
|
|||
// Unmarshal JSON
|
|||
var config MaintenanceConfig |
|||
if err := json.Unmarshal(configData, &config); err != nil { |
|||
return nil, fmt.Errorf("failed to unmarshal maintenance config: %v", err) |
|||
} |
|||
|
|||
glog.V(1).Infof("Loaded maintenance configuration from %s", configPath) |
|||
return &config, nil |
|||
} |
|||
|
|||
// SaveAdminConfig saves general admin configuration to JSON file
|
|||
func (cp *ConfigPersistence) SaveAdminConfig(config map[string]interface{}) error { |
|||
if cp.dataDir == "" { |
|||
return fmt.Errorf("no data directory specified, cannot save configuration") |
|||
} |
|||
|
|||
configPath := filepath.Join(cp.dataDir, AdminConfigFile) |
|||
|
|||
// Create directory if it doesn't exist
|
|||
if err := os.MkdirAll(cp.dataDir, ConfigDirPermissions); err != nil { |
|||
return fmt.Errorf("failed to create config directory: %v", err) |
|||
} |
|||
|
|||
// Marshal configuration to JSON
|
|||
configData, err := json.MarshalIndent(config, "", " ") |
|||
if err != nil { |
|||
return fmt.Errorf("failed to marshal admin config: %v", err) |
|||
} |
|||
|
|||
// Write to file
|
|||
if err := os.WriteFile(configPath, configData, ConfigFilePermissions); err != nil { |
|||
return fmt.Errorf("failed to write admin config file: %v", err) |
|||
} |
|||
|
|||
glog.V(1).Infof("Saved admin configuration to %s", configPath) |
|||
return nil |
|||
} |
|||
|
|||
// LoadAdminConfig loads general admin configuration from JSON file
|
|||
func (cp *ConfigPersistence) LoadAdminConfig() (map[string]interface{}, error) { |
|||
if cp.dataDir == "" { |
|||
glog.V(1).Infof("No data directory specified, using default admin configuration") |
|||
return make(map[string]interface{}), nil |
|||
} |
|||
|
|||
configPath := filepath.Join(cp.dataDir, AdminConfigFile) |
|||
|
|||
// Check if file exists
|
|||
if _, err := os.Stat(configPath); os.IsNotExist(err) { |
|||
glog.V(1).Infof("Admin config file does not exist, using defaults: %s", configPath) |
|||
return make(map[string]interface{}), nil |
|||
} |
|||
|
|||
// Read file
|
|||
configData, err := os.ReadFile(configPath) |
|||
if err != nil { |
|||
return nil, fmt.Errorf("failed to read admin config file: %v", err) |
|||
} |
|||
|
|||
// Unmarshal JSON
|
|||
var config map[string]interface{} |
|||
if err := json.Unmarshal(configData, &config); err != nil { |
|||
return nil, fmt.Errorf("failed to unmarshal admin config: %v", err) |
|||
} |
|||
|
|||
glog.V(1).Infof("Loaded admin configuration from %s", configPath) |
|||
return config, nil |
|||
} |
|||
|
|||
// GetConfigPath returns the path to a configuration file
|
|||
func (cp *ConfigPersistence) GetConfigPath(filename string) string { |
|||
if cp.dataDir == "" { |
|||
return "" |
|||
} |
|||
return filepath.Join(cp.dataDir, filename) |
|||
} |
|||
|
|||
// ListConfigFiles returns all configuration files in the data directory
|
|||
func (cp *ConfigPersistence) ListConfigFiles() ([]string, error) { |
|||
if cp.dataDir == "" { |
|||
return nil, fmt.Errorf("no data directory specified") |
|||
} |
|||
|
|||
files, err := os.ReadDir(cp.dataDir) |
|||
if err != nil { |
|||
return nil, fmt.Errorf("failed to read config directory: %v", err) |
|||
} |
|||
|
|||
var configFiles []string |
|||
for _, file := range files { |
|||
if !file.IsDir() && filepath.Ext(file.Name()) == ".json" { |
|||
configFiles = append(configFiles, file.Name()) |
|||
} |
|||
} |
|||
|
|||
return configFiles, nil |
|||
} |
|||
|
|||
// BackupConfig creates a backup of a configuration file
|
|||
func (cp *ConfigPersistence) BackupConfig(filename string) error { |
|||
if cp.dataDir == "" { |
|||
return fmt.Errorf("no data directory specified") |
|||
} |
|||
|
|||
configPath := filepath.Join(cp.dataDir, filename) |
|||
if _, err := os.Stat(configPath); os.IsNotExist(err) { |
|||
return fmt.Errorf("config file does not exist: %s", filename) |
|||
} |
|||
|
|||
// Create backup filename with timestamp
|
|||
timestamp := time.Now().Format("2006-01-02_15-04-05") |
|||
backupName := fmt.Sprintf("%s.backup_%s", filename, timestamp) |
|||
backupPath := filepath.Join(cp.dataDir, backupName) |
|||
|
|||
// Copy file
|
|||
configData, err := os.ReadFile(configPath) |
|||
if err != nil { |
|||
return fmt.Errorf("failed to read config file: %v", err) |
|||
} |
|||
|
|||
if err := os.WriteFile(backupPath, configData, ConfigFilePermissions); err != nil { |
|||
return fmt.Errorf("failed to create backup: %v", err) |
|||
} |
|||
|
|||
glog.V(1).Infof("Created backup of %s as %s", filename, backupName) |
|||
return nil |
|||
} |
|||
|
|||
// RestoreConfig restores a configuration file from a backup
|
|||
func (cp *ConfigPersistence) RestoreConfig(filename, backupName string) error { |
|||
if cp.dataDir == "" { |
|||
return fmt.Errorf("no data directory specified") |
|||
} |
|||
|
|||
backupPath := filepath.Join(cp.dataDir, backupName) |
|||
if _, err := os.Stat(backupPath); os.IsNotExist(err) { |
|||
return fmt.Errorf("backup file does not exist: %s", backupName) |
|||
} |
|||
|
|||
// Read backup file
|
|||
backupData, err := os.ReadFile(backupPath) |
|||
if err != nil { |
|||
return fmt.Errorf("failed to read backup file: %v", err) |
|||
} |
|||
|
|||
// Write to config file
|
|||
configPath := filepath.Join(cp.dataDir, filename) |
|||
if err := os.WriteFile(configPath, backupData, ConfigFilePermissions); err != nil { |
|||
return fmt.Errorf("failed to restore config: %v", err) |
|||
} |
|||
|
|||
glog.V(1).Infof("Restored %s from backup %s", filename, backupName) |
|||
return nil |
|||
} |
|||
|
|||
// GetDataDir returns the data directory path
|
|||
func (cp *ConfigPersistence) GetDataDir() string { |
|||
return cp.dataDir |
|||
} |
|||
|
|||
// IsConfigured returns true if a data directory is configured
|
|||
func (cp *ConfigPersistence) IsConfigured() bool { |
|||
return cp.dataDir != "" |
|||
} |
|||
|
|||
// GetConfigInfo returns information about the configuration storage
|
|||
func (cp *ConfigPersistence) GetConfigInfo() map[string]interface{} { |
|||
info := map[string]interface{}{ |
|||
"data_dir_configured": cp.IsConfigured(), |
|||
"data_dir": cp.dataDir, |
|||
} |
|||
|
|||
if cp.IsConfigured() { |
|||
// Check if data directory exists
|
|||
if _, err := os.Stat(cp.dataDir); err == nil { |
|||
info["data_dir_exists"] = true |
|||
|
|||
// List config files
|
|||
configFiles, err := cp.ListConfigFiles() |
|||
if err == nil { |
|||
info["config_files"] = configFiles |
|||
} |
|||
} else { |
|||
info["data_dir_exists"] = false |
|||
} |
|||
} |
|||
|
|||
return info |
|||
} |
|||
@ -0,0 +1,461 @@ |
|||
package dash |
|||
|
|||
import ( |
|||
"context" |
|||
"fmt" |
|||
"io" |
|||
"net" |
|||
"sync" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/pb" |
|||
"github.com/seaweedfs/seaweedfs/weed/pb/worker_pb" |
|||
"github.com/seaweedfs/seaweedfs/weed/security" |
|||
"github.com/seaweedfs/seaweedfs/weed/util" |
|||
"google.golang.org/grpc" |
|||
"google.golang.org/grpc/peer" |
|||
) |
|||
|
|||
// WorkerGrpcServer implements the WorkerService gRPC interface
|
|||
type WorkerGrpcServer struct { |
|||
worker_pb.UnimplementedWorkerServiceServer |
|||
adminServer *AdminServer |
|||
|
|||
// Worker connection management
|
|||
connections map[string]*WorkerConnection |
|||
connMutex sync.RWMutex |
|||
|
|||
// gRPC server
|
|||
grpcServer *grpc.Server |
|||
listener net.Listener |
|||
running bool |
|||
stopChan chan struct{} |
|||
} |
|||
|
|||
// WorkerConnection represents an active worker connection
|
|||
type WorkerConnection struct { |
|||
workerID string |
|||
stream worker_pb.WorkerService_WorkerStreamServer |
|||
lastSeen time.Time |
|||
capabilities []MaintenanceTaskType |
|||
address string |
|||
maxConcurrent int32 |
|||
outgoing chan *worker_pb.AdminMessage |
|||
ctx context.Context |
|||
cancel context.CancelFunc |
|||
} |
|||
|
|||
// NewWorkerGrpcServer creates a new gRPC server for worker connections
|
|||
func NewWorkerGrpcServer(adminServer *AdminServer) *WorkerGrpcServer { |
|||
return &WorkerGrpcServer{ |
|||
adminServer: adminServer, |
|||
connections: make(map[string]*WorkerConnection), |
|||
stopChan: make(chan struct{}), |
|||
} |
|||
} |
|||
|
|||
// StartWithTLS starts the gRPC server on the specified port with optional TLS
|
|||
func (s *WorkerGrpcServer) StartWithTLS(port int) error { |
|||
if s.running { |
|||
return fmt.Errorf("worker gRPC server is already running") |
|||
} |
|||
|
|||
// Create listener
|
|||
listener, err := net.Listen("tcp", fmt.Sprintf(":%d", port)) |
|||
if err != nil { |
|||
return fmt.Errorf("failed to listen on port %d: %v", port, err) |
|||
} |
|||
|
|||
// Create gRPC server with optional TLS
|
|||
grpcServer := pb.NewGrpcServer(security.LoadServerTLS(util.GetViper(), "grpc.admin")) |
|||
|
|||
worker_pb.RegisterWorkerServiceServer(grpcServer, s) |
|||
|
|||
s.grpcServer = grpcServer |
|||
s.listener = listener |
|||
s.running = true |
|||
|
|||
// Start cleanup routine
|
|||
go s.cleanupRoutine() |
|||
|
|||
// Start serving in a goroutine
|
|||
go func() { |
|||
if err := s.grpcServer.Serve(listener); err != nil { |
|||
if s.running { |
|||
glog.Errorf("Worker gRPC server error: %v", err) |
|||
} |
|||
} |
|||
}() |
|||
|
|||
return nil |
|||
} |
|||
|
|||
// Stop stops the gRPC server
|
|||
func (s *WorkerGrpcServer) Stop() error { |
|||
if !s.running { |
|||
return nil |
|||
} |
|||
|
|||
s.running = false |
|||
close(s.stopChan) |
|||
|
|||
// Close all worker connections
|
|||
s.connMutex.Lock() |
|||
for _, conn := range s.connections { |
|||
conn.cancel() |
|||
close(conn.outgoing) |
|||
} |
|||
s.connections = make(map[string]*WorkerConnection) |
|||
s.connMutex.Unlock() |
|||
|
|||
// Stop gRPC server
|
|||
if s.grpcServer != nil { |
|||
s.grpcServer.GracefulStop() |
|||
} |
|||
|
|||
// Close listener
|
|||
if s.listener != nil { |
|||
s.listener.Close() |
|||
} |
|||
|
|||
glog.Infof("Worker gRPC server stopped") |
|||
return nil |
|||
} |
|||
|
|||
// WorkerStream handles bidirectional communication with workers
|
|||
func (s *WorkerGrpcServer) WorkerStream(stream worker_pb.WorkerService_WorkerStreamServer) error { |
|||
ctx := stream.Context() |
|||
|
|||
// get client address
|
|||
address := findClientAddress(ctx) |
|||
|
|||
// Wait for initial registration message
|
|||
msg, err := stream.Recv() |
|||
if err != nil { |
|||
return fmt.Errorf("failed to receive registration message: %v", err) |
|||
} |
|||
|
|||
registration := msg.GetRegistration() |
|||
if registration == nil { |
|||
return fmt.Errorf("first message must be registration") |
|||
} |
|||
registration.Address = address |
|||
|
|||
workerID := registration.WorkerId |
|||
if workerID == "" { |
|||
return fmt.Errorf("worker ID cannot be empty") |
|||
} |
|||
|
|||
glog.Infof("Worker %s connecting from %s", workerID, registration.Address) |
|||
|
|||
// Create worker connection
|
|||
connCtx, connCancel := context.WithCancel(ctx) |
|||
conn := &WorkerConnection{ |
|||
workerID: workerID, |
|||
stream: stream, |
|||
lastSeen: time.Now(), |
|||
address: registration.Address, |
|||
maxConcurrent: registration.MaxConcurrent, |
|||
outgoing: make(chan *worker_pb.AdminMessage, 100), |
|||
ctx: connCtx, |
|||
cancel: connCancel, |
|||
} |
|||
|
|||
// Convert capabilities
|
|||
capabilities := make([]MaintenanceTaskType, len(registration.Capabilities)) |
|||
for i, cap := range registration.Capabilities { |
|||
capabilities[i] = MaintenanceTaskType(cap) |
|||
} |
|||
conn.capabilities = capabilities |
|||
|
|||
// Register connection
|
|||
s.connMutex.Lock() |
|||
s.connections[workerID] = conn |
|||
s.connMutex.Unlock() |
|||
|
|||
// Register worker with maintenance manager
|
|||
s.registerWorkerWithManager(conn) |
|||
|
|||
// Send registration response
|
|||
regResponse := &worker_pb.AdminMessage{ |
|||
Timestamp: time.Now().Unix(), |
|||
Message: &worker_pb.AdminMessage_RegistrationResponse{ |
|||
RegistrationResponse: &worker_pb.RegistrationResponse{ |
|||
Success: true, |
|||
Message: "Worker registered successfully", |
|||
}, |
|||
}, |
|||
} |
|||
|
|||
select { |
|||
case conn.outgoing <- regResponse: |
|||
case <-time.After(5 * time.Second): |
|||
glog.Errorf("Failed to send registration response to worker %s", workerID) |
|||
} |
|||
|
|||
// Start outgoing message handler
|
|||
go s.handleOutgoingMessages(conn) |
|||
|
|||
// Handle incoming messages
|
|||
for { |
|||
select { |
|||
case <-ctx.Done(): |
|||
glog.Infof("Worker %s connection closed: %v", workerID, ctx.Err()) |
|||
s.unregisterWorker(workerID) |
|||
return nil |
|||
case <-connCtx.Done(): |
|||
glog.Infof("Worker %s connection cancelled", workerID) |
|||
s.unregisterWorker(workerID) |
|||
return nil |
|||
default: |
|||
} |
|||
|
|||
msg, err := stream.Recv() |
|||
if err != nil { |
|||
if err == io.EOF { |
|||
glog.Infof("Worker %s disconnected", workerID) |
|||
} else { |
|||
glog.Errorf("Error receiving from worker %s: %v", workerID, err) |
|||
} |
|||
s.unregisterWorker(workerID) |
|||
return err |
|||
} |
|||
|
|||
conn.lastSeen = time.Now() |
|||
s.handleWorkerMessage(conn, msg) |
|||
} |
|||
} |
|||
|
|||
// handleOutgoingMessages sends messages to worker
|
|||
func (s *WorkerGrpcServer) handleOutgoingMessages(conn *WorkerConnection) { |
|||
for { |
|||
select { |
|||
case <-conn.ctx.Done(): |
|||
return |
|||
case msg, ok := <-conn.outgoing: |
|||
if !ok { |
|||
return |
|||
} |
|||
|
|||
if err := conn.stream.Send(msg); err != nil { |
|||
glog.Errorf("Failed to send message to worker %s: %v", conn.workerID, err) |
|||
conn.cancel() |
|||
return |
|||
} |
|||
} |
|||
} |
|||
} |
|||
|
|||
// handleWorkerMessage processes incoming messages from workers
|
|||
func (s *WorkerGrpcServer) handleWorkerMessage(conn *WorkerConnection, msg *worker_pb.WorkerMessage) { |
|||
workerID := conn.workerID |
|||
|
|||
switch m := msg.Message.(type) { |
|||
case *worker_pb.WorkerMessage_Heartbeat: |
|||
s.handleHeartbeat(conn, m.Heartbeat) |
|||
|
|||
case *worker_pb.WorkerMessage_TaskRequest: |
|||
s.handleTaskRequest(conn, m.TaskRequest) |
|||
|
|||
case *worker_pb.WorkerMessage_TaskUpdate: |
|||
s.handleTaskUpdate(conn, m.TaskUpdate) |
|||
|
|||
case *worker_pb.WorkerMessage_TaskComplete: |
|||
s.handleTaskCompletion(conn, m.TaskComplete) |
|||
|
|||
case *worker_pb.WorkerMessage_Shutdown: |
|||
glog.Infof("Worker %s shutting down: %s", workerID, m.Shutdown.Reason) |
|||
s.unregisterWorker(workerID) |
|||
|
|||
default: |
|||
glog.Warningf("Unknown message type from worker %s", workerID) |
|||
} |
|||
} |
|||
|
|||
// registerWorkerWithManager registers the worker with the maintenance manager
|
|||
func (s *WorkerGrpcServer) registerWorkerWithManager(conn *WorkerConnection) { |
|||
if s.adminServer.maintenanceManager == nil { |
|||
return |
|||
} |
|||
|
|||
worker := &MaintenanceWorker{ |
|||
ID: conn.workerID, |
|||
Address: conn.address, |
|||
LastHeartbeat: time.Now(), |
|||
Status: "active", |
|||
Capabilities: conn.capabilities, |
|||
MaxConcurrent: int(conn.maxConcurrent), |
|||
CurrentLoad: 0, |
|||
} |
|||
|
|||
s.adminServer.maintenanceManager.RegisterWorker(worker) |
|||
glog.V(1).Infof("Registered worker %s with maintenance manager", conn.workerID) |
|||
} |
|||
|
|||
// handleHeartbeat processes heartbeat messages
|
|||
func (s *WorkerGrpcServer) handleHeartbeat(conn *WorkerConnection, heartbeat *worker_pb.WorkerHeartbeat) { |
|||
if s.adminServer.maintenanceManager != nil { |
|||
s.adminServer.maintenanceManager.UpdateWorkerHeartbeat(conn.workerID) |
|||
} |
|||
|
|||
// Send heartbeat response
|
|||
response := &worker_pb.AdminMessage{ |
|||
Timestamp: time.Now().Unix(), |
|||
Message: &worker_pb.AdminMessage_HeartbeatResponse{ |
|||
HeartbeatResponse: &worker_pb.HeartbeatResponse{ |
|||
Success: true, |
|||
Message: "Heartbeat acknowledged", |
|||
}, |
|||
}, |
|||
} |
|||
|
|||
select { |
|||
case conn.outgoing <- response: |
|||
case <-time.After(time.Second): |
|||
glog.Warningf("Failed to send heartbeat response to worker %s", conn.workerID) |
|||
} |
|||
} |
|||
|
|||
// handleTaskRequest processes task requests from workers
|
|||
func (s *WorkerGrpcServer) handleTaskRequest(conn *WorkerConnection, request *worker_pb.TaskRequest) { |
|||
if s.adminServer.maintenanceManager == nil { |
|||
return |
|||
} |
|||
|
|||
// Get next task from maintenance manager
|
|||
task := s.adminServer.maintenanceManager.GetNextTask(conn.workerID, conn.capabilities) |
|||
|
|||
if task != nil { |
|||
// Send task assignment
|
|||
assignment := &worker_pb.AdminMessage{ |
|||
Timestamp: time.Now().Unix(), |
|||
Message: &worker_pb.AdminMessage_TaskAssignment{ |
|||
TaskAssignment: &worker_pb.TaskAssignment{ |
|||
TaskId: task.ID, |
|||
TaskType: string(task.Type), |
|||
Params: &worker_pb.TaskParams{ |
|||
VolumeId: task.VolumeID, |
|||
Server: task.Server, |
|||
Collection: task.Collection, |
|||
Parameters: convertTaskParameters(task.Parameters), |
|||
}, |
|||
Priority: int32(task.Priority), |
|||
CreatedTime: time.Now().Unix(), |
|||
}, |
|||
}, |
|||
} |
|||
|
|||
select { |
|||
case conn.outgoing <- assignment: |
|||
glog.V(2).Infof("Assigned task %s to worker %s", task.ID, conn.workerID) |
|||
case <-time.After(time.Second): |
|||
glog.Warningf("Failed to send task assignment to worker %s", conn.workerID) |
|||
} |
|||
} |
|||
} |
|||
|
|||
// handleTaskUpdate processes task progress updates
|
|||
func (s *WorkerGrpcServer) handleTaskUpdate(conn *WorkerConnection, update *worker_pb.TaskUpdate) { |
|||
if s.adminServer.maintenanceManager != nil { |
|||
s.adminServer.maintenanceManager.UpdateTaskProgress(update.TaskId, float64(update.Progress)) |
|||
glog.V(3).Infof("Updated task %s progress: %.1f%%", update.TaskId, update.Progress) |
|||
} |
|||
} |
|||
|
|||
// handleTaskCompletion processes task completion notifications
|
|||
func (s *WorkerGrpcServer) handleTaskCompletion(conn *WorkerConnection, completion *worker_pb.TaskComplete) { |
|||
if s.adminServer.maintenanceManager != nil { |
|||
errorMsg := "" |
|||
if !completion.Success { |
|||
errorMsg = completion.ErrorMessage |
|||
} |
|||
s.adminServer.maintenanceManager.CompleteTask(completion.TaskId, errorMsg) |
|||
|
|||
if completion.Success { |
|||
glog.V(1).Infof("Worker %s completed task %s successfully", conn.workerID, completion.TaskId) |
|||
} else { |
|||
glog.Errorf("Worker %s failed task %s: %s", conn.workerID, completion.TaskId, completion.ErrorMessage) |
|||
} |
|||
} |
|||
} |
|||
|
|||
// unregisterWorker removes a worker connection
|
|||
func (s *WorkerGrpcServer) unregisterWorker(workerID string) { |
|||
s.connMutex.Lock() |
|||
if conn, exists := s.connections[workerID]; exists { |
|||
conn.cancel() |
|||
close(conn.outgoing) |
|||
delete(s.connections, workerID) |
|||
} |
|||
s.connMutex.Unlock() |
|||
|
|||
glog.V(1).Infof("Unregistered worker %s", workerID) |
|||
} |
|||
|
|||
// cleanupRoutine periodically cleans up stale connections
|
|||
func (s *WorkerGrpcServer) cleanupRoutine() { |
|||
ticker := time.NewTicker(30 * time.Second) |
|||
defer ticker.Stop() |
|||
|
|||
for { |
|||
select { |
|||
case <-s.stopChan: |
|||
return |
|||
case <-ticker.C: |
|||
s.cleanupStaleConnections() |
|||
} |
|||
} |
|||
} |
|||
|
|||
// cleanupStaleConnections removes connections that haven't been seen recently
|
|||
func (s *WorkerGrpcServer) cleanupStaleConnections() { |
|||
cutoff := time.Now().Add(-2 * time.Minute) |
|||
|
|||
s.connMutex.Lock() |
|||
defer s.connMutex.Unlock() |
|||
|
|||
for workerID, conn := range s.connections { |
|||
if conn.lastSeen.Before(cutoff) { |
|||
glog.Warningf("Cleaning up stale worker connection: %s", workerID) |
|||
conn.cancel() |
|||
close(conn.outgoing) |
|||
delete(s.connections, workerID) |
|||
} |
|||
} |
|||
} |
|||
|
|||
// GetConnectedWorkers returns a list of currently connected workers
|
|||
func (s *WorkerGrpcServer) GetConnectedWorkers() []string { |
|||
s.connMutex.RLock() |
|||
defer s.connMutex.RUnlock() |
|||
|
|||
workers := make([]string, 0, len(s.connections)) |
|||
for workerID := range s.connections { |
|||
workers = append(workers, workerID) |
|||
} |
|||
return workers |
|||
} |
|||
|
|||
// convertTaskParameters converts task parameters to protobuf format
|
|||
func convertTaskParameters(params map[string]interface{}) map[string]string { |
|||
result := make(map[string]string) |
|||
for key, value := range params { |
|||
result[key] = fmt.Sprintf("%v", value) |
|||
} |
|||
return result |
|||
} |
|||
|
|||
func findClientAddress(ctx context.Context) string { |
|||
// fmt.Printf("FromContext %+v\n", ctx)
|
|||
pr, ok := peer.FromContext(ctx) |
|||
if !ok { |
|||
glog.Error("failed to get peer from ctx") |
|||
return "" |
|||
} |
|||
if pr.Addr == net.Addr(nil) { |
|||
glog.Error("failed to get peer address") |
|||
return "" |
|||
} |
|||
return pr.Addr.String() |
|||
} |
|||
@ -0,0 +1,388 @@ |
|||
package handlers |
|||
|
|||
import ( |
|||
"net/http" |
|||
"time" |
|||
|
|||
"github.com/gin-gonic/gin" |
|||
"github.com/seaweedfs/seaweedfs/weed/admin/dash" |
|||
"github.com/seaweedfs/seaweedfs/weed/admin/maintenance" |
|||
"github.com/seaweedfs/seaweedfs/weed/admin/view/app" |
|||
"github.com/seaweedfs/seaweedfs/weed/admin/view/components" |
|||
"github.com/seaweedfs/seaweedfs/weed/admin/view/layout" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/tasks" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/tasks/balance" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/tasks/erasure_coding" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/tasks/vacuum" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// MaintenanceHandlers handles maintenance-related HTTP requests
|
|||
type MaintenanceHandlers struct { |
|||
adminServer *dash.AdminServer |
|||
} |
|||
|
|||
// NewMaintenanceHandlers creates a new instance of MaintenanceHandlers
|
|||
func NewMaintenanceHandlers(adminServer *dash.AdminServer) *MaintenanceHandlers { |
|||
return &MaintenanceHandlers{ |
|||
adminServer: adminServer, |
|||
} |
|||
} |
|||
|
|||
// ShowMaintenanceQueue displays the maintenance queue page
|
|||
func (h *MaintenanceHandlers) ShowMaintenanceQueue(c *gin.Context) { |
|||
data, err := h.getMaintenanceQueueData() |
|||
if err != nil { |
|||
c.JSON(http.StatusInternalServerError, gin.H{"error": err.Error()}) |
|||
return |
|||
} |
|||
|
|||
// Render HTML template
|
|||
c.Header("Content-Type", "text/html") |
|||
maintenanceComponent := app.MaintenanceQueue(data) |
|||
layoutComponent := layout.Layout(c, maintenanceComponent) |
|||
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 |
|||
} |
|||
} |
|||
|
|||
// ShowMaintenanceWorkers displays the maintenance workers page
|
|||
func (h *MaintenanceHandlers) ShowMaintenanceWorkers(c *gin.Context) { |
|||
workersData, err := h.adminServer.GetMaintenanceWorkersData() |
|||
if err != nil { |
|||
c.JSON(http.StatusInternalServerError, gin.H{"error": err.Error()}) |
|||
return |
|||
} |
|||
|
|||
// Render HTML template
|
|||
c.Header("Content-Type", "text/html") |
|||
workersComponent := app.MaintenanceWorkers(workersData) |
|||
layoutComponent := layout.Layout(c, workersComponent) |
|||
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 |
|||
} |
|||
} |
|||
|
|||
// ShowMaintenanceConfig displays the maintenance configuration page
|
|||
func (h *MaintenanceHandlers) ShowMaintenanceConfig(c *gin.Context) { |
|||
config, err := h.getMaintenanceConfig() |
|||
if err != nil { |
|||
c.JSON(http.StatusInternalServerError, gin.H{"error": err.Error()}) |
|||
return |
|||
} |
|||
|
|||
// Render HTML template
|
|||
c.Header("Content-Type", "text/html") |
|||
configComponent := app.MaintenanceConfig(config) |
|||
layoutComponent := layout.Layout(c, configComponent) |
|||
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 |
|||
} |
|||
} |
|||
|
|||
// ShowTaskConfig displays the configuration page for a specific task type
|
|||
func (h *MaintenanceHandlers) ShowTaskConfig(c *gin.Context) { |
|||
taskTypeName := c.Param("taskType") |
|||
|
|||
// Get the task type
|
|||
taskType := maintenance.GetMaintenanceTaskType(taskTypeName) |
|||
if taskType == "" { |
|||
c.JSON(http.StatusNotFound, gin.H{"error": "Task type not found"}) |
|||
return |
|||
} |
|||
|
|||
// Get the UI provider for this task type
|
|||
uiRegistry := tasks.GetGlobalUIRegistry() |
|||
typesRegistry := tasks.GetGlobalTypesRegistry() |
|||
|
|||
var provider types.TaskUIProvider |
|||
for workerTaskType := range typesRegistry.GetAllDetectors() { |
|||
if string(workerTaskType) == string(taskType) { |
|||
provider = uiRegistry.GetProvider(workerTaskType) |
|||
break |
|||
} |
|||
} |
|||
|
|||
if provider == nil { |
|||
c.JSON(http.StatusNotFound, gin.H{"error": "UI provider not found for task type"}) |
|||
return |
|||
} |
|||
|
|||
// Try to get templ UI provider first
|
|||
templUIProvider := getTemplUIProvider(taskType) |
|||
var configSections []components.ConfigSectionData |
|||
|
|||
if templUIProvider != nil { |
|||
// Use the new templ-based UI provider
|
|||
currentConfig := templUIProvider.GetCurrentConfig() |
|||
sections, err := templUIProvider.RenderConfigSections(currentConfig) |
|||
if err != nil { |
|||
c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to render configuration sections: " + err.Error()}) |
|||
return |
|||
} |
|||
configSections = sections |
|||
} else { |
|||
// Fallback to basic configuration for providers that haven't been migrated yet
|
|||
configSections = []components.ConfigSectionData{ |
|||
{ |
|||
Title: "Configuration Settings", |
|||
Icon: "fas fa-cogs", |
|||
Description: "Configure task detection and scheduling parameters", |
|||
Fields: []interface{}{ |
|||
components.CheckboxFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "enabled", |
|||
Label: "Enable Task", |
|||
Description: "Whether this task type should be enabled", |
|||
}, |
|||
Checked: true, |
|||
}, |
|||
components.NumberFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "max_concurrent", |
|||
Label: "Max Concurrent Tasks", |
|||
Description: "Maximum number of concurrent tasks", |
|||
Required: true, |
|||
}, |
|||
Value: 2, |
|||
Step: "1", |
|||
Min: floatPtr(1), |
|||
}, |
|||
components.DurationFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "scan_interval", |
|||
Label: "Scan Interval", |
|||
Description: "How often to scan for tasks", |
|||
Required: true, |
|||
}, |
|||
Value: "30m", |
|||
}, |
|||
}, |
|||
}, |
|||
} |
|||
} |
|||
|
|||
// Create task configuration data using templ components
|
|||
configData := &app.TaskConfigTemplData{ |
|||
TaskType: taskType, |
|||
TaskName: provider.GetDisplayName(), |
|||
TaskIcon: provider.GetIcon(), |
|||
Description: provider.GetDescription(), |
|||
ConfigSections: configSections, |
|||
} |
|||
|
|||
// Render HTML template using templ components
|
|||
c.Header("Content-Type", "text/html") |
|||
taskConfigComponent := app.TaskConfigTempl(configData) |
|||
layoutComponent := layout.Layout(c, taskConfigComponent) |
|||
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 |
|||
} |
|||
} |
|||
|
|||
// UpdateTaskConfig updates configuration for a specific task type
|
|||
func (h *MaintenanceHandlers) UpdateTaskConfig(c *gin.Context) { |
|||
taskTypeName := c.Param("taskType") |
|||
|
|||
// Get the task type
|
|||
taskType := maintenance.GetMaintenanceTaskType(taskTypeName) |
|||
if taskType == "" { |
|||
c.JSON(http.StatusNotFound, gin.H{"error": "Task type not found"}) |
|||
return |
|||
} |
|||
|
|||
// Try to get templ UI provider first
|
|||
templUIProvider := getTemplUIProvider(taskType) |
|||
|
|||
// Parse form data
|
|||
err := c.Request.ParseForm() |
|||
if err != nil { |
|||
c.JSON(http.StatusBadRequest, gin.H{"error": "Failed to parse form data: " + err.Error()}) |
|||
return |
|||
} |
|||
|
|||
// Convert form data to map
|
|||
formData := make(map[string][]string) |
|||
for key, values := range c.Request.PostForm { |
|||
formData[key] = values |
|||
} |
|||
|
|||
var config interface{} |
|||
|
|||
if templUIProvider != nil { |
|||
// Use the new templ-based UI provider
|
|||
config, err = templUIProvider.ParseConfigForm(formData) |
|||
if err != nil { |
|||
c.JSON(http.StatusBadRequest, gin.H{"error": "Failed to parse configuration: " + err.Error()}) |
|||
return |
|||
} |
|||
|
|||
// Apply configuration using templ provider
|
|||
err = templUIProvider.ApplyConfig(config) |
|||
if err != nil { |
|||
c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to apply configuration: " + err.Error()}) |
|||
return |
|||
} |
|||
} else { |
|||
// Fallback to old UI provider for tasks that haven't been migrated yet
|
|||
uiRegistry := tasks.GetGlobalUIRegistry() |
|||
typesRegistry := tasks.GetGlobalTypesRegistry() |
|||
|
|||
var provider types.TaskUIProvider |
|||
for workerTaskType := range typesRegistry.GetAllDetectors() { |
|||
if string(workerTaskType) == string(taskType) { |
|||
provider = uiRegistry.GetProvider(workerTaskType) |
|||
break |
|||
} |
|||
} |
|||
|
|||
if provider == nil { |
|||
c.JSON(http.StatusNotFound, gin.H{"error": "UI provider not found for task type"}) |
|||
return |
|||
} |
|||
|
|||
// Parse configuration from form using old provider
|
|||
config, err = provider.ParseConfigForm(formData) |
|||
if err != nil { |
|||
c.JSON(http.StatusBadRequest, gin.H{"error": "Failed to parse configuration: " + err.Error()}) |
|||
return |
|||
} |
|||
|
|||
// Apply configuration using old provider
|
|||
err = provider.ApplyConfig(config) |
|||
if err != nil { |
|||
c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to apply configuration: " + err.Error()}) |
|||
return |
|||
} |
|||
} |
|||
|
|||
// Redirect back to task configuration page
|
|||
c.Redirect(http.StatusSeeOther, "/maintenance/config/"+taskTypeName) |
|||
} |
|||
|
|||
// UpdateMaintenanceConfig updates maintenance configuration from form
|
|||
func (h *MaintenanceHandlers) UpdateMaintenanceConfig(c *gin.Context) { |
|||
var config maintenance.MaintenanceConfig |
|||
if err := c.ShouldBind(&config); err != nil { |
|||
c.JSON(http.StatusBadRequest, gin.H{"error": err.Error()}) |
|||
return |
|||
} |
|||
|
|||
err := h.updateMaintenanceConfig(&config) |
|||
if err != nil { |
|||
c.JSON(http.StatusInternalServerError, gin.H{"error": err.Error()}) |
|||
return |
|||
} |
|||
|
|||
c.Redirect(http.StatusSeeOther, "/maintenance/config") |
|||
} |
|||
|
|||
// Helper methods that delegate to AdminServer
|
|||
|
|||
func (h *MaintenanceHandlers) getMaintenanceQueueData() (*maintenance.MaintenanceQueueData, error) { |
|||
tasks, err := h.getMaintenanceTasks() |
|||
if err != nil { |
|||
return nil, err |
|||
} |
|||
|
|||
workers, err := h.getMaintenanceWorkers() |
|||
if err != nil { |
|||
return nil, err |
|||
} |
|||
|
|||
stats, err := h.getMaintenanceQueueStats() |
|||
if err != nil { |
|||
return nil, err |
|||
} |
|||
|
|||
return &maintenance.MaintenanceQueueData{ |
|||
Tasks: tasks, |
|||
Workers: workers, |
|||
Stats: stats, |
|||
LastUpdated: time.Now(), |
|||
}, nil |
|||
} |
|||
|
|||
func (h *MaintenanceHandlers) getMaintenanceQueueStats() (*maintenance.QueueStats, error) { |
|||
// This would integrate with the maintenance queue to get real statistics
|
|||
// For now, return mock data
|
|||
return &maintenance.QueueStats{ |
|||
PendingTasks: 5, |
|||
RunningTasks: 2, |
|||
CompletedToday: 15, |
|||
FailedToday: 1, |
|||
TotalTasks: 23, |
|||
}, nil |
|||
} |
|||
|
|||
func (h *MaintenanceHandlers) getMaintenanceTasks() ([]*maintenance.MaintenanceTask, error) { |
|||
// This would integrate with the maintenance queue to get real tasks
|
|||
// For now, return mock data
|
|||
return []*maintenance.MaintenanceTask{}, nil |
|||
} |
|||
|
|||
func (h *MaintenanceHandlers) getMaintenanceWorkers() ([]*maintenance.MaintenanceWorker, error) { |
|||
// This would integrate with the maintenance system to get real workers
|
|||
// For now, return mock data
|
|||
return []*maintenance.MaintenanceWorker{}, nil |
|||
} |
|||
|
|||
func (h *MaintenanceHandlers) getMaintenanceConfig() (*maintenance.MaintenanceConfigData, error) { |
|||
// Delegate to AdminServer's real persistence method
|
|||
return h.adminServer.GetMaintenanceConfigData() |
|||
} |
|||
|
|||
func (h *MaintenanceHandlers) updateMaintenanceConfig(config *maintenance.MaintenanceConfig) error { |
|||
// Delegate to AdminServer's real persistence method
|
|||
return h.adminServer.UpdateMaintenanceConfigData(config) |
|||
} |
|||
|
|||
// floatPtr is a helper function to create float64 pointers
|
|||
func floatPtr(f float64) *float64 { |
|||
return &f |
|||
} |
|||
|
|||
// Global templ UI registry
|
|||
var globalTemplUIRegistry *types.UITemplRegistry |
|||
|
|||
// initTemplUIRegistry initializes the global templ UI registry
|
|||
func initTemplUIRegistry() { |
|||
if globalTemplUIRegistry == nil { |
|||
globalTemplUIRegistry = types.NewUITemplRegistry() |
|||
|
|||
// Register vacuum templ UI provider using shared instances
|
|||
vacuumDetector, vacuumScheduler := vacuum.GetSharedInstances() |
|||
vacuum.RegisterUITempl(globalTemplUIRegistry, vacuumDetector, vacuumScheduler) |
|||
|
|||
// Register erasure coding templ UI provider using shared instances
|
|||
erasureCodingDetector, erasureCodingScheduler := erasure_coding.GetSharedInstances() |
|||
erasure_coding.RegisterUITempl(globalTemplUIRegistry, erasureCodingDetector, erasureCodingScheduler) |
|||
|
|||
// Register balance templ UI provider using shared instances
|
|||
balanceDetector, balanceScheduler := balance.GetSharedInstances() |
|||
balance.RegisterUITempl(globalTemplUIRegistry, balanceDetector, balanceScheduler) |
|||
} |
|||
} |
|||
|
|||
// getTemplUIProvider gets the templ UI provider for a task type
|
|||
func getTemplUIProvider(taskType maintenance.MaintenanceTaskType) types.TaskUITemplProvider { |
|||
initTemplUIRegistry() |
|||
|
|||
// Convert maintenance task type to worker task type
|
|||
typesRegistry := tasks.GetGlobalTypesRegistry() |
|||
for workerTaskType := range typesRegistry.GetAllDetectors() { |
|||
if string(workerTaskType) == string(taskType) { |
|||
return globalTemplUIRegistry.GetProvider(workerTaskType) |
|||
} |
|||
} |
|||
|
|||
return nil |
|||
} |
|||
@ -0,0 +1,409 @@ |
|||
package maintenance |
|||
|
|||
import ( |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/tasks" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// MaintenanceIntegration bridges the task system with existing maintenance
|
|||
type MaintenanceIntegration struct { |
|||
taskRegistry *types.TaskRegistry |
|||
uiRegistry *types.UIRegistry |
|||
|
|||
// Bridge to existing system
|
|||
maintenanceQueue *MaintenanceQueue |
|||
maintenancePolicy *MaintenancePolicy |
|||
|
|||
// Type conversion maps
|
|||
taskTypeMap map[types.TaskType]MaintenanceTaskType |
|||
revTaskTypeMap map[MaintenanceTaskType]types.TaskType |
|||
priorityMap map[types.TaskPriority]MaintenanceTaskPriority |
|||
revPriorityMap map[MaintenanceTaskPriority]types.TaskPriority |
|||
} |
|||
|
|||
// NewMaintenanceIntegration creates the integration bridge
|
|||
func NewMaintenanceIntegration(queue *MaintenanceQueue, policy *MaintenancePolicy) *MaintenanceIntegration { |
|||
integration := &MaintenanceIntegration{ |
|||
taskRegistry: tasks.GetGlobalTypesRegistry(), // Use global types registry with auto-registered tasks
|
|||
uiRegistry: tasks.GetGlobalUIRegistry(), // Use global UI registry with auto-registered UI providers
|
|||
maintenanceQueue: queue, |
|||
maintenancePolicy: policy, |
|||
} |
|||
|
|||
// Initialize type conversion maps
|
|||
integration.initializeTypeMaps() |
|||
|
|||
// Register all tasks
|
|||
integration.registerAllTasks() |
|||
|
|||
return integration |
|||
} |
|||
|
|||
// initializeTypeMaps creates the type conversion maps for dynamic conversion
|
|||
func (s *MaintenanceIntegration) initializeTypeMaps() { |
|||
// Initialize empty maps
|
|||
s.taskTypeMap = make(map[types.TaskType]MaintenanceTaskType) |
|||
s.revTaskTypeMap = make(map[MaintenanceTaskType]types.TaskType) |
|||
|
|||
// Build task type mappings dynamically from registered tasks after registration
|
|||
// This will be called from registerAllTasks() after all tasks are registered
|
|||
|
|||
// Priority mappings (these are static and don't depend on registered tasks)
|
|||
s.priorityMap = map[types.TaskPriority]MaintenanceTaskPriority{ |
|||
types.TaskPriorityLow: PriorityLow, |
|||
types.TaskPriorityNormal: PriorityNormal, |
|||
types.TaskPriorityHigh: PriorityHigh, |
|||
} |
|||
|
|||
// Reverse priority mappings
|
|||
s.revPriorityMap = map[MaintenanceTaskPriority]types.TaskPriority{ |
|||
PriorityLow: types.TaskPriorityLow, |
|||
PriorityNormal: types.TaskPriorityNormal, |
|||
PriorityHigh: types.TaskPriorityHigh, |
|||
PriorityCritical: types.TaskPriorityHigh, // Map critical to high
|
|||
} |
|||
} |
|||
|
|||
// buildTaskTypeMappings dynamically builds task type mappings from registered tasks
|
|||
func (s *MaintenanceIntegration) buildTaskTypeMappings() { |
|||
// Clear existing mappings
|
|||
s.taskTypeMap = make(map[types.TaskType]MaintenanceTaskType) |
|||
s.revTaskTypeMap = make(map[MaintenanceTaskType]types.TaskType) |
|||
|
|||
// Build mappings from registered detectors
|
|||
for workerTaskType := range s.taskRegistry.GetAllDetectors() { |
|||
// Convert types.TaskType to MaintenanceTaskType by string conversion
|
|||
maintenanceTaskType := MaintenanceTaskType(string(workerTaskType)) |
|||
|
|||
s.taskTypeMap[workerTaskType] = maintenanceTaskType |
|||
s.revTaskTypeMap[maintenanceTaskType] = workerTaskType |
|||
|
|||
glog.V(3).Infof("Dynamically mapped task type: %s <-> %s", workerTaskType, maintenanceTaskType) |
|||
} |
|||
|
|||
glog.V(2).Infof("Built %d dynamic task type mappings", len(s.taskTypeMap)) |
|||
} |
|||
|
|||
// registerAllTasks registers all available tasks
|
|||
func (s *MaintenanceIntegration) registerAllTasks() { |
|||
// Tasks are already auto-registered via import statements
|
|||
// No manual registration needed
|
|||
|
|||
// Build dynamic type mappings from registered tasks
|
|||
s.buildTaskTypeMappings() |
|||
|
|||
// Configure tasks from policy
|
|||
s.configureTasksFromPolicy() |
|||
|
|||
registeredTaskTypes := make([]string, 0, len(s.taskTypeMap)) |
|||
for _, maintenanceTaskType := range s.taskTypeMap { |
|||
registeredTaskTypes = append(registeredTaskTypes, string(maintenanceTaskType)) |
|||
} |
|||
glog.V(1).Infof("Registered tasks: %v", registeredTaskTypes) |
|||
} |
|||
|
|||
// configureTasksFromPolicy dynamically configures all registered tasks based on the maintenance policy
|
|||
func (s *MaintenanceIntegration) configureTasksFromPolicy() { |
|||
if s.maintenancePolicy == nil { |
|||
return |
|||
} |
|||
|
|||
// Configure all registered detectors and schedulers dynamically using policy configuration
|
|||
configuredCount := 0 |
|||
|
|||
// Get all registered task types from the registry
|
|||
for taskType, detector := range s.taskRegistry.GetAllDetectors() { |
|||
// Configure detector using policy-based configuration
|
|||
s.configureDetectorFromPolicy(taskType, detector) |
|||
configuredCount++ |
|||
} |
|||
|
|||
for taskType, scheduler := range s.taskRegistry.GetAllSchedulers() { |
|||
// Configure scheduler using policy-based configuration
|
|||
s.configureSchedulerFromPolicy(taskType, scheduler) |
|||
} |
|||
|
|||
glog.V(1).Infof("Dynamically configured %d task types from maintenance policy", configuredCount) |
|||
} |
|||
|
|||
// configureDetectorFromPolicy configures a detector using policy-based configuration
|
|||
func (s *MaintenanceIntegration) configureDetectorFromPolicy(taskType types.TaskType, detector types.TaskDetector) { |
|||
// Try to configure using PolicyConfigurableDetector interface if supported
|
|||
if configurableDetector, ok := detector.(types.PolicyConfigurableDetector); ok { |
|||
configurableDetector.ConfigureFromPolicy(s.maintenancePolicy) |
|||
glog.V(2).Infof("Configured detector %s using policy interface", taskType) |
|||
return |
|||
} |
|||
|
|||
// Apply basic configuration that all detectors should support
|
|||
if basicDetector, ok := detector.(interface{ SetEnabled(bool) }); ok { |
|||
// Convert task system type to maintenance task type for policy lookup
|
|||
maintenanceTaskType, exists := s.taskTypeMap[taskType] |
|||
if exists { |
|||
enabled := s.maintenancePolicy.IsTaskEnabled(maintenanceTaskType) |
|||
basicDetector.SetEnabled(enabled) |
|||
glog.V(3).Infof("Set enabled=%v for detector %s", enabled, taskType) |
|||
} |
|||
} |
|||
|
|||
// For detectors that don't implement PolicyConfigurableDetector interface,
|
|||
// they should be updated to implement it for full policy-based configuration
|
|||
glog.V(2).Infof("Detector %s should implement PolicyConfigurableDetector interface for full policy support", taskType) |
|||
} |
|||
|
|||
// configureSchedulerFromPolicy configures a scheduler using policy-based configuration
|
|||
func (s *MaintenanceIntegration) configureSchedulerFromPolicy(taskType types.TaskType, scheduler types.TaskScheduler) { |
|||
// Try to configure using PolicyConfigurableScheduler interface if supported
|
|||
if configurableScheduler, ok := scheduler.(types.PolicyConfigurableScheduler); ok { |
|||
configurableScheduler.ConfigureFromPolicy(s.maintenancePolicy) |
|||
glog.V(2).Infof("Configured scheduler %s using policy interface", taskType) |
|||
return |
|||
} |
|||
|
|||
// Apply basic configuration that all schedulers should support
|
|||
maintenanceTaskType, exists := s.taskTypeMap[taskType] |
|||
if !exists { |
|||
glog.V(3).Infof("No maintenance task type mapping for %s, skipping configuration", taskType) |
|||
return |
|||
} |
|||
|
|||
// Set enabled status if scheduler supports it
|
|||
if enableableScheduler, ok := scheduler.(interface{ SetEnabled(bool) }); ok { |
|||
enabled := s.maintenancePolicy.IsTaskEnabled(maintenanceTaskType) |
|||
enableableScheduler.SetEnabled(enabled) |
|||
glog.V(3).Infof("Set enabled=%v for scheduler %s", enabled, taskType) |
|||
} |
|||
|
|||
// Set max concurrent if scheduler supports it
|
|||
if concurrentScheduler, ok := scheduler.(interface{ SetMaxConcurrent(int) }); ok { |
|||
maxConcurrent := s.maintenancePolicy.GetMaxConcurrent(maintenanceTaskType) |
|||
if maxConcurrent > 0 { |
|||
concurrentScheduler.SetMaxConcurrent(maxConcurrent) |
|||
glog.V(3).Infof("Set max concurrent=%d for scheduler %s", maxConcurrent, taskType) |
|||
} |
|||
} |
|||
|
|||
// For schedulers that don't implement PolicyConfigurableScheduler interface,
|
|||
// they should be updated to implement it for full policy-based configuration
|
|||
glog.V(2).Infof("Scheduler %s should implement PolicyConfigurableScheduler interface for full policy support", taskType) |
|||
} |
|||
|
|||
// ScanWithTaskDetectors performs a scan using the task system
|
|||
func (s *MaintenanceIntegration) ScanWithTaskDetectors(volumeMetrics []*types.VolumeHealthMetrics) ([]*TaskDetectionResult, error) { |
|||
var allResults []*TaskDetectionResult |
|||
|
|||
// Create cluster info
|
|||
clusterInfo := &types.ClusterInfo{ |
|||
TotalVolumes: len(volumeMetrics), |
|||
LastUpdated: time.Now(), |
|||
} |
|||
|
|||
// Run detection for each registered task type
|
|||
for taskType, detector := range s.taskRegistry.GetAllDetectors() { |
|||
if !detector.IsEnabled() { |
|||
continue |
|||
} |
|||
|
|||
glog.V(2).Infof("Running detection for task type: %s", taskType) |
|||
|
|||
results, err := detector.ScanForTasks(volumeMetrics, clusterInfo) |
|||
if err != nil { |
|||
glog.Errorf("Failed to scan for %s tasks: %v", taskType, err) |
|||
continue |
|||
} |
|||
|
|||
// Convert results to existing system format
|
|||
for _, result := range results { |
|||
existingResult := s.convertToExistingFormat(result) |
|||
if existingResult != nil { |
|||
allResults = append(allResults, existingResult) |
|||
} |
|||
} |
|||
|
|||
glog.V(2).Infof("Found %d %s tasks", len(results), taskType) |
|||
} |
|||
|
|||
return allResults, nil |
|||
} |
|||
|
|||
// convertToExistingFormat converts task results to existing system format using dynamic mapping
|
|||
func (s *MaintenanceIntegration) convertToExistingFormat(result *types.TaskDetectionResult) *TaskDetectionResult { |
|||
// Convert types using mapping tables
|
|||
existingType, exists := s.taskTypeMap[result.TaskType] |
|||
if !exists { |
|||
glog.Warningf("Unknown task type %s, skipping conversion", result.TaskType) |
|||
// Return nil to indicate conversion failed - caller should handle this
|
|||
return nil |
|||
} |
|||
|
|||
existingPriority, exists := s.priorityMap[result.Priority] |
|||
if !exists { |
|||
glog.Warningf("Unknown priority %d, defaulting to normal", result.Priority) |
|||
existingPriority = PriorityNormal |
|||
} |
|||
|
|||
return &TaskDetectionResult{ |
|||
TaskType: existingType, |
|||
VolumeID: result.VolumeID, |
|||
Server: result.Server, |
|||
Collection: result.Collection, |
|||
Priority: existingPriority, |
|||
Reason: result.Reason, |
|||
Parameters: result.Parameters, |
|||
ScheduleAt: result.ScheduleAt, |
|||
} |
|||
} |
|||
|
|||
// CanScheduleWithTaskSchedulers determines if a task can be scheduled using task schedulers with dynamic type conversion
|
|||
func (s *MaintenanceIntegration) CanScheduleWithTaskSchedulers(task *MaintenanceTask, runningTasks []*MaintenanceTask, availableWorkers []*MaintenanceWorker) bool { |
|||
// Convert existing types to task types using mapping
|
|||
taskType, exists := s.revTaskTypeMap[task.Type] |
|||
if !exists { |
|||
glog.V(2).Infof("Unknown task type %s for scheduling, falling back to existing logic", task.Type) |
|||
return false // Fallback to existing logic for unknown types
|
|||
} |
|||
|
|||
// Convert task objects
|
|||
taskObject := s.convertTaskToTaskSystem(task) |
|||
if taskObject == nil { |
|||
glog.V(2).Infof("Failed to convert task %s for scheduling", task.ID) |
|||
return false |
|||
} |
|||
|
|||
runningTaskObjects := s.convertTasksToTaskSystem(runningTasks) |
|||
workerObjects := s.convertWorkersToTaskSystem(availableWorkers) |
|||
|
|||
// Get the appropriate scheduler
|
|||
scheduler := s.taskRegistry.GetScheduler(taskType) |
|||
if scheduler == nil { |
|||
glog.V(2).Infof("No scheduler found for task type %s", taskType) |
|||
return false |
|||
} |
|||
|
|||
return scheduler.CanScheduleNow(taskObject, runningTaskObjects, workerObjects) |
|||
} |
|||
|
|||
// convertTaskToTaskSystem converts existing task to task system format using dynamic mapping
|
|||
func (s *MaintenanceIntegration) convertTaskToTaskSystem(task *MaintenanceTask) *types.Task { |
|||
// Convert task type using mapping
|
|||
taskType, exists := s.revTaskTypeMap[task.Type] |
|||
if !exists { |
|||
glog.Errorf("Unknown task type %s in conversion, cannot convert task", task.Type) |
|||
// Return nil to indicate conversion failed
|
|||
return nil |
|||
} |
|||
|
|||
// Convert priority using mapping
|
|||
priority, exists := s.revPriorityMap[task.Priority] |
|||
if !exists { |
|||
glog.Warningf("Unknown priority %d in conversion, defaulting to normal", task.Priority) |
|||
priority = types.TaskPriorityNormal |
|||
} |
|||
|
|||
return &types.Task{ |
|||
ID: task.ID, |
|||
Type: taskType, |
|||
Priority: priority, |
|||
VolumeID: task.VolumeID, |
|||
Server: task.Server, |
|||
Collection: task.Collection, |
|||
Parameters: task.Parameters, |
|||
CreatedAt: task.CreatedAt, |
|||
} |
|||
} |
|||
|
|||
// convertTasksToTaskSystem converts multiple tasks
|
|||
func (s *MaintenanceIntegration) convertTasksToTaskSystem(tasks []*MaintenanceTask) []*types.Task { |
|||
var result []*types.Task |
|||
for _, task := range tasks { |
|||
converted := s.convertTaskToTaskSystem(task) |
|||
if converted != nil { |
|||
result = append(result, converted) |
|||
} |
|||
} |
|||
return result |
|||
} |
|||
|
|||
// convertWorkersToTaskSystem converts workers to task system format using dynamic mapping
|
|||
func (s *MaintenanceIntegration) convertWorkersToTaskSystem(workers []*MaintenanceWorker) []*types.Worker { |
|||
var result []*types.Worker |
|||
for _, worker := range workers { |
|||
capabilities := make([]types.TaskType, 0, len(worker.Capabilities)) |
|||
for _, cap := range worker.Capabilities { |
|||
// Convert capability using mapping
|
|||
taskType, exists := s.revTaskTypeMap[cap] |
|||
if exists { |
|||
capabilities = append(capabilities, taskType) |
|||
} else { |
|||
glog.V(3).Infof("Unknown capability %s for worker %s, skipping", cap, worker.ID) |
|||
} |
|||
} |
|||
|
|||
result = append(result, &types.Worker{ |
|||
ID: worker.ID, |
|||
Address: worker.Address, |
|||
Capabilities: capabilities, |
|||
MaxConcurrent: worker.MaxConcurrent, |
|||
CurrentLoad: worker.CurrentLoad, |
|||
}) |
|||
} |
|||
return result |
|||
} |
|||
|
|||
// GetTaskScheduler returns the scheduler for a task type using dynamic mapping
|
|||
func (s *MaintenanceIntegration) GetTaskScheduler(taskType MaintenanceTaskType) types.TaskScheduler { |
|||
// Convert task type using mapping
|
|||
taskSystemType, exists := s.revTaskTypeMap[taskType] |
|||
if !exists { |
|||
glog.V(3).Infof("Unknown task type %s for scheduler", taskType) |
|||
return nil |
|||
} |
|||
|
|||
return s.taskRegistry.GetScheduler(taskSystemType) |
|||
} |
|||
|
|||
// GetUIProvider returns the UI provider for a task type using dynamic mapping
|
|||
func (s *MaintenanceIntegration) GetUIProvider(taskType MaintenanceTaskType) types.TaskUIProvider { |
|||
// Convert task type using mapping
|
|||
taskSystemType, exists := s.revTaskTypeMap[taskType] |
|||
if !exists { |
|||
glog.V(3).Infof("Unknown task type %s for UI provider", taskType) |
|||
return nil |
|||
} |
|||
|
|||
return s.uiRegistry.GetProvider(taskSystemType) |
|||
} |
|||
|
|||
// GetAllTaskStats returns stats for all registered tasks
|
|||
func (s *MaintenanceIntegration) GetAllTaskStats() []*types.TaskStats { |
|||
var stats []*types.TaskStats |
|||
|
|||
for taskType, detector := range s.taskRegistry.GetAllDetectors() { |
|||
uiProvider := s.uiRegistry.GetProvider(taskType) |
|||
if uiProvider == nil { |
|||
continue |
|||
} |
|||
|
|||
stat := &types.TaskStats{ |
|||
TaskType: taskType, |
|||
DisplayName: uiProvider.GetDisplayName(), |
|||
Enabled: detector.IsEnabled(), |
|||
LastScan: time.Now().Add(-detector.ScanInterval()), |
|||
NextScan: time.Now().Add(detector.ScanInterval()), |
|||
ScanInterval: detector.ScanInterval(), |
|||
MaxConcurrent: s.taskRegistry.GetScheduler(taskType).GetMaxConcurrent(), |
|||
// Would need to get these from actual queue/stats
|
|||
PendingTasks: 0, |
|||
RunningTasks: 0, |
|||
CompletedToday: 0, |
|||
FailedToday: 0, |
|||
} |
|||
|
|||
stats = append(stats, stat) |
|||
} |
|||
|
|||
return stats |
|||
} |
|||
@ -0,0 +1,407 @@ |
|||
package maintenance |
|||
|
|||
import ( |
|||
"fmt" |
|||
"strings" |
|||
"sync" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
) |
|||
|
|||
// MaintenanceManager coordinates the maintenance system
|
|||
type MaintenanceManager struct { |
|||
config *MaintenanceConfig |
|||
scanner *MaintenanceScanner |
|||
queue *MaintenanceQueue |
|||
adminClient AdminClient |
|||
running bool |
|||
stopChan chan struct{} |
|||
// Error handling and backoff
|
|||
errorCount int |
|||
lastError error |
|||
lastErrorTime time.Time |
|||
backoffDelay time.Duration |
|||
mutex sync.RWMutex |
|||
} |
|||
|
|||
// NewMaintenanceManager creates a new maintenance manager
|
|||
func NewMaintenanceManager(adminClient AdminClient, config *MaintenanceConfig) *MaintenanceManager { |
|||
if config == nil { |
|||
config = DefaultMaintenanceConfig() |
|||
} |
|||
|
|||
queue := NewMaintenanceQueue(config.Policy) |
|||
scanner := NewMaintenanceScanner(adminClient, config.Policy, queue) |
|||
|
|||
return &MaintenanceManager{ |
|||
config: config, |
|||
scanner: scanner, |
|||
queue: queue, |
|||
adminClient: adminClient, |
|||
stopChan: make(chan struct{}), |
|||
backoffDelay: time.Second, // Start with 1 second backoff
|
|||
} |
|||
} |
|||
|
|||
// Start begins the maintenance manager
|
|||
func (mm *MaintenanceManager) Start() error { |
|||
if !mm.config.Enabled { |
|||
glog.V(1).Infof("Maintenance system is disabled") |
|||
return nil |
|||
} |
|||
|
|||
// Validate configuration durations to prevent ticker panics
|
|||
if err := mm.validateConfig(); err != nil { |
|||
return fmt.Errorf("invalid maintenance configuration: %v", err) |
|||
} |
|||
|
|||
mm.running = true |
|||
|
|||
// Start background processes
|
|||
go mm.scanLoop() |
|||
go mm.cleanupLoop() |
|||
|
|||
glog.Infof("Maintenance manager started with scan interval %ds", mm.config.ScanIntervalSeconds) |
|||
return nil |
|||
} |
|||
|
|||
// validateConfig validates the maintenance configuration durations
|
|||
func (mm *MaintenanceManager) validateConfig() error { |
|||
if mm.config.ScanIntervalSeconds <= 0 { |
|||
glog.Warningf("Invalid scan interval %ds, using default 30m", mm.config.ScanIntervalSeconds) |
|||
mm.config.ScanIntervalSeconds = 30 * 60 // 30 minutes in seconds
|
|||
} |
|||
|
|||
if mm.config.CleanupIntervalSeconds <= 0 { |
|||
glog.Warningf("Invalid cleanup interval %ds, using default 24h", mm.config.CleanupIntervalSeconds) |
|||
mm.config.CleanupIntervalSeconds = 24 * 60 * 60 // 24 hours in seconds
|
|||
} |
|||
|
|||
if mm.config.WorkerTimeoutSeconds <= 0 { |
|||
glog.Warningf("Invalid worker timeout %ds, using default 5m", mm.config.WorkerTimeoutSeconds) |
|||
mm.config.WorkerTimeoutSeconds = 5 * 60 // 5 minutes in seconds
|
|||
} |
|||
|
|||
if mm.config.TaskTimeoutSeconds <= 0 { |
|||
glog.Warningf("Invalid task timeout %ds, using default 2h", mm.config.TaskTimeoutSeconds) |
|||
mm.config.TaskTimeoutSeconds = 2 * 60 * 60 // 2 hours in seconds
|
|||
} |
|||
|
|||
if mm.config.RetryDelaySeconds <= 0 { |
|||
glog.Warningf("Invalid retry delay %ds, using default 15m", mm.config.RetryDelaySeconds) |
|||
mm.config.RetryDelaySeconds = 15 * 60 // 15 minutes in seconds
|
|||
} |
|||
|
|||
if mm.config.TaskRetentionSeconds <= 0 { |
|||
glog.Warningf("Invalid task retention %ds, using default 168h", mm.config.TaskRetentionSeconds) |
|||
mm.config.TaskRetentionSeconds = 7 * 24 * 60 * 60 // 7 days in seconds
|
|||
} |
|||
|
|||
return nil |
|||
} |
|||
|
|||
// IsRunning returns whether the maintenance manager is currently running
|
|||
func (mm *MaintenanceManager) IsRunning() bool { |
|||
return mm.running |
|||
} |
|||
|
|||
// Stop terminates the maintenance manager
|
|||
func (mm *MaintenanceManager) Stop() { |
|||
mm.running = false |
|||
close(mm.stopChan) |
|||
glog.Infof("Maintenance manager stopped") |
|||
} |
|||
|
|||
// scanLoop periodically scans for maintenance tasks with adaptive timing
|
|||
func (mm *MaintenanceManager) scanLoop() { |
|||
scanInterval := time.Duration(mm.config.ScanIntervalSeconds) * time.Second |
|||
ticker := time.NewTicker(scanInterval) |
|||
defer ticker.Stop() |
|||
|
|||
for mm.running { |
|||
select { |
|||
case <-mm.stopChan: |
|||
return |
|||
case <-ticker.C: |
|||
glog.V(1).Infof("Performing maintenance scan every %v", scanInterval) |
|||
mm.performScan() |
|||
|
|||
// Adjust ticker interval based on error state
|
|||
mm.mutex.RLock() |
|||
currentInterval := scanInterval |
|||
if mm.errorCount > 0 { |
|||
// Use backoff delay when there are errors
|
|||
currentInterval = mm.backoffDelay |
|||
if currentInterval > scanInterval { |
|||
// Don't make it longer than the configured interval * 10
|
|||
maxInterval := scanInterval * 10 |
|||
if currentInterval > maxInterval { |
|||
currentInterval = maxInterval |
|||
} |
|||
} |
|||
} |
|||
mm.mutex.RUnlock() |
|||
|
|||
// Reset ticker with new interval if needed
|
|||
if currentInterval != scanInterval { |
|||
ticker.Stop() |
|||
ticker = time.NewTicker(currentInterval) |
|||
} |
|||
} |
|||
} |
|||
} |
|||
|
|||
// cleanupLoop periodically cleans up old tasks and stale workers
|
|||
func (mm *MaintenanceManager) cleanupLoop() { |
|||
cleanupInterval := time.Duration(mm.config.CleanupIntervalSeconds) * time.Second |
|||
ticker := time.NewTicker(cleanupInterval) |
|||
defer ticker.Stop() |
|||
|
|||
for mm.running { |
|||
select { |
|||
case <-mm.stopChan: |
|||
return |
|||
case <-ticker.C: |
|||
mm.performCleanup() |
|||
} |
|||
} |
|||
} |
|||
|
|||
// performScan executes a maintenance scan with error handling and backoff
|
|||
func (mm *MaintenanceManager) performScan() { |
|||
mm.mutex.Lock() |
|||
defer mm.mutex.Unlock() |
|||
|
|||
glog.V(2).Infof("Starting maintenance scan") |
|||
|
|||
results, err := mm.scanner.ScanForMaintenanceTasks() |
|||
if err != nil { |
|||
mm.handleScanError(err) |
|||
return |
|||
} |
|||
|
|||
// Scan succeeded, reset error tracking
|
|||
mm.resetErrorTracking() |
|||
|
|||
if len(results) > 0 { |
|||
mm.queue.AddTasksFromResults(results) |
|||
glog.V(1).Infof("Maintenance scan completed: added %d tasks", len(results)) |
|||
} else { |
|||
glog.V(2).Infof("Maintenance scan completed: no tasks needed") |
|||
} |
|||
} |
|||
|
|||
// handleScanError handles scan errors with exponential backoff and reduced logging
|
|||
func (mm *MaintenanceManager) handleScanError(err error) { |
|||
now := time.Now() |
|||
mm.errorCount++ |
|||
mm.lastError = err |
|||
mm.lastErrorTime = now |
|||
|
|||
// Use exponential backoff with jitter
|
|||
if mm.errorCount > 1 { |
|||
mm.backoffDelay = mm.backoffDelay * 2 |
|||
if mm.backoffDelay > 5*time.Minute { |
|||
mm.backoffDelay = 5 * time.Minute // Cap at 5 minutes
|
|||
} |
|||
} |
|||
|
|||
// Reduce log frequency based on error count and time
|
|||
shouldLog := false |
|||
if mm.errorCount <= 3 { |
|||
// Log first 3 errors immediately
|
|||
shouldLog = true |
|||
} else if mm.errorCount <= 10 && mm.errorCount%3 == 0 { |
|||
// Log every 3rd error for errors 4-10
|
|||
shouldLog = true |
|||
} else if mm.errorCount%10 == 0 { |
|||
// Log every 10th error after that
|
|||
shouldLog = true |
|||
} |
|||
|
|||
if shouldLog { |
|||
// Check if it's a connection error to provide better messaging
|
|||
if isConnectionError(err) { |
|||
if mm.errorCount == 1 { |
|||
glog.Errorf("Maintenance scan failed: %v (will retry with backoff)", err) |
|||
} else { |
|||
glog.Errorf("Maintenance scan still failing after %d attempts: %v (backoff: %v)", |
|||
mm.errorCount, err, mm.backoffDelay) |
|||
} |
|||
} else { |
|||
glog.Errorf("Maintenance scan failed: %v", err) |
|||
} |
|||
} else { |
|||
// Use debug level for suppressed errors
|
|||
glog.V(3).Infof("Maintenance scan failed (error #%d, suppressed): %v", mm.errorCount, err) |
|||
} |
|||
} |
|||
|
|||
// resetErrorTracking resets error tracking when scan succeeds
|
|||
func (mm *MaintenanceManager) resetErrorTracking() { |
|||
if mm.errorCount > 0 { |
|||
glog.V(1).Infof("Maintenance scan recovered after %d failed attempts", mm.errorCount) |
|||
mm.errorCount = 0 |
|||
mm.lastError = nil |
|||
mm.backoffDelay = time.Second // Reset to initial delay
|
|||
} |
|||
} |
|||
|
|||
// isConnectionError checks if the error is a connection-related error
|
|||
func isConnectionError(err error) bool { |
|||
if err == nil { |
|||
return false |
|||
} |
|||
errStr := err.Error() |
|||
return strings.Contains(errStr, "connection refused") || |
|||
strings.Contains(errStr, "connection error") || |
|||
strings.Contains(errStr, "dial tcp") || |
|||
strings.Contains(errStr, "connection timeout") || |
|||
strings.Contains(errStr, "no route to host") || |
|||
strings.Contains(errStr, "network unreachable") |
|||
} |
|||
|
|||
// performCleanup cleans up old tasks and stale workers
|
|||
func (mm *MaintenanceManager) performCleanup() { |
|||
glog.V(2).Infof("Starting maintenance cleanup") |
|||
|
|||
taskRetention := time.Duration(mm.config.TaskRetentionSeconds) * time.Second |
|||
workerTimeout := time.Duration(mm.config.WorkerTimeoutSeconds) * time.Second |
|||
|
|||
removedTasks := mm.queue.CleanupOldTasks(taskRetention) |
|||
removedWorkers := mm.queue.RemoveStaleWorkers(workerTimeout) |
|||
|
|||
if removedTasks > 0 || removedWorkers > 0 { |
|||
glog.V(1).Infof("Cleanup completed: removed %d old tasks and %d stale workers", removedTasks, removedWorkers) |
|||
} |
|||
} |
|||
|
|||
// GetQueue returns the maintenance queue
|
|||
func (mm *MaintenanceManager) GetQueue() *MaintenanceQueue { |
|||
return mm.queue |
|||
} |
|||
|
|||
// GetConfig returns the maintenance configuration
|
|||
func (mm *MaintenanceManager) GetConfig() *MaintenanceConfig { |
|||
return mm.config |
|||
} |
|||
|
|||
// GetStats returns maintenance statistics
|
|||
func (mm *MaintenanceManager) GetStats() *MaintenanceStats { |
|||
stats := mm.queue.GetStats() |
|||
|
|||
mm.mutex.RLock() |
|||
defer mm.mutex.RUnlock() |
|||
|
|||
stats.LastScanTime = time.Now() // Would need to track this properly
|
|||
|
|||
// Calculate next scan time based on current error state
|
|||
scanInterval := time.Duration(mm.config.ScanIntervalSeconds) * time.Second |
|||
nextScanInterval := scanInterval |
|||
if mm.errorCount > 0 { |
|||
nextScanInterval = mm.backoffDelay |
|||
maxInterval := scanInterval * 10 |
|||
if nextScanInterval > maxInterval { |
|||
nextScanInterval = maxInterval |
|||
} |
|||
} |
|||
stats.NextScanTime = time.Now().Add(nextScanInterval) |
|||
|
|||
return stats |
|||
} |
|||
|
|||
// GetErrorState returns the current error state for monitoring
|
|||
func (mm *MaintenanceManager) GetErrorState() (errorCount int, lastError error, backoffDelay time.Duration) { |
|||
mm.mutex.RLock() |
|||
defer mm.mutex.RUnlock() |
|||
return mm.errorCount, mm.lastError, mm.backoffDelay |
|||
} |
|||
|
|||
// GetTasks returns tasks with filtering
|
|||
func (mm *MaintenanceManager) GetTasks(status MaintenanceTaskStatus, taskType MaintenanceTaskType, limit int) []*MaintenanceTask { |
|||
return mm.queue.GetTasks(status, taskType, limit) |
|||
} |
|||
|
|||
// GetWorkers returns all registered workers
|
|||
func (mm *MaintenanceManager) GetWorkers() []*MaintenanceWorker { |
|||
return mm.queue.GetWorkers() |
|||
} |
|||
|
|||
// TriggerScan manually triggers a maintenance scan
|
|||
func (mm *MaintenanceManager) TriggerScan() error { |
|||
if !mm.running { |
|||
return fmt.Errorf("maintenance manager is not running") |
|||
} |
|||
|
|||
go mm.performScan() |
|||
return nil |
|||
} |
|||
|
|||
// UpdateConfig updates the maintenance configuration
|
|||
func (mm *MaintenanceManager) UpdateConfig(config *MaintenanceConfig) error { |
|||
if config == nil { |
|||
return fmt.Errorf("config cannot be nil") |
|||
} |
|||
|
|||
mm.config = config |
|||
mm.queue.policy = config.Policy |
|||
mm.scanner.policy = config.Policy |
|||
|
|||
glog.V(1).Infof("Maintenance configuration updated") |
|||
return nil |
|||
} |
|||
|
|||
// CancelTask cancels a pending task
|
|||
func (mm *MaintenanceManager) CancelTask(taskID string) error { |
|||
mm.queue.mutex.Lock() |
|||
defer mm.queue.mutex.Unlock() |
|||
|
|||
task, exists := mm.queue.tasks[taskID] |
|||
if !exists { |
|||
return fmt.Errorf("task %s not found", taskID) |
|||
} |
|||
|
|||
if task.Status == TaskStatusPending { |
|||
task.Status = TaskStatusCancelled |
|||
task.CompletedAt = &[]time.Time{time.Now()}[0] |
|||
|
|||
// Remove from pending tasks
|
|||
for i, pendingTask := range mm.queue.pendingTasks { |
|||
if pendingTask.ID == taskID { |
|||
mm.queue.pendingTasks = append(mm.queue.pendingTasks[:i], mm.queue.pendingTasks[i+1:]...) |
|||
break |
|||
} |
|||
} |
|||
|
|||
glog.V(2).Infof("Cancelled task %s", taskID) |
|||
return nil |
|||
} |
|||
|
|||
return fmt.Errorf("task %s cannot be cancelled (status: %s)", taskID, task.Status) |
|||
} |
|||
|
|||
// RegisterWorker registers a new worker
|
|||
func (mm *MaintenanceManager) RegisterWorker(worker *MaintenanceWorker) { |
|||
mm.queue.RegisterWorker(worker) |
|||
} |
|||
|
|||
// GetNextTask returns the next task for a worker
|
|||
func (mm *MaintenanceManager) GetNextTask(workerID string, capabilities []MaintenanceTaskType) *MaintenanceTask { |
|||
return mm.queue.GetNextTask(workerID, capabilities) |
|||
} |
|||
|
|||
// CompleteTask marks a task as completed
|
|||
func (mm *MaintenanceManager) CompleteTask(taskID string, error string) { |
|||
mm.queue.CompleteTask(taskID, error) |
|||
} |
|||
|
|||
// UpdateTaskProgress updates task progress
|
|||
func (mm *MaintenanceManager) UpdateTaskProgress(taskID string, progress float64) { |
|||
mm.queue.UpdateTaskProgress(taskID, progress) |
|||
} |
|||
|
|||
// UpdateWorkerHeartbeat updates worker heartbeat
|
|||
func (mm *MaintenanceManager) UpdateWorkerHeartbeat(workerID string) { |
|||
mm.queue.UpdateWorkerHeartbeat(workerID) |
|||
} |
|||
@ -0,0 +1,140 @@ |
|||
package maintenance |
|||
|
|||
import ( |
|||
"errors" |
|||
"testing" |
|||
"time" |
|||
) |
|||
|
|||
func TestMaintenanceManager_ErrorHandling(t *testing.T) { |
|||
config := DefaultMaintenanceConfig() |
|||
config.ScanIntervalSeconds = 1 // Short interval for testing (1 second)
|
|||
|
|||
manager := NewMaintenanceManager(nil, config) |
|||
|
|||
// Test initial state
|
|||
if manager.errorCount != 0 { |
|||
t.Errorf("Expected initial error count to be 0, got %d", manager.errorCount) |
|||
} |
|||
|
|||
if manager.backoffDelay != time.Second { |
|||
t.Errorf("Expected initial backoff delay to be 1s, got %v", manager.backoffDelay) |
|||
} |
|||
|
|||
// Test error handling
|
|||
err := errors.New("dial tcp [::1]:19333: connect: connection refused") |
|||
manager.handleScanError(err) |
|||
|
|||
if manager.errorCount != 1 { |
|||
t.Errorf("Expected error count to be 1, got %d", manager.errorCount) |
|||
} |
|||
|
|||
if manager.lastError != err { |
|||
t.Errorf("Expected last error to be set") |
|||
} |
|||
|
|||
// Test exponential backoff
|
|||
initialDelay := manager.backoffDelay |
|||
manager.handleScanError(err) |
|||
|
|||
if manager.backoffDelay != initialDelay*2 { |
|||
t.Errorf("Expected backoff delay to double, got %v", manager.backoffDelay) |
|||
} |
|||
|
|||
if manager.errorCount != 2 { |
|||
t.Errorf("Expected error count to be 2, got %d", manager.errorCount) |
|||
} |
|||
|
|||
// Test backoff cap
|
|||
for i := 0; i < 10; i++ { |
|||
manager.handleScanError(err) |
|||
} |
|||
|
|||
if manager.backoffDelay > 5*time.Minute { |
|||
t.Errorf("Expected backoff delay to be capped at 5 minutes, got %v", manager.backoffDelay) |
|||
} |
|||
|
|||
// Test error reset
|
|||
manager.resetErrorTracking() |
|||
|
|||
if manager.errorCount != 0 { |
|||
t.Errorf("Expected error count to be reset to 0, got %d", manager.errorCount) |
|||
} |
|||
|
|||
if manager.backoffDelay != time.Second { |
|||
t.Errorf("Expected backoff delay to be reset to 1s, got %v", manager.backoffDelay) |
|||
} |
|||
|
|||
if manager.lastError != nil { |
|||
t.Errorf("Expected last error to be reset to nil") |
|||
} |
|||
} |
|||
|
|||
func TestIsConnectionError(t *testing.T) { |
|||
tests := []struct { |
|||
err error |
|||
expected bool |
|||
}{ |
|||
{nil, false}, |
|||
{errors.New("connection refused"), true}, |
|||
{errors.New("dial tcp [::1]:19333: connect: connection refused"), true}, |
|||
{errors.New("connection error: desc = \"transport: Error while dialing\""), true}, |
|||
{errors.New("connection timeout"), true}, |
|||
{errors.New("no route to host"), true}, |
|||
{errors.New("network unreachable"), true}, |
|||
{errors.New("some other error"), false}, |
|||
{errors.New("invalid argument"), false}, |
|||
} |
|||
|
|||
for _, test := range tests { |
|||
result := isConnectionError(test.err) |
|||
if result != test.expected { |
|||
t.Errorf("For error %v, expected %v, got %v", test.err, test.expected, result) |
|||
} |
|||
} |
|||
} |
|||
|
|||
func TestMaintenanceManager_GetErrorState(t *testing.T) { |
|||
config := DefaultMaintenanceConfig() |
|||
manager := NewMaintenanceManager(nil, config) |
|||
|
|||
// Test initial state
|
|||
errorCount, lastError, backoffDelay := manager.GetErrorState() |
|||
if errorCount != 0 || lastError != nil || backoffDelay != time.Second { |
|||
t.Errorf("Expected initial state to be clean") |
|||
} |
|||
|
|||
// Add some errors
|
|||
err := errors.New("test error") |
|||
manager.handleScanError(err) |
|||
manager.handleScanError(err) |
|||
|
|||
errorCount, lastError, backoffDelay = manager.GetErrorState() |
|||
if errorCount != 2 || lastError != err || backoffDelay != 2*time.Second { |
|||
t.Errorf("Expected error state to be tracked correctly: count=%d, err=%v, delay=%v", |
|||
errorCount, lastError, backoffDelay) |
|||
} |
|||
} |
|||
|
|||
func TestMaintenanceManager_LogThrottling(t *testing.T) { |
|||
config := DefaultMaintenanceConfig() |
|||
manager := NewMaintenanceManager(nil, config) |
|||
|
|||
// This is a basic test to ensure the error handling doesn't panic
|
|||
// In practice, you'd want to capture log output to verify throttling
|
|||
err := errors.New("test error") |
|||
|
|||
// Generate many errors to test throttling
|
|||
for i := 0; i < 25; i++ { |
|||
manager.handleScanError(err) |
|||
} |
|||
|
|||
// Should not panic and should have capped backoff
|
|||
if manager.backoffDelay > 5*time.Minute { |
|||
t.Errorf("Expected backoff to be capped at 5 minutes") |
|||
} |
|||
|
|||
if manager.errorCount != 25 { |
|||
t.Errorf("Expected error count to be 25, got %d", manager.errorCount) |
|||
} |
|||
} |
|||
@ -0,0 +1,500 @@ |
|||
package maintenance |
|||
|
|||
import ( |
|||
"sort" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
) |
|||
|
|||
// NewMaintenanceQueue creates a new maintenance queue
|
|||
func NewMaintenanceQueue(policy *MaintenancePolicy) *MaintenanceQueue { |
|||
queue := &MaintenanceQueue{ |
|||
tasks: make(map[string]*MaintenanceTask), |
|||
workers: make(map[string]*MaintenanceWorker), |
|||
pendingTasks: make([]*MaintenanceTask, 0), |
|||
policy: policy, |
|||
} |
|||
return queue |
|||
} |
|||
|
|||
// SetIntegration sets the integration reference
|
|||
func (mq *MaintenanceQueue) SetIntegration(integration *MaintenanceIntegration) { |
|||
mq.integration = integration |
|||
glog.V(1).Infof("Maintenance queue configured with integration") |
|||
} |
|||
|
|||
// AddTask adds a new maintenance task to the queue
|
|||
func (mq *MaintenanceQueue) AddTask(task *MaintenanceTask) { |
|||
mq.mutex.Lock() |
|||
defer mq.mutex.Unlock() |
|||
|
|||
task.ID = generateTaskID() |
|||
task.Status = TaskStatusPending |
|||
task.CreatedAt = time.Now() |
|||
task.MaxRetries = 3 // Default retry count
|
|||
|
|||
mq.tasks[task.ID] = task |
|||
mq.pendingTasks = append(mq.pendingTasks, task) |
|||
|
|||
// Sort pending tasks by priority and schedule time
|
|||
sort.Slice(mq.pendingTasks, func(i, j int) bool { |
|||
if mq.pendingTasks[i].Priority != mq.pendingTasks[j].Priority { |
|||
return mq.pendingTasks[i].Priority > mq.pendingTasks[j].Priority |
|||
} |
|||
return mq.pendingTasks[i].ScheduledAt.Before(mq.pendingTasks[j].ScheduledAt) |
|||
}) |
|||
|
|||
glog.V(2).Infof("Added maintenance task %s: %s for volume %d", task.ID, task.Type, task.VolumeID) |
|||
} |
|||
|
|||
// AddTasksFromResults converts detection results to tasks and adds them to the queue
|
|||
func (mq *MaintenanceQueue) AddTasksFromResults(results []*TaskDetectionResult) { |
|||
for _, result := range results { |
|||
task := &MaintenanceTask{ |
|||
Type: result.TaskType, |
|||
Priority: result.Priority, |
|||
VolumeID: result.VolumeID, |
|||
Server: result.Server, |
|||
Collection: result.Collection, |
|||
Parameters: result.Parameters, |
|||
Reason: result.Reason, |
|||
ScheduledAt: result.ScheduleAt, |
|||
} |
|||
mq.AddTask(task) |
|||
} |
|||
} |
|||
|
|||
// GetNextTask returns the next available task for a worker
|
|||
func (mq *MaintenanceQueue) GetNextTask(workerID string, capabilities []MaintenanceTaskType) *MaintenanceTask { |
|||
mq.mutex.Lock() |
|||
defer mq.mutex.Unlock() |
|||
|
|||
worker, exists := mq.workers[workerID] |
|||
if !exists { |
|||
return nil |
|||
} |
|||
|
|||
// Check if worker has capacity
|
|||
if worker.CurrentLoad >= worker.MaxConcurrent { |
|||
return nil |
|||
} |
|||
|
|||
now := time.Now() |
|||
|
|||
// Find the next suitable task
|
|||
for i, task := range mq.pendingTasks { |
|||
// Check if it's time to execute the task
|
|||
if task.ScheduledAt.After(now) { |
|||
continue |
|||
} |
|||
|
|||
// Check if worker can handle this task type
|
|||
if !mq.workerCanHandle(task.Type, capabilities) { |
|||
continue |
|||
} |
|||
|
|||
// Check scheduling logic - use simplified system if available, otherwise fallback
|
|||
if !mq.canScheduleTaskNow(task) { |
|||
continue |
|||
} |
|||
|
|||
// Assign task to worker
|
|||
task.Status = TaskStatusAssigned |
|||
task.WorkerID = workerID |
|||
startTime := now |
|||
task.StartedAt = &startTime |
|||
|
|||
// Remove from pending tasks
|
|||
mq.pendingTasks = append(mq.pendingTasks[:i], mq.pendingTasks[i+1:]...) |
|||
|
|||
// Update worker
|
|||
worker.CurrentTask = task |
|||
worker.CurrentLoad++ |
|||
worker.Status = "busy" |
|||
|
|||
glog.V(2).Infof("Assigned task %s to worker %s", task.ID, workerID) |
|||
return task |
|||
} |
|||
|
|||
return nil |
|||
} |
|||
|
|||
// CompleteTask marks a task as completed
|
|||
func (mq *MaintenanceQueue) CompleteTask(taskID string, error string) { |
|||
mq.mutex.Lock() |
|||
defer mq.mutex.Unlock() |
|||
|
|||
task, exists := mq.tasks[taskID] |
|||
if !exists { |
|||
return |
|||
} |
|||
|
|||
completedTime := time.Now() |
|||
task.CompletedAt = &completedTime |
|||
|
|||
if error != "" { |
|||
task.Status = TaskStatusFailed |
|||
task.Error = error |
|||
|
|||
// Check if task should be retried
|
|||
if task.RetryCount < task.MaxRetries { |
|||
task.RetryCount++ |
|||
task.Status = TaskStatusPending |
|||
task.WorkerID = "" |
|||
task.StartedAt = nil |
|||
task.CompletedAt = nil |
|||
task.Error = "" |
|||
task.ScheduledAt = time.Now().Add(15 * time.Minute) // Retry delay
|
|||
|
|||
mq.pendingTasks = append(mq.pendingTasks, task) |
|||
glog.V(2).Infof("Retrying task %s (attempt %d/%d)", taskID, task.RetryCount, task.MaxRetries) |
|||
} else { |
|||
glog.Errorf("Task %s failed permanently after %d retries: %s", taskID, task.MaxRetries, error) |
|||
} |
|||
} else { |
|||
task.Status = TaskStatusCompleted |
|||
task.Progress = 100 |
|||
glog.V(2).Infof("Task %s completed successfully", taskID) |
|||
} |
|||
|
|||
// Update worker
|
|||
if task.WorkerID != "" { |
|||
if worker, exists := mq.workers[task.WorkerID]; exists { |
|||
worker.CurrentTask = nil |
|||
worker.CurrentLoad-- |
|||
if worker.CurrentLoad == 0 { |
|||
worker.Status = "active" |
|||
} |
|||
} |
|||
} |
|||
} |
|||
|
|||
// UpdateTaskProgress updates the progress of a running task
|
|||
func (mq *MaintenanceQueue) UpdateTaskProgress(taskID string, progress float64) { |
|||
mq.mutex.RLock() |
|||
defer mq.mutex.RUnlock() |
|||
|
|||
if task, exists := mq.tasks[taskID]; exists { |
|||
task.Progress = progress |
|||
task.Status = TaskStatusInProgress |
|||
} |
|||
} |
|||
|
|||
// RegisterWorker registers a new worker
|
|||
func (mq *MaintenanceQueue) RegisterWorker(worker *MaintenanceWorker) { |
|||
mq.mutex.Lock() |
|||
defer mq.mutex.Unlock() |
|||
|
|||
worker.LastHeartbeat = time.Now() |
|||
worker.Status = "active" |
|||
worker.CurrentLoad = 0 |
|||
mq.workers[worker.ID] = worker |
|||
|
|||
glog.V(1).Infof("Registered maintenance worker %s at %s", worker.ID, worker.Address) |
|||
} |
|||
|
|||
// UpdateWorkerHeartbeat updates worker heartbeat
|
|||
func (mq *MaintenanceQueue) UpdateWorkerHeartbeat(workerID string) { |
|||
mq.mutex.Lock() |
|||
defer mq.mutex.Unlock() |
|||
|
|||
if worker, exists := mq.workers[workerID]; exists { |
|||
worker.LastHeartbeat = time.Now() |
|||
} |
|||
} |
|||
|
|||
// GetRunningTaskCount returns the number of running tasks of a specific type
|
|||
func (mq *MaintenanceQueue) GetRunningTaskCount(taskType MaintenanceTaskType) int { |
|||
mq.mutex.RLock() |
|||
defer mq.mutex.RUnlock() |
|||
|
|||
count := 0 |
|||
for _, task := range mq.tasks { |
|||
if task.Type == taskType && (task.Status == TaskStatusAssigned || task.Status == TaskStatusInProgress) { |
|||
count++ |
|||
} |
|||
} |
|||
return count |
|||
} |
|||
|
|||
// WasTaskRecentlyCompleted checks if a similar task was recently completed
|
|||
func (mq *MaintenanceQueue) WasTaskRecentlyCompleted(taskType MaintenanceTaskType, volumeID uint32, server string, now time.Time) bool { |
|||
mq.mutex.RLock() |
|||
defer mq.mutex.RUnlock() |
|||
|
|||
// Get the repeat prevention interval for this task type
|
|||
interval := mq.getRepeatPreventionInterval(taskType) |
|||
cutoff := now.Add(-interval) |
|||
|
|||
for _, task := range mq.tasks { |
|||
if task.Type == taskType && |
|||
task.VolumeID == volumeID && |
|||
task.Server == server && |
|||
task.Status == TaskStatusCompleted && |
|||
task.CompletedAt != nil && |
|||
task.CompletedAt.After(cutoff) { |
|||
return true |
|||
} |
|||
} |
|||
return false |
|||
} |
|||
|
|||
// getRepeatPreventionInterval returns the interval for preventing task repetition
|
|||
func (mq *MaintenanceQueue) getRepeatPreventionInterval(taskType MaintenanceTaskType) time.Duration { |
|||
// First try to get default from task scheduler
|
|||
if mq.integration != nil { |
|||
if scheduler := mq.integration.GetTaskScheduler(taskType); scheduler != nil { |
|||
defaultInterval := scheduler.GetDefaultRepeatInterval() |
|||
if defaultInterval > 0 { |
|||
glog.V(3).Infof("Using task scheduler default repeat interval for %s: %v", taskType, defaultInterval) |
|||
return defaultInterval |
|||
} |
|||
} |
|||
} |
|||
|
|||
// Fallback to policy configuration if no scheduler available or scheduler doesn't provide default
|
|||
if mq.policy != nil { |
|||
repeatIntervalHours := mq.policy.GetRepeatInterval(taskType) |
|||
if repeatIntervalHours > 0 { |
|||
interval := time.Duration(repeatIntervalHours) * time.Hour |
|||
glog.V(3).Infof("Using policy configuration repeat interval for %s: %v", taskType, interval) |
|||
return interval |
|||
} |
|||
} |
|||
|
|||
// Ultimate fallback - but avoid hardcoded values where possible
|
|||
glog.V(2).Infof("No scheduler or policy configuration found for task type %s, using minimal default: 1h", taskType) |
|||
return time.Hour // Minimal safe default
|
|||
} |
|||
|
|||
// GetTasks returns tasks with optional filtering
|
|||
func (mq *MaintenanceQueue) GetTasks(status MaintenanceTaskStatus, taskType MaintenanceTaskType, limit int) []*MaintenanceTask { |
|||
mq.mutex.RLock() |
|||
defer mq.mutex.RUnlock() |
|||
|
|||
var tasks []*MaintenanceTask |
|||
for _, task := range mq.tasks { |
|||
if status != "" && task.Status != status { |
|||
continue |
|||
} |
|||
if taskType != "" && task.Type != taskType { |
|||
continue |
|||
} |
|||
tasks = append(tasks, task) |
|||
if limit > 0 && len(tasks) >= limit { |
|||
break |
|||
} |
|||
} |
|||
|
|||
// Sort by creation time (newest first)
|
|||
sort.Slice(tasks, func(i, j int) bool { |
|||
return tasks[i].CreatedAt.After(tasks[j].CreatedAt) |
|||
}) |
|||
|
|||
return tasks |
|||
} |
|||
|
|||
// GetWorkers returns all registered workers
|
|||
func (mq *MaintenanceQueue) GetWorkers() []*MaintenanceWorker { |
|||
mq.mutex.RLock() |
|||
defer mq.mutex.RUnlock() |
|||
|
|||
var workers []*MaintenanceWorker |
|||
for _, worker := range mq.workers { |
|||
workers = append(workers, worker) |
|||
} |
|||
return workers |
|||
} |
|||
|
|||
// generateTaskID generates a unique ID for tasks
|
|||
func generateTaskID() string { |
|||
const charset = "abcdefghijklmnopqrstuvwxyz0123456789" |
|||
b := make([]byte, 8) |
|||
for i := range b { |
|||
b[i] = charset[i%len(charset)] |
|||
} |
|||
return string(b) |
|||
} |
|||
|
|||
// CleanupOldTasks removes old completed and failed tasks
|
|||
func (mq *MaintenanceQueue) CleanupOldTasks(retention time.Duration) int { |
|||
mq.mutex.Lock() |
|||
defer mq.mutex.Unlock() |
|||
|
|||
cutoff := time.Now().Add(-retention) |
|||
removed := 0 |
|||
|
|||
for id, task := range mq.tasks { |
|||
if (task.Status == TaskStatusCompleted || task.Status == TaskStatusFailed) && |
|||
task.CompletedAt != nil && |
|||
task.CompletedAt.Before(cutoff) { |
|||
delete(mq.tasks, id) |
|||
removed++ |
|||
} |
|||
} |
|||
|
|||
glog.V(2).Infof("Cleaned up %d old maintenance tasks", removed) |
|||
return removed |
|||
} |
|||
|
|||
// RemoveStaleWorkers removes workers that haven't sent heartbeat recently
|
|||
func (mq *MaintenanceQueue) RemoveStaleWorkers(timeout time.Duration) int { |
|||
mq.mutex.Lock() |
|||
defer mq.mutex.Unlock() |
|||
|
|||
cutoff := time.Now().Add(-timeout) |
|||
removed := 0 |
|||
|
|||
for id, worker := range mq.workers { |
|||
if worker.LastHeartbeat.Before(cutoff) { |
|||
// Mark any assigned tasks as failed
|
|||
for _, task := range mq.tasks { |
|||
if task.WorkerID == id && (task.Status == TaskStatusAssigned || task.Status == TaskStatusInProgress) { |
|||
task.Status = TaskStatusFailed |
|||
task.Error = "Worker became unavailable" |
|||
completedTime := time.Now() |
|||
task.CompletedAt = &completedTime |
|||
} |
|||
} |
|||
|
|||
delete(mq.workers, id) |
|||
removed++ |
|||
glog.Warningf("Removed stale maintenance worker %s", id) |
|||
} |
|||
} |
|||
|
|||
return removed |
|||
} |
|||
|
|||
// GetStats returns maintenance statistics
|
|||
func (mq *MaintenanceQueue) GetStats() *MaintenanceStats { |
|||
mq.mutex.RLock() |
|||
defer mq.mutex.RUnlock() |
|||
|
|||
stats := &MaintenanceStats{ |
|||
TotalTasks: len(mq.tasks), |
|||
TasksByStatus: make(map[MaintenanceTaskStatus]int), |
|||
TasksByType: make(map[MaintenanceTaskType]int), |
|||
ActiveWorkers: 0, |
|||
} |
|||
|
|||
today := time.Now().Truncate(24 * time.Hour) |
|||
var totalDuration time.Duration |
|||
var completedTasks int |
|||
|
|||
for _, task := range mq.tasks { |
|||
stats.TasksByStatus[task.Status]++ |
|||
stats.TasksByType[task.Type]++ |
|||
|
|||
if task.CompletedAt != nil && task.CompletedAt.After(today) { |
|||
if task.Status == TaskStatusCompleted { |
|||
stats.CompletedToday++ |
|||
} else if task.Status == TaskStatusFailed { |
|||
stats.FailedToday++ |
|||
} |
|||
|
|||
if task.StartedAt != nil { |
|||
duration := task.CompletedAt.Sub(*task.StartedAt) |
|||
totalDuration += duration |
|||
completedTasks++ |
|||
} |
|||
} |
|||
} |
|||
|
|||
for _, worker := range mq.workers { |
|||
if worker.Status == "active" || worker.Status == "busy" { |
|||
stats.ActiveWorkers++ |
|||
} |
|||
} |
|||
|
|||
if completedTasks > 0 { |
|||
stats.AverageTaskTime = totalDuration / time.Duration(completedTasks) |
|||
} |
|||
|
|||
return stats |
|||
} |
|||
|
|||
// workerCanHandle checks if a worker can handle a specific task type
|
|||
func (mq *MaintenanceQueue) workerCanHandle(taskType MaintenanceTaskType, capabilities []MaintenanceTaskType) bool { |
|||
for _, capability := range capabilities { |
|||
if capability == taskType { |
|||
return true |
|||
} |
|||
} |
|||
return false |
|||
} |
|||
|
|||
// canScheduleTaskNow determines if a task can be scheduled using task schedulers or fallback logic
|
|||
func (mq *MaintenanceQueue) canScheduleTaskNow(task *MaintenanceTask) bool { |
|||
// Try task scheduling logic first
|
|||
if mq.integration != nil { |
|||
// Get all running tasks and available workers
|
|||
runningTasks := mq.getRunningTasks() |
|||
availableWorkers := mq.getAvailableWorkers() |
|||
|
|||
canSchedule := mq.integration.CanScheduleWithTaskSchedulers(task, runningTasks, availableWorkers) |
|||
glog.V(3).Infof("Task scheduler decision for task %s (%s): %v", task.ID, task.Type, canSchedule) |
|||
return canSchedule |
|||
} |
|||
|
|||
// Fallback to hardcoded logic
|
|||
return mq.canExecuteTaskType(task.Type) |
|||
} |
|||
|
|||
// canExecuteTaskType checks if we can execute more tasks of this type (concurrency limits) - fallback logic
|
|||
func (mq *MaintenanceQueue) canExecuteTaskType(taskType MaintenanceTaskType) bool { |
|||
runningCount := mq.GetRunningTaskCount(taskType) |
|||
maxConcurrent := mq.getMaxConcurrentForTaskType(taskType) |
|||
|
|||
return runningCount < maxConcurrent |
|||
} |
|||
|
|||
// getMaxConcurrentForTaskType returns the maximum concurrent tasks allowed for a task type
|
|||
func (mq *MaintenanceQueue) getMaxConcurrentForTaskType(taskType MaintenanceTaskType) int { |
|||
// First try to get default from task scheduler
|
|||
if mq.integration != nil { |
|||
if scheduler := mq.integration.GetTaskScheduler(taskType); scheduler != nil { |
|||
maxConcurrent := scheduler.GetMaxConcurrent() |
|||
if maxConcurrent > 0 { |
|||
glog.V(3).Infof("Using task scheduler max concurrent for %s: %d", taskType, maxConcurrent) |
|||
return maxConcurrent |
|||
} |
|||
} |
|||
} |
|||
|
|||
// Fallback to policy configuration if no scheduler available or scheduler doesn't provide default
|
|||
if mq.policy != nil { |
|||
maxConcurrent := mq.policy.GetMaxConcurrent(taskType) |
|||
if maxConcurrent > 0 { |
|||
glog.V(3).Infof("Using policy configuration max concurrent for %s: %d", taskType, maxConcurrent) |
|||
return maxConcurrent |
|||
} |
|||
} |
|||
|
|||
// Ultimate fallback - minimal safe default
|
|||
glog.V(2).Infof("No scheduler or policy configuration found for task type %s, using minimal default: 1", taskType) |
|||
return 1 |
|||
} |
|||
|
|||
// getRunningTasks returns all currently running tasks
|
|||
func (mq *MaintenanceQueue) getRunningTasks() []*MaintenanceTask { |
|||
var runningTasks []*MaintenanceTask |
|||
for _, task := range mq.tasks { |
|||
if task.Status == TaskStatusAssigned || task.Status == TaskStatusInProgress { |
|||
runningTasks = append(runningTasks, task) |
|||
} |
|||
} |
|||
return runningTasks |
|||
} |
|||
|
|||
// getAvailableWorkers returns all workers that can take more work
|
|||
func (mq *MaintenanceQueue) getAvailableWorkers() []*MaintenanceWorker { |
|||
var availableWorkers []*MaintenanceWorker |
|||
for _, worker := range mq.workers { |
|||
if worker.Status == "active" && worker.CurrentLoad < worker.MaxConcurrent { |
|||
availableWorkers = append(availableWorkers, worker) |
|||
} |
|||
} |
|||
return availableWorkers |
|||
} |
|||
@ -0,0 +1,163 @@ |
|||
package maintenance |
|||
|
|||
import ( |
|||
"context" |
|||
"fmt" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// NewMaintenanceScanner creates a new maintenance scanner
|
|||
func NewMaintenanceScanner(adminClient AdminClient, policy *MaintenancePolicy, queue *MaintenanceQueue) *MaintenanceScanner { |
|||
scanner := &MaintenanceScanner{ |
|||
adminClient: adminClient, |
|||
policy: policy, |
|||
queue: queue, |
|||
lastScan: make(map[MaintenanceTaskType]time.Time), |
|||
} |
|||
|
|||
// Initialize integration
|
|||
scanner.integration = NewMaintenanceIntegration(queue, policy) |
|||
|
|||
// Set up bidirectional relationship
|
|||
queue.SetIntegration(scanner.integration) |
|||
|
|||
glog.V(1).Infof("Initialized maintenance scanner with task system") |
|||
|
|||
return scanner |
|||
} |
|||
|
|||
// ScanForMaintenanceTasks analyzes the cluster and generates maintenance tasks
|
|||
func (ms *MaintenanceScanner) ScanForMaintenanceTasks() ([]*TaskDetectionResult, error) { |
|||
// Get volume health metrics
|
|||
volumeMetrics, err := ms.getVolumeHealthMetrics() |
|||
if err != nil { |
|||
return nil, fmt.Errorf("failed to get volume health metrics: %v", err) |
|||
} |
|||
|
|||
// Use task system for all task types
|
|||
if ms.integration != nil { |
|||
// Convert metrics to task system format
|
|||
taskMetrics := ms.convertToTaskMetrics(volumeMetrics) |
|||
|
|||
// Use task detection system
|
|||
results, err := ms.integration.ScanWithTaskDetectors(taskMetrics) |
|||
if err != nil { |
|||
glog.Errorf("Task scanning failed: %v", err) |
|||
return nil, err |
|||
} |
|||
|
|||
glog.V(1).Infof("Maintenance scan completed: found %d tasks", len(results)) |
|||
return results, nil |
|||
} |
|||
|
|||
// No integration available
|
|||
glog.Warningf("No integration available, no tasks will be scheduled") |
|||
return []*TaskDetectionResult{}, nil |
|||
} |
|||
|
|||
// getVolumeHealthMetrics collects health information for all volumes
|
|||
func (ms *MaintenanceScanner) getVolumeHealthMetrics() ([]*VolumeHealthMetrics, error) { |
|||
var metrics []*VolumeHealthMetrics |
|||
|
|||
err := ms.adminClient.WithMasterClient(func(client master_pb.SeaweedClient) error { |
|||
resp, err := client.VolumeList(context.Background(), &master_pb.VolumeListRequest{}) |
|||
if err != nil { |
|||
return err |
|||
} |
|||
|
|||
if resp.TopologyInfo == nil { |
|||
return nil |
|||
} |
|||
|
|||
for _, dc := range resp.TopologyInfo.DataCenterInfos { |
|||
for _, rack := range dc.RackInfos { |
|||
for _, node := range rack.DataNodeInfos { |
|||
for _, diskInfo := range node.DiskInfos { |
|||
for _, volInfo := range diskInfo.VolumeInfos { |
|||
metric := &VolumeHealthMetrics{ |
|||
VolumeID: volInfo.Id, |
|||
Server: node.Id, |
|||
Collection: volInfo.Collection, |
|||
Size: volInfo.Size, |
|||
DeletedBytes: volInfo.DeletedByteCount, |
|||
LastModified: time.Unix(int64(volInfo.ModifiedAtSecond), 0), |
|||
IsReadOnly: volInfo.ReadOnly, |
|||
IsECVolume: false, // Will be determined from volume structure
|
|||
ReplicaCount: 1, // Will be counted
|
|||
ExpectedReplicas: int(volInfo.ReplicaPlacement), |
|||
} |
|||
|
|||
// Calculate derived metrics
|
|||
if metric.Size > 0 { |
|||
metric.GarbageRatio = float64(metric.DeletedBytes) / float64(metric.Size) |
|||
// Calculate fullness ratio (would need volume size limit)
|
|||
// metric.FullnessRatio = float64(metric.Size) / float64(volumeSizeLimit)
|
|||
} |
|||
metric.Age = time.Since(metric.LastModified) |
|||
|
|||
metrics = append(metrics, metric) |
|||
} |
|||
} |
|||
} |
|||
} |
|||
} |
|||
|
|||
return nil |
|||
}) |
|||
|
|||
if err != nil { |
|||
return nil, err |
|||
} |
|||
|
|||
// Count actual replicas and identify EC volumes
|
|||
ms.enrichVolumeMetrics(metrics) |
|||
|
|||
return metrics, nil |
|||
} |
|||
|
|||
// enrichVolumeMetrics adds additional information like replica counts
|
|||
func (ms *MaintenanceScanner) enrichVolumeMetrics(metrics []*VolumeHealthMetrics) { |
|||
// Group volumes by ID to count replicas
|
|||
volumeGroups := make(map[uint32][]*VolumeHealthMetrics) |
|||
for _, metric := range metrics { |
|||
volumeGroups[metric.VolumeID] = append(volumeGroups[metric.VolumeID], metric) |
|||
} |
|||
|
|||
// Update replica counts
|
|||
for _, group := range volumeGroups { |
|||
actualReplicas := len(group) |
|||
for _, metric := range group { |
|||
metric.ReplicaCount = actualReplicas |
|||
} |
|||
} |
|||
} |
|||
|
|||
// convertToTaskMetrics converts existing volume metrics to task system format
|
|||
func (ms *MaintenanceScanner) convertToTaskMetrics(metrics []*VolumeHealthMetrics) []*types.VolumeHealthMetrics { |
|||
var simplified []*types.VolumeHealthMetrics |
|||
|
|||
for _, metric := range metrics { |
|||
simplified = append(simplified, &types.VolumeHealthMetrics{ |
|||
VolumeID: metric.VolumeID, |
|||
Server: metric.Server, |
|||
Collection: metric.Collection, |
|||
Size: metric.Size, |
|||
DeletedBytes: metric.DeletedBytes, |
|||
GarbageRatio: metric.GarbageRatio, |
|||
LastModified: metric.LastModified, |
|||
Age: metric.Age, |
|||
ReplicaCount: metric.ReplicaCount, |
|||
ExpectedReplicas: metric.ExpectedReplicas, |
|||
IsReadOnly: metric.IsReadOnly, |
|||
HasRemoteCopy: metric.HasRemoteCopy, |
|||
IsECVolume: metric.IsECVolume, |
|||
FullnessRatio: metric.FullnessRatio, |
|||
}) |
|||
} |
|||
|
|||
return simplified |
|||
} |
|||
@ -0,0 +1,560 @@ |
|||
package maintenance |
|||
|
|||
import ( |
|||
"html/template" |
|||
"sort" |
|||
"sync" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/tasks" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// AdminClient interface defines what the maintenance system needs from the admin server
|
|||
type AdminClient interface { |
|||
WithMasterClient(fn func(client master_pb.SeaweedClient) error) error |
|||
} |
|||
|
|||
// MaintenanceTaskType represents different types of maintenance operations
|
|||
type MaintenanceTaskType string |
|||
|
|||
// GetRegisteredMaintenanceTaskTypes returns all registered task types as MaintenanceTaskType values
|
|||
// sorted alphabetically for consistent menu ordering
|
|||
func GetRegisteredMaintenanceTaskTypes() []MaintenanceTaskType { |
|||
typesRegistry := tasks.GetGlobalTypesRegistry() |
|||
var taskTypes []MaintenanceTaskType |
|||
|
|||
for workerTaskType := range typesRegistry.GetAllDetectors() { |
|||
maintenanceTaskType := MaintenanceTaskType(string(workerTaskType)) |
|||
taskTypes = append(taskTypes, maintenanceTaskType) |
|||
} |
|||
|
|||
// Sort task types alphabetically to ensure consistent menu ordering
|
|||
sort.Slice(taskTypes, func(i, j int) bool { |
|||
return string(taskTypes[i]) < string(taskTypes[j]) |
|||
}) |
|||
|
|||
return taskTypes |
|||
} |
|||
|
|||
// GetMaintenanceTaskType returns a specific task type if it's registered, or empty string if not found
|
|||
func GetMaintenanceTaskType(taskTypeName string) MaintenanceTaskType { |
|||
typesRegistry := tasks.GetGlobalTypesRegistry() |
|||
|
|||
for workerTaskType := range typesRegistry.GetAllDetectors() { |
|||
if string(workerTaskType) == taskTypeName { |
|||
return MaintenanceTaskType(taskTypeName) |
|||
} |
|||
} |
|||
|
|||
return MaintenanceTaskType("") |
|||
} |
|||
|
|||
// IsMaintenanceTaskTypeRegistered checks if a task type is registered
|
|||
func IsMaintenanceTaskTypeRegistered(taskType MaintenanceTaskType) bool { |
|||
typesRegistry := tasks.GetGlobalTypesRegistry() |
|||
|
|||
for workerTaskType := range typesRegistry.GetAllDetectors() { |
|||
if string(workerTaskType) == string(taskType) { |
|||
return true |
|||
} |
|||
} |
|||
|
|||
return false |
|||
} |
|||
|
|||
// MaintenanceTaskPriority represents task execution priority
|
|||
type MaintenanceTaskPriority int |
|||
|
|||
const ( |
|||
PriorityLow MaintenanceTaskPriority = iota |
|||
PriorityNormal |
|||
PriorityHigh |
|||
PriorityCritical |
|||
) |
|||
|
|||
// MaintenanceTaskStatus represents the current status of a task
|
|||
type MaintenanceTaskStatus string |
|||
|
|||
const ( |
|||
TaskStatusPending MaintenanceTaskStatus = "pending" |
|||
TaskStatusAssigned MaintenanceTaskStatus = "assigned" |
|||
TaskStatusInProgress MaintenanceTaskStatus = "in_progress" |
|||
TaskStatusCompleted MaintenanceTaskStatus = "completed" |
|||
TaskStatusFailed MaintenanceTaskStatus = "failed" |
|||
TaskStatusCancelled MaintenanceTaskStatus = "cancelled" |
|||
) |
|||
|
|||
// MaintenanceTask represents a single maintenance operation
|
|||
type MaintenanceTask struct { |
|||
ID string `json:"id"` |
|||
Type MaintenanceTaskType `json:"type"` |
|||
Priority MaintenanceTaskPriority `json:"priority"` |
|||
Status MaintenanceTaskStatus `json:"status"` |
|||
VolumeID uint32 `json:"volume_id,omitempty"` |
|||
Server string `json:"server,omitempty"` |
|||
Collection string `json:"collection,omitempty"` |
|||
Parameters map[string]interface{} `json:"parameters,omitempty"` |
|||
Reason string `json:"reason"` |
|||
CreatedAt time.Time `json:"created_at"` |
|||
ScheduledAt time.Time `json:"scheduled_at"` |
|||
StartedAt *time.Time `json:"started_at,omitempty"` |
|||
CompletedAt *time.Time `json:"completed_at,omitempty"` |
|||
WorkerID string `json:"worker_id,omitempty"` |
|||
Error string `json:"error,omitempty"` |
|||
Progress float64 `json:"progress"` // 0-100
|
|||
RetryCount int `json:"retry_count"` |
|||
MaxRetries int `json:"max_retries"` |
|||
} |
|||
|
|||
// TaskPolicy represents configuration for a specific task type
|
|||
type TaskPolicy struct { |
|||
Enabled bool `json:"enabled"` |
|||
MaxConcurrent int `json:"max_concurrent"` |
|||
RepeatInterval int `json:"repeat_interval"` // Hours to wait before repeating
|
|||
CheckInterval int `json:"check_interval"` // Hours between checks
|
|||
Configuration map[string]interface{} `json:"configuration"` // Task-specific config
|
|||
} |
|||
|
|||
// MaintenancePolicy defines policies for maintenance operations using a dynamic structure
|
|||
type MaintenancePolicy struct { |
|||
// Task-specific policies mapped by task type
|
|||
TaskPolicies map[MaintenanceTaskType]*TaskPolicy `json:"task_policies"` |
|||
|
|||
// Global policy settings
|
|||
GlobalMaxConcurrent int `json:"global_max_concurrent"` // Overall limit across all task types
|
|||
DefaultRepeatInterval int `json:"default_repeat_interval"` // Default hours if task doesn't specify
|
|||
DefaultCheckInterval int `json:"default_check_interval"` // Default hours for periodic checks
|
|||
} |
|||
|
|||
// GetTaskPolicy returns the policy for a specific task type, creating generic defaults if needed
|
|||
func (mp *MaintenancePolicy) GetTaskPolicy(taskType MaintenanceTaskType) *TaskPolicy { |
|||
if mp.TaskPolicies == nil { |
|||
mp.TaskPolicies = make(map[MaintenanceTaskType]*TaskPolicy) |
|||
} |
|||
|
|||
policy, exists := mp.TaskPolicies[taskType] |
|||
if !exists { |
|||
// Create generic default policy using global settings - no hardcoded fallbacks
|
|||
policy = &TaskPolicy{ |
|||
Enabled: false, // Conservative default - require explicit enabling
|
|||
MaxConcurrent: 1, // Conservative default concurrency
|
|||
RepeatInterval: mp.DefaultRepeatInterval, // Use configured default, 0 if not set
|
|||
CheckInterval: mp.DefaultCheckInterval, // Use configured default, 0 if not set
|
|||
Configuration: make(map[string]interface{}), |
|||
} |
|||
mp.TaskPolicies[taskType] = policy |
|||
} |
|||
|
|||
return policy |
|||
} |
|||
|
|||
// SetTaskPolicy sets the policy for a specific task type
|
|||
func (mp *MaintenancePolicy) SetTaskPolicy(taskType MaintenanceTaskType, policy *TaskPolicy) { |
|||
if mp.TaskPolicies == nil { |
|||
mp.TaskPolicies = make(map[MaintenanceTaskType]*TaskPolicy) |
|||
} |
|||
mp.TaskPolicies[taskType] = policy |
|||
} |
|||
|
|||
// IsTaskEnabled returns whether a task type is enabled
|
|||
func (mp *MaintenancePolicy) IsTaskEnabled(taskType MaintenanceTaskType) bool { |
|||
policy := mp.GetTaskPolicy(taskType) |
|||
return policy.Enabled |
|||
} |
|||
|
|||
// GetMaxConcurrent returns the max concurrent limit for a task type
|
|||
func (mp *MaintenancePolicy) GetMaxConcurrent(taskType MaintenanceTaskType) int { |
|||
policy := mp.GetTaskPolicy(taskType) |
|||
return policy.MaxConcurrent |
|||
} |
|||
|
|||
// GetRepeatInterval returns the repeat interval for a task type
|
|||
func (mp *MaintenancePolicy) GetRepeatInterval(taskType MaintenanceTaskType) int { |
|||
policy := mp.GetTaskPolicy(taskType) |
|||
return policy.RepeatInterval |
|||
} |
|||
|
|||
// GetTaskConfig returns a configuration value for a task type
|
|||
func (mp *MaintenancePolicy) GetTaskConfig(taskType MaintenanceTaskType, key string) (interface{}, bool) { |
|||
policy := mp.GetTaskPolicy(taskType) |
|||
value, exists := policy.Configuration[key] |
|||
return value, exists |
|||
} |
|||
|
|||
// SetTaskConfig sets a configuration value for a task type
|
|||
func (mp *MaintenancePolicy) SetTaskConfig(taskType MaintenanceTaskType, key string, value interface{}) { |
|||
policy := mp.GetTaskPolicy(taskType) |
|||
if policy.Configuration == nil { |
|||
policy.Configuration = make(map[string]interface{}) |
|||
} |
|||
policy.Configuration[key] = value |
|||
} |
|||
|
|||
// MaintenanceWorker represents a worker instance
|
|||
type MaintenanceWorker struct { |
|||
ID string `json:"id"` |
|||
Address string `json:"address"` |
|||
LastHeartbeat time.Time `json:"last_heartbeat"` |
|||
Status string `json:"status"` // active, inactive, busy
|
|||
CurrentTask *MaintenanceTask `json:"current_task,omitempty"` |
|||
Capabilities []MaintenanceTaskType `json:"capabilities"` |
|||
MaxConcurrent int `json:"max_concurrent"` |
|||
CurrentLoad int `json:"current_load"` |
|||
} |
|||
|
|||
// MaintenanceQueue manages the task queue and worker coordination
|
|||
type MaintenanceQueue struct { |
|||
tasks map[string]*MaintenanceTask |
|||
workers map[string]*MaintenanceWorker |
|||
pendingTasks []*MaintenanceTask |
|||
mutex sync.RWMutex |
|||
policy *MaintenancePolicy |
|||
integration *MaintenanceIntegration |
|||
} |
|||
|
|||
// MaintenanceScanner analyzes the cluster and generates maintenance tasks
|
|||
type MaintenanceScanner struct { |
|||
adminClient AdminClient |
|||
policy *MaintenancePolicy |
|||
queue *MaintenanceQueue |
|||
lastScan map[MaintenanceTaskType]time.Time |
|||
integration *MaintenanceIntegration |
|||
} |
|||
|
|||
// TaskDetectionResult represents the result of scanning for maintenance needs
|
|||
type TaskDetectionResult struct { |
|||
TaskType MaintenanceTaskType `json:"task_type"` |
|||
VolumeID uint32 `json:"volume_id,omitempty"` |
|||
Server string `json:"server,omitempty"` |
|||
Collection string `json:"collection,omitempty"` |
|||
Priority MaintenanceTaskPriority `json:"priority"` |
|||
Reason string `json:"reason"` |
|||
Parameters map[string]interface{} `json:"parameters,omitempty"` |
|||
ScheduleAt time.Time `json:"schedule_at"` |
|||
} |
|||
|
|||
// VolumeHealthMetrics contains health information about a volume
|
|||
type VolumeHealthMetrics struct { |
|||
VolumeID uint32 `json:"volume_id"` |
|||
Server string `json:"server"` |
|||
Collection string `json:"collection"` |
|||
Size uint64 `json:"size"` |
|||
DeletedBytes uint64 `json:"deleted_bytes"` |
|||
GarbageRatio float64 `json:"garbage_ratio"` |
|||
LastModified time.Time `json:"last_modified"` |
|||
Age time.Duration `json:"age"` |
|||
ReplicaCount int `json:"replica_count"` |
|||
ExpectedReplicas int `json:"expected_replicas"` |
|||
IsReadOnly bool `json:"is_read_only"` |
|||
HasRemoteCopy bool `json:"has_remote_copy"` |
|||
IsECVolume bool `json:"is_ec_volume"` |
|||
FullnessRatio float64 `json:"fullness_ratio"` |
|||
} |
|||
|
|||
// MaintenanceStats provides statistics about maintenance operations
|
|||
type MaintenanceStats struct { |
|||
TotalTasks int `json:"total_tasks"` |
|||
TasksByStatus map[MaintenanceTaskStatus]int `json:"tasks_by_status"` |
|||
TasksByType map[MaintenanceTaskType]int `json:"tasks_by_type"` |
|||
ActiveWorkers int `json:"active_workers"` |
|||
CompletedToday int `json:"completed_today"` |
|||
FailedToday int `json:"failed_today"` |
|||
AverageTaskTime time.Duration `json:"average_task_time"` |
|||
LastScanTime time.Time `json:"last_scan_time"` |
|||
NextScanTime time.Time `json:"next_scan_time"` |
|||
} |
|||
|
|||
// MaintenanceConfig holds configuration for the maintenance system
|
|||
type MaintenanceConfig struct { |
|||
Enabled bool `json:"enabled"` |
|||
ScanIntervalSeconds int `json:"scan_interval_seconds"` // How often to scan for maintenance needs (in seconds)
|
|||
WorkerTimeoutSeconds int `json:"worker_timeout_seconds"` // Worker heartbeat timeout (in seconds)
|
|||
TaskTimeoutSeconds int `json:"task_timeout_seconds"` // Individual task timeout (in seconds)
|
|||
RetryDelaySeconds int `json:"retry_delay_seconds"` // Delay between retries (in seconds)
|
|||
MaxRetries int `json:"max_retries"` // Default max retries for tasks
|
|||
CleanupIntervalSeconds int `json:"cleanup_interval_seconds"` // How often to clean up old tasks (in seconds)
|
|||
TaskRetentionSeconds int `json:"task_retention_seconds"` // How long to keep completed/failed tasks (in seconds)
|
|||
Policy *MaintenancePolicy `json:"policy"` |
|||
} |
|||
|
|||
// Default configuration values
|
|||
func DefaultMaintenanceConfig() *MaintenanceConfig { |
|||
return &MaintenanceConfig{ |
|||
Enabled: false, // Disabled by default for safety
|
|||
ScanIntervalSeconds: 30 * 60, // 30 minutes
|
|||
WorkerTimeoutSeconds: 5 * 60, // 5 minutes
|
|||
TaskTimeoutSeconds: 2 * 60 * 60, // 2 hours
|
|||
RetryDelaySeconds: 15 * 60, // 15 minutes
|
|||
MaxRetries: 3, |
|||
CleanupIntervalSeconds: 24 * 60 * 60, // 24 hours
|
|||
TaskRetentionSeconds: 7 * 24 * 60 * 60, // 7 days
|
|||
Policy: &MaintenancePolicy{ |
|||
GlobalMaxConcurrent: 4, |
|||
DefaultRepeatInterval: 6, |
|||
DefaultCheckInterval: 12, |
|||
}, |
|||
} |
|||
} |
|||
|
|||
// MaintenanceQueueData represents data for the queue visualization UI
|
|||
type MaintenanceQueueData struct { |
|||
Tasks []*MaintenanceTask `json:"tasks"` |
|||
Workers []*MaintenanceWorker `json:"workers"` |
|||
Stats *QueueStats `json:"stats"` |
|||
LastUpdated time.Time `json:"last_updated"` |
|||
} |
|||
|
|||
// QueueStats provides statistics for the queue UI
|
|||
type QueueStats struct { |
|||
PendingTasks int `json:"pending_tasks"` |
|||
RunningTasks int `json:"running_tasks"` |
|||
CompletedToday int `json:"completed_today"` |
|||
FailedToday int `json:"failed_today"` |
|||
TotalTasks int `json:"total_tasks"` |
|||
} |
|||
|
|||
// MaintenanceConfigData represents configuration data for the UI
|
|||
type MaintenanceConfigData struct { |
|||
Config *MaintenanceConfig `json:"config"` |
|||
IsEnabled bool `json:"is_enabled"` |
|||
LastScanTime time.Time `json:"last_scan_time"` |
|||
NextScanTime time.Time `json:"next_scan_time"` |
|||
SystemStats *MaintenanceStats `json:"system_stats"` |
|||
MenuItems []*MaintenanceMenuItem `json:"menu_items"` |
|||
} |
|||
|
|||
// MaintenanceMenuItem represents a menu item for task configuration
|
|||
type MaintenanceMenuItem struct { |
|||
TaskType MaintenanceTaskType `json:"task_type"` |
|||
DisplayName string `json:"display_name"` |
|||
Description string `json:"description"` |
|||
Icon string `json:"icon"` |
|||
IsEnabled bool `json:"is_enabled"` |
|||
Path string `json:"path"` |
|||
} |
|||
|
|||
// WorkerDetailsData represents detailed worker information
|
|||
type WorkerDetailsData struct { |
|||
Worker *MaintenanceWorker `json:"worker"` |
|||
CurrentTasks []*MaintenanceTask `json:"current_tasks"` |
|||
RecentTasks []*MaintenanceTask `json:"recent_tasks"` |
|||
Performance *WorkerPerformance `json:"performance"` |
|||
LastUpdated time.Time `json:"last_updated"` |
|||
} |
|||
|
|||
// WorkerPerformance tracks worker performance metrics
|
|||
type WorkerPerformance struct { |
|||
TasksCompleted int `json:"tasks_completed"` |
|||
TasksFailed int `json:"tasks_failed"` |
|||
AverageTaskTime time.Duration `json:"average_task_time"` |
|||
Uptime time.Duration `json:"uptime"` |
|||
SuccessRate float64 `json:"success_rate"` |
|||
} |
|||
|
|||
// TaskConfigData represents data for individual task configuration page
|
|||
type TaskConfigData struct { |
|||
TaskType MaintenanceTaskType `json:"task_type"` |
|||
TaskName string `json:"task_name"` |
|||
TaskIcon string `json:"task_icon"` |
|||
Description string `json:"description"` |
|||
ConfigFormHTML template.HTML `json:"config_form_html"` |
|||
} |
|||
|
|||
// ClusterReplicationTask represents a cluster replication task parameters
|
|||
type ClusterReplicationTask struct { |
|||
SourcePath string `json:"source_path"` |
|||
TargetCluster string `json:"target_cluster"` |
|||
TargetPath string `json:"target_path"` |
|||
ReplicationMode string `json:"replication_mode"` // "sync", "async", "backup"
|
|||
Priority int `json:"priority"` |
|||
Checksum string `json:"checksum,omitempty"` |
|||
FileSize int64 `json:"file_size"` |
|||
CreatedAt time.Time `json:"created_at"` |
|||
Metadata map[string]string `json:"metadata,omitempty"` |
|||
} |
|||
|
|||
// BuildMaintenancePolicyFromTasks creates a maintenance policy with configurations
|
|||
// from all registered tasks using their UI providers
|
|||
func BuildMaintenancePolicyFromTasks() *MaintenancePolicy { |
|||
policy := &MaintenancePolicy{ |
|||
TaskPolicies: make(map[MaintenanceTaskType]*TaskPolicy), |
|||
GlobalMaxConcurrent: 4, |
|||
DefaultRepeatInterval: 6, |
|||
DefaultCheckInterval: 12, |
|||
} |
|||
|
|||
// Get all registered task types from the UI registry
|
|||
uiRegistry := tasks.GetGlobalUIRegistry() |
|||
typesRegistry := tasks.GetGlobalTypesRegistry() |
|||
|
|||
for taskType, provider := range uiRegistry.GetAllProviders() { |
|||
// Convert task type to maintenance task type
|
|||
maintenanceTaskType := MaintenanceTaskType(string(taskType)) |
|||
|
|||
// Get the default configuration from the UI provider
|
|||
defaultConfig := provider.GetCurrentConfig() |
|||
|
|||
// Create task policy from UI configuration
|
|||
taskPolicy := &TaskPolicy{ |
|||
Enabled: true, // Default enabled
|
|||
MaxConcurrent: 2, // Default concurrency
|
|||
RepeatInterval: policy.DefaultRepeatInterval, |
|||
CheckInterval: policy.DefaultCheckInterval, |
|||
Configuration: make(map[string]interface{}), |
|||
} |
|||
|
|||
// Extract configuration from UI provider's config
|
|||
if configMap, ok := defaultConfig.(map[string]interface{}); ok { |
|||
// Copy all configuration values
|
|||
for key, value := range configMap { |
|||
taskPolicy.Configuration[key] = value |
|||
} |
|||
|
|||
// Extract common fields
|
|||
if enabled, exists := configMap["enabled"]; exists { |
|||
if enabledBool, ok := enabled.(bool); ok { |
|||
taskPolicy.Enabled = enabledBool |
|||
} |
|||
} |
|||
if maxConcurrent, exists := configMap["max_concurrent"]; exists { |
|||
if maxConcurrentInt, ok := maxConcurrent.(int); ok { |
|||
taskPolicy.MaxConcurrent = maxConcurrentInt |
|||
} else if maxConcurrentFloat, ok := maxConcurrent.(float64); ok { |
|||
taskPolicy.MaxConcurrent = int(maxConcurrentFloat) |
|||
} |
|||
} |
|||
} |
|||
|
|||
// Also get defaults from scheduler if available (using types.TaskScheduler explicitly)
|
|||
var scheduler types.TaskScheduler = typesRegistry.GetScheduler(taskType) |
|||
if scheduler != nil { |
|||
if taskPolicy.MaxConcurrent <= 0 { |
|||
taskPolicy.MaxConcurrent = scheduler.GetMaxConcurrent() |
|||
} |
|||
// Convert default repeat interval to hours
|
|||
if repeatInterval := scheduler.GetDefaultRepeatInterval(); repeatInterval > 0 { |
|||
taskPolicy.RepeatInterval = int(repeatInterval.Hours()) |
|||
} |
|||
} |
|||
|
|||
// Also get defaults from detector if available (using types.TaskDetector explicitly)
|
|||
var detector types.TaskDetector = typesRegistry.GetDetector(taskType) |
|||
if detector != nil { |
|||
// Convert scan interval to check interval (hours)
|
|||
if scanInterval := detector.ScanInterval(); scanInterval > 0 { |
|||
taskPolicy.CheckInterval = int(scanInterval.Hours()) |
|||
} |
|||
} |
|||
|
|||
policy.TaskPolicies[maintenanceTaskType] = taskPolicy |
|||
glog.V(3).Infof("Built policy for task type %s: enabled=%v, max_concurrent=%d", |
|||
maintenanceTaskType, taskPolicy.Enabled, taskPolicy.MaxConcurrent) |
|||
} |
|||
|
|||
glog.V(2).Infof("Built maintenance policy with %d task configurations", len(policy.TaskPolicies)) |
|||
return policy |
|||
} |
|||
|
|||
// SetPolicyFromTasks sets the maintenance policy from registered tasks
|
|||
func SetPolicyFromTasks(policy *MaintenancePolicy) { |
|||
if policy == nil { |
|||
return |
|||
} |
|||
|
|||
// Build new policy from tasks
|
|||
newPolicy := BuildMaintenancePolicyFromTasks() |
|||
|
|||
// Copy task policies
|
|||
policy.TaskPolicies = newPolicy.TaskPolicies |
|||
|
|||
glog.V(1).Infof("Updated maintenance policy with %d task configurations from registered tasks", len(policy.TaskPolicies)) |
|||
} |
|||
|
|||
// GetTaskIcon returns the icon CSS class for a task type from its UI provider
|
|||
func GetTaskIcon(taskType MaintenanceTaskType) string { |
|||
typesRegistry := tasks.GetGlobalTypesRegistry() |
|||
uiRegistry := tasks.GetGlobalUIRegistry() |
|||
|
|||
// Convert MaintenanceTaskType to TaskType
|
|||
for workerTaskType := range typesRegistry.GetAllDetectors() { |
|||
if string(workerTaskType) == string(taskType) { |
|||
// Get the UI provider for this task type
|
|||
provider := uiRegistry.GetProvider(workerTaskType) |
|||
if provider != nil { |
|||
return provider.GetIcon() |
|||
} |
|||
break |
|||
} |
|||
} |
|||
|
|||
// Default icon if no UI provider found
|
|||
return "fas fa-cog text-muted" |
|||
} |
|||
|
|||
// GetTaskDisplayName returns the display name for a task type from its UI provider
|
|||
func GetTaskDisplayName(taskType MaintenanceTaskType) string { |
|||
typesRegistry := tasks.GetGlobalTypesRegistry() |
|||
uiRegistry := tasks.GetGlobalUIRegistry() |
|||
|
|||
// Convert MaintenanceTaskType to TaskType
|
|||
for workerTaskType := range typesRegistry.GetAllDetectors() { |
|||
if string(workerTaskType) == string(taskType) { |
|||
// Get the UI provider for this task type
|
|||
provider := uiRegistry.GetProvider(workerTaskType) |
|||
if provider != nil { |
|||
return provider.GetDisplayName() |
|||
} |
|||
break |
|||
} |
|||
} |
|||
|
|||
// Fallback to the task type string
|
|||
return string(taskType) |
|||
} |
|||
|
|||
// GetTaskDescription returns the description for a task type from its UI provider
|
|||
func GetTaskDescription(taskType MaintenanceTaskType) string { |
|||
typesRegistry := tasks.GetGlobalTypesRegistry() |
|||
uiRegistry := tasks.GetGlobalUIRegistry() |
|||
|
|||
// Convert MaintenanceTaskType to TaskType
|
|||
for workerTaskType := range typesRegistry.GetAllDetectors() { |
|||
if string(workerTaskType) == string(taskType) { |
|||
// Get the UI provider for this task type
|
|||
provider := uiRegistry.GetProvider(workerTaskType) |
|||
if provider != nil { |
|||
return provider.GetDescription() |
|||
} |
|||
break |
|||
} |
|||
} |
|||
|
|||
// Fallback to a generic description
|
|||
return "Configure detailed settings for " + string(taskType) + " tasks." |
|||
} |
|||
|
|||
// BuildMaintenanceMenuItems creates menu items for all registered task types
|
|||
func BuildMaintenanceMenuItems() []*MaintenanceMenuItem { |
|||
var menuItems []*MaintenanceMenuItem |
|||
|
|||
// Get all registered task types
|
|||
registeredTypes := GetRegisteredMaintenanceTaskTypes() |
|||
|
|||
for _, taskType := range registeredTypes { |
|||
menuItem := &MaintenanceMenuItem{ |
|||
TaskType: taskType, |
|||
DisplayName: GetTaskDisplayName(taskType), |
|||
Description: GetTaskDescription(taskType), |
|||
Icon: GetTaskIcon(taskType), |
|||
IsEnabled: IsMaintenanceTaskTypeRegistered(taskType), |
|||
Path: "/maintenance/config/" + string(taskType), |
|||
} |
|||
|
|||
menuItems = append(menuItems, menuItem) |
|||
} |
|||
|
|||
return menuItems |
|||
} |
|||
@ -0,0 +1,413 @@ |
|||
package maintenance |
|||
|
|||
import ( |
|||
"fmt" |
|||
"os" |
|||
"sync" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/tasks" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
|
|||
// Import task packages to trigger their auto-registration
|
|||
_ "github.com/seaweedfs/seaweedfs/weed/worker/tasks/balance" |
|||
_ "github.com/seaweedfs/seaweedfs/weed/worker/tasks/erasure_coding" |
|||
_ "github.com/seaweedfs/seaweedfs/weed/worker/tasks/vacuum" |
|||
) |
|||
|
|||
// MaintenanceWorkerService manages maintenance task execution
|
|||
// TaskExecutor defines the function signature for task execution
|
|||
type TaskExecutor func(*MaintenanceWorkerService, *MaintenanceTask) error |
|||
|
|||
// TaskExecutorFactory creates a task executor for a given worker service
|
|||
type TaskExecutorFactory func() TaskExecutor |
|||
|
|||
// Global registry for task executor factories
|
|||
var taskExecutorFactories = make(map[MaintenanceTaskType]TaskExecutorFactory) |
|||
var executorRegistryMutex sync.RWMutex |
|||
var executorRegistryInitOnce sync.Once |
|||
|
|||
// initializeExecutorFactories dynamically registers executor factories for all auto-registered task types
|
|||
func initializeExecutorFactories() { |
|||
executorRegistryInitOnce.Do(func() { |
|||
// Get all registered task types from the global registry
|
|||
typesRegistry := tasks.GetGlobalTypesRegistry() |
|||
|
|||
var taskTypes []MaintenanceTaskType |
|||
for workerTaskType := range typesRegistry.GetAllDetectors() { |
|||
// Convert types.TaskType to MaintenanceTaskType by string conversion
|
|||
maintenanceTaskType := MaintenanceTaskType(string(workerTaskType)) |
|||
taskTypes = append(taskTypes, maintenanceTaskType) |
|||
} |
|||
|
|||
// Register generic executor for all task types
|
|||
for _, taskType := range taskTypes { |
|||
RegisterTaskExecutorFactory(taskType, createGenericTaskExecutor) |
|||
} |
|||
|
|||
glog.V(1).Infof("Dynamically registered generic task executor for %d task types: %v", len(taskTypes), taskTypes) |
|||
}) |
|||
} |
|||
|
|||
// RegisterTaskExecutorFactory registers a factory function for creating task executors
|
|||
func RegisterTaskExecutorFactory(taskType MaintenanceTaskType, factory TaskExecutorFactory) { |
|||
executorRegistryMutex.Lock() |
|||
defer executorRegistryMutex.Unlock() |
|||
taskExecutorFactories[taskType] = factory |
|||
glog.V(2).Infof("Registered executor factory for task type: %s", taskType) |
|||
} |
|||
|
|||
// GetTaskExecutorFactory returns the factory for a task type
|
|||
func GetTaskExecutorFactory(taskType MaintenanceTaskType) (TaskExecutorFactory, bool) { |
|||
// Ensure executor factories are initialized
|
|||
initializeExecutorFactories() |
|||
|
|||
executorRegistryMutex.RLock() |
|||
defer executorRegistryMutex.RUnlock() |
|||
factory, exists := taskExecutorFactories[taskType] |
|||
return factory, exists |
|||
} |
|||
|
|||
// GetSupportedExecutorTaskTypes returns all task types with registered executor factories
|
|||
func GetSupportedExecutorTaskTypes() []MaintenanceTaskType { |
|||
// Ensure executor factories are initialized
|
|||
initializeExecutorFactories() |
|||
|
|||
executorRegistryMutex.RLock() |
|||
defer executorRegistryMutex.RUnlock() |
|||
|
|||
taskTypes := make([]MaintenanceTaskType, 0, len(taskExecutorFactories)) |
|||
for taskType := range taskExecutorFactories { |
|||
taskTypes = append(taskTypes, taskType) |
|||
} |
|||
return taskTypes |
|||
} |
|||
|
|||
// createGenericTaskExecutor creates a generic task executor that uses the task registry
|
|||
func createGenericTaskExecutor() TaskExecutor { |
|||
return func(mws *MaintenanceWorkerService, task *MaintenanceTask) error { |
|||
return mws.executeGenericTask(task) |
|||
} |
|||
} |
|||
|
|||
// init does minimal initialization - actual registration happens lazily
|
|||
func init() { |
|||
// Executor factory registration will happen lazily when first accessed
|
|||
glog.V(1).Infof("Maintenance worker initialized - executor factories will be registered on first access") |
|||
} |
|||
|
|||
type MaintenanceWorkerService struct { |
|||
workerID string |
|||
address string |
|||
adminServer string |
|||
capabilities []MaintenanceTaskType |
|||
maxConcurrent int |
|||
currentTasks map[string]*MaintenanceTask |
|||
queue *MaintenanceQueue |
|||
adminClient AdminClient |
|||
running bool |
|||
stopChan chan struct{} |
|||
|
|||
// Task execution registry
|
|||
taskExecutors map[MaintenanceTaskType]TaskExecutor |
|||
|
|||
// Task registry for creating task instances
|
|||
taskRegistry *tasks.TaskRegistry |
|||
} |
|||
|
|||
// NewMaintenanceWorkerService creates a new maintenance worker service
|
|||
func NewMaintenanceWorkerService(workerID, address, adminServer string) *MaintenanceWorkerService { |
|||
// Get all registered maintenance task types dynamically
|
|||
capabilities := GetRegisteredMaintenanceTaskTypes() |
|||
|
|||
worker := &MaintenanceWorkerService{ |
|||
workerID: workerID, |
|||
address: address, |
|||
adminServer: adminServer, |
|||
capabilities: capabilities, |
|||
maxConcurrent: 2, // Default concurrent task limit
|
|||
currentTasks: make(map[string]*MaintenanceTask), |
|||
stopChan: make(chan struct{}), |
|||
taskExecutors: make(map[MaintenanceTaskType]TaskExecutor), |
|||
taskRegistry: tasks.GetGlobalRegistry(), // Use global registry with auto-registered tasks
|
|||
} |
|||
|
|||
// Initialize task executor registry
|
|||
worker.initializeTaskExecutors() |
|||
|
|||
glog.V(1).Infof("Created maintenance worker with %d registered task types", len(worker.taskRegistry.GetSupportedTypes())) |
|||
|
|||
return worker |
|||
} |
|||
|
|||
// executeGenericTask executes a task using the task registry instead of hardcoded methods
|
|||
func (mws *MaintenanceWorkerService) executeGenericTask(task *MaintenanceTask) error { |
|||
glog.V(2).Infof("Executing generic task %s: %s for volume %d", task.ID, task.Type, task.VolumeID) |
|||
|
|||
// Convert MaintenanceTask to types.TaskType
|
|||
taskType := types.TaskType(string(task.Type)) |
|||
|
|||
// Create task parameters
|
|||
taskParams := types.TaskParams{ |
|||
VolumeID: task.VolumeID, |
|||
Server: task.Server, |
|||
Collection: task.Collection, |
|||
Parameters: task.Parameters, |
|||
} |
|||
|
|||
// Create task instance using the registry
|
|||
taskInstance, err := mws.taskRegistry.CreateTask(taskType, taskParams) |
|||
if err != nil { |
|||
return fmt.Errorf("failed to create task instance: %v", err) |
|||
} |
|||
|
|||
// Update progress to show task has started
|
|||
mws.updateTaskProgress(task.ID, 5) |
|||
|
|||
// Execute the task
|
|||
err = taskInstance.Execute(taskParams) |
|||
if err != nil { |
|||
return fmt.Errorf("task execution failed: %v", err) |
|||
} |
|||
|
|||
// Update progress to show completion
|
|||
mws.updateTaskProgress(task.ID, 100) |
|||
|
|||
glog.V(2).Infof("Generic task %s completed successfully", task.ID) |
|||
return nil |
|||
} |
|||
|
|||
// initializeTaskExecutors sets up the task execution registry dynamically
|
|||
func (mws *MaintenanceWorkerService) initializeTaskExecutors() { |
|||
mws.taskExecutors = make(map[MaintenanceTaskType]TaskExecutor) |
|||
|
|||
// Get all registered executor factories and create executors
|
|||
executorRegistryMutex.RLock() |
|||
defer executorRegistryMutex.RUnlock() |
|||
|
|||
for taskType, factory := range taskExecutorFactories { |
|||
executor := factory() |
|||
mws.taskExecutors[taskType] = executor |
|||
glog.V(3).Infof("Initialized executor for task type: %s", taskType) |
|||
} |
|||
|
|||
glog.V(2).Infof("Initialized %d task executors", len(mws.taskExecutors)) |
|||
} |
|||
|
|||
// RegisterTaskExecutor allows dynamic registration of new task executors
|
|||
func (mws *MaintenanceWorkerService) RegisterTaskExecutor(taskType MaintenanceTaskType, executor TaskExecutor) { |
|||
if mws.taskExecutors == nil { |
|||
mws.taskExecutors = make(map[MaintenanceTaskType]TaskExecutor) |
|||
} |
|||
mws.taskExecutors[taskType] = executor |
|||
glog.V(1).Infof("Registered executor for task type: %s", taskType) |
|||
} |
|||
|
|||
// GetSupportedTaskTypes returns all task types that this worker can execute
|
|||
func (mws *MaintenanceWorkerService) GetSupportedTaskTypes() []MaintenanceTaskType { |
|||
return GetSupportedExecutorTaskTypes() |
|||
} |
|||
|
|||
// Start begins the worker service
|
|||
func (mws *MaintenanceWorkerService) Start() error { |
|||
mws.running = true |
|||
|
|||
// Register with admin server
|
|||
worker := &MaintenanceWorker{ |
|||
ID: mws.workerID, |
|||
Address: mws.address, |
|||
Capabilities: mws.capabilities, |
|||
MaxConcurrent: mws.maxConcurrent, |
|||
} |
|||
|
|||
if mws.queue != nil { |
|||
mws.queue.RegisterWorker(worker) |
|||
} |
|||
|
|||
// Start worker loop
|
|||
go mws.workerLoop() |
|||
|
|||
glog.Infof("Maintenance worker %s started at %s", mws.workerID, mws.address) |
|||
return nil |
|||
} |
|||
|
|||
// Stop terminates the worker service
|
|||
func (mws *MaintenanceWorkerService) Stop() { |
|||
mws.running = false |
|||
close(mws.stopChan) |
|||
|
|||
// Wait for current tasks to complete or timeout
|
|||
timeout := time.NewTimer(30 * time.Second) |
|||
defer timeout.Stop() |
|||
|
|||
for len(mws.currentTasks) > 0 { |
|||
select { |
|||
case <-timeout.C: |
|||
glog.Warningf("Worker %s stopping with %d tasks still running", mws.workerID, len(mws.currentTasks)) |
|||
return |
|||
case <-time.After(time.Second): |
|||
// Check again
|
|||
} |
|||
} |
|||
|
|||
glog.Infof("Maintenance worker %s stopped", mws.workerID) |
|||
} |
|||
|
|||
// workerLoop is the main worker event loop
|
|||
func (mws *MaintenanceWorkerService) workerLoop() { |
|||
heartbeatTicker := time.NewTicker(30 * time.Second) |
|||
defer heartbeatTicker.Stop() |
|||
|
|||
taskRequestTicker := time.NewTicker(5 * time.Second) |
|||
defer taskRequestTicker.Stop() |
|||
|
|||
for mws.running { |
|||
select { |
|||
case <-mws.stopChan: |
|||
return |
|||
case <-heartbeatTicker.C: |
|||
mws.sendHeartbeat() |
|||
case <-taskRequestTicker.C: |
|||
mws.requestTasks() |
|||
} |
|||
} |
|||
} |
|||
|
|||
// sendHeartbeat sends heartbeat to admin server
|
|||
func (mws *MaintenanceWorkerService) sendHeartbeat() { |
|||
if mws.queue != nil { |
|||
mws.queue.UpdateWorkerHeartbeat(mws.workerID) |
|||
} |
|||
} |
|||
|
|||
// requestTasks requests new tasks from the admin server
|
|||
func (mws *MaintenanceWorkerService) requestTasks() { |
|||
if len(mws.currentTasks) >= mws.maxConcurrent { |
|||
return // Already at capacity
|
|||
} |
|||
|
|||
if mws.queue != nil { |
|||
task := mws.queue.GetNextTask(mws.workerID, mws.capabilities) |
|||
if task != nil { |
|||
mws.executeTask(task) |
|||
} |
|||
} |
|||
} |
|||
|
|||
// executeTask executes a maintenance task
|
|||
func (mws *MaintenanceWorkerService) executeTask(task *MaintenanceTask) { |
|||
mws.currentTasks[task.ID] = task |
|||
|
|||
go func() { |
|||
defer func() { |
|||
delete(mws.currentTasks, task.ID) |
|||
}() |
|||
|
|||
glog.Infof("Worker %s executing task %s: %s", mws.workerID, task.ID, task.Type) |
|||
|
|||
// Execute task using dynamic executor registry
|
|||
var err error |
|||
if executor, exists := mws.taskExecutors[task.Type]; exists { |
|||
err = executor(mws, task) |
|||
} else { |
|||
err = fmt.Errorf("unsupported task type: %s", task.Type) |
|||
glog.Errorf("No executor registered for task type: %s", task.Type) |
|||
} |
|||
|
|||
// Report task completion
|
|||
if mws.queue != nil { |
|||
errorMsg := "" |
|||
if err != nil { |
|||
errorMsg = err.Error() |
|||
} |
|||
mws.queue.CompleteTask(task.ID, errorMsg) |
|||
} |
|||
|
|||
if err != nil { |
|||
glog.Errorf("Worker %s failed to execute task %s: %v", mws.workerID, task.ID, err) |
|||
} else { |
|||
glog.Infof("Worker %s completed task %s successfully", mws.workerID, task.ID) |
|||
} |
|||
}() |
|||
} |
|||
|
|||
// updateTaskProgress updates the progress of a task
|
|||
func (mws *MaintenanceWorkerService) updateTaskProgress(taskID string, progress float64) { |
|||
if mws.queue != nil { |
|||
mws.queue.UpdateTaskProgress(taskID, progress) |
|||
} |
|||
} |
|||
|
|||
// GetStatus returns the current status of the worker
|
|||
func (mws *MaintenanceWorkerService) GetStatus() map[string]interface{} { |
|||
return map[string]interface{}{ |
|||
"worker_id": mws.workerID, |
|||
"address": mws.address, |
|||
"running": mws.running, |
|||
"capabilities": mws.capabilities, |
|||
"max_concurrent": mws.maxConcurrent, |
|||
"current_tasks": len(mws.currentTasks), |
|||
"task_details": mws.currentTasks, |
|||
} |
|||
} |
|||
|
|||
// SetQueue sets the maintenance queue for the worker
|
|||
func (mws *MaintenanceWorkerService) SetQueue(queue *MaintenanceQueue) { |
|||
mws.queue = queue |
|||
} |
|||
|
|||
// SetAdminClient sets the admin client for the worker
|
|||
func (mws *MaintenanceWorkerService) SetAdminClient(client AdminClient) { |
|||
mws.adminClient = client |
|||
} |
|||
|
|||
// SetCapabilities sets the worker capabilities
|
|||
func (mws *MaintenanceWorkerService) SetCapabilities(capabilities []MaintenanceTaskType) { |
|||
mws.capabilities = capabilities |
|||
} |
|||
|
|||
// SetMaxConcurrent sets the maximum concurrent tasks
|
|||
func (mws *MaintenanceWorkerService) SetMaxConcurrent(max int) { |
|||
mws.maxConcurrent = max |
|||
} |
|||
|
|||
// SetHeartbeatInterval sets the heartbeat interval (placeholder for future use)
|
|||
func (mws *MaintenanceWorkerService) SetHeartbeatInterval(interval time.Duration) { |
|||
// Future implementation for configurable heartbeat
|
|||
} |
|||
|
|||
// SetTaskRequestInterval sets the task request interval (placeholder for future use)
|
|||
func (mws *MaintenanceWorkerService) SetTaskRequestInterval(interval time.Duration) { |
|||
// Future implementation for configurable task requests
|
|||
} |
|||
|
|||
// MaintenanceWorkerCommand represents a standalone maintenance worker command
|
|||
type MaintenanceWorkerCommand struct { |
|||
workerService *MaintenanceWorkerService |
|||
} |
|||
|
|||
// NewMaintenanceWorkerCommand creates a new worker command
|
|||
func NewMaintenanceWorkerCommand(workerID, address, adminServer string) *MaintenanceWorkerCommand { |
|||
return &MaintenanceWorkerCommand{ |
|||
workerService: NewMaintenanceWorkerService(workerID, address, adminServer), |
|||
} |
|||
} |
|||
|
|||
// Run starts the maintenance worker as a standalone service
|
|||
func (mwc *MaintenanceWorkerCommand) Run() error { |
|||
// Generate worker ID if not provided
|
|||
if mwc.workerService.workerID == "" { |
|||
hostname, _ := os.Hostname() |
|||
mwc.workerService.workerID = fmt.Sprintf("worker-%s-%d", hostname, time.Now().Unix()) |
|||
} |
|||
|
|||
// Start the worker service
|
|||
err := mwc.workerService.Start() |
|||
if err != nil { |
|||
return fmt.Errorf("failed to start maintenance worker: %v", err) |
|||
} |
|||
|
|||
// Wait for interrupt signal
|
|||
select {} |
|||
} |
|||
@ -0,0 +1,244 @@ |
|||
package app |
|||
|
|||
import ( |
|||
"fmt" |
|||
"github.com/seaweedfs/seaweedfs/weed/admin/maintenance" |
|||
) |
|||
|
|||
templ MaintenanceConfig(data *maintenance.MaintenanceConfigData) { |
|||
<div class="container-fluid"> |
|||
<div class="row mb-4"> |
|||
<div class="col-12"> |
|||
<div class="d-flex justify-content-between align-items-center"> |
|||
<h2 class="mb-0"> |
|||
<i class="fas fa-cog me-2"></i> |
|||
Maintenance Configuration |
|||
</h2> |
|||
<div class="btn-group"> |
|||
<a href="/maintenance" class="btn btn-outline-secondary"> |
|||
<i class="fas fa-arrow-left me-1"></i> |
|||
Back to Queue |
|||
</a> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<div class="row"> |
|||
<div class="col-12"> |
|||
<div class="card"> |
|||
<div class="card-header"> |
|||
<h5 class="mb-0">System Settings</h5> |
|||
</div> |
|||
<div class="card-body"> |
|||
<form> |
|||
<div class="mb-3"> |
|||
<div class="form-check form-switch"> |
|||
<input class="form-check-input" type="checkbox" id="enabled" checked?={data.IsEnabled}> |
|||
<label class="form-check-label" for="enabled"> |
|||
<strong>Enable Maintenance System</strong> |
|||
</label> |
|||
</div> |
|||
<small class="form-text text-muted"> |
|||
When enabled, the system will automatically scan for and execute maintenance tasks. |
|||
</small> |
|||
</div> |
|||
|
|||
<div class="mb-3"> |
|||
<label for="scanInterval" class="form-label">Scan Interval (minutes)</label> |
|||
<input type="number" class="form-control" id="scanInterval" |
|||
value={fmt.Sprintf("%.0f", float64(data.Config.ScanIntervalSeconds)/60)} min="1" max="1440"> |
|||
<small class="form-text text-muted"> |
|||
How often to scan for maintenance tasks (1-1440 minutes). |
|||
</small> |
|||
</div> |
|||
|
|||
<div class="mb-3"> |
|||
<label for="workerTimeout" class="form-label">Worker Timeout (minutes)</label> |
|||
<input type="number" class="form-control" id="workerTimeout" |
|||
value={fmt.Sprintf("%.0f", float64(data.Config.WorkerTimeoutSeconds)/60)} min="1" max="60"> |
|||
<small class="form-text text-muted"> |
|||
How long to wait for worker heartbeat before considering it inactive (1-60 minutes). |
|||
</small> |
|||
</div> |
|||
|
|||
<div class="mb-3"> |
|||
<label for="taskTimeout" class="form-label">Task Timeout (hours)</label> |
|||
<input type="number" class="form-control" id="taskTimeout" |
|||
value={fmt.Sprintf("%.0f", float64(data.Config.TaskTimeoutSeconds)/3600)} min="1" max="24"> |
|||
<small class="form-text text-muted"> |
|||
Maximum time allowed for a single task to complete (1-24 hours). |
|||
</small> |
|||
</div> |
|||
|
|||
<div class="mb-3"> |
|||
<label for="globalMaxConcurrent" class="form-label">Global Concurrent Limit</label> |
|||
<input type="number" class="form-control" id="globalMaxConcurrent" |
|||
value={fmt.Sprintf("%d", data.Config.Policy.GlobalMaxConcurrent)} min="1" max="20"> |
|||
<small class="form-text text-muted"> |
|||
Maximum number of maintenance tasks that can run simultaneously across all workers (1-20). |
|||
</small> |
|||
</div> |
|||
|
|||
<div class="mb-3"> |
|||
<label for="maxRetries" class="form-label">Default Max Retries</label> |
|||
<input type="number" class="form-control" id="maxRetries" |
|||
value={fmt.Sprintf("%d", data.Config.MaxRetries)} min="0" max="10"> |
|||
<small class="form-text text-muted"> |
|||
Default number of times to retry failed tasks (0-10). |
|||
</small> |
|||
</div> |
|||
|
|||
<div class="mb-3"> |
|||
<label for="retryDelay" class="form-label">Retry Delay (minutes)</label> |
|||
<input type="number" class="form-control" id="retryDelay" |
|||
value={fmt.Sprintf("%.0f", float64(data.Config.RetryDelaySeconds)/60)} min="1" max="120"> |
|||
<small class="form-text text-muted"> |
|||
Time to wait before retrying failed tasks (1-120 minutes). |
|||
</small> |
|||
</div> |
|||
|
|||
<div class="mb-3"> |
|||
<label for="taskRetention" class="form-label">Task Retention (days)</label> |
|||
<input type="number" class="form-control" id="taskRetention" |
|||
value={fmt.Sprintf("%.0f", float64(data.Config.TaskRetentionSeconds)/(24*3600))} min="1" max="30"> |
|||
<small class="form-text text-muted"> |
|||
How long to keep completed/failed task records (1-30 days). |
|||
</small> |
|||
</div> |
|||
|
|||
<div class="d-flex gap-2"> |
|||
<button type="button" class="btn btn-primary" onclick="saveConfiguration()"> |
|||
<i class="fas fa-save me-1"></i> |
|||
Save Configuration |
|||
</button> |
|||
<button type="button" class="btn btn-secondary" onclick="resetToDefaults()"> |
|||
<i class="fas fa-undo me-1"></i> |
|||
Reset to Defaults |
|||
</button> |
|||
</div> |
|||
</form> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<!-- Individual Task Configuration Menu --> |
|||
<div class="row mt-4"> |
|||
<div class="col-12"> |
|||
<div class="card"> |
|||
<div class="card-header"> |
|||
<h5 class="mb-0"> |
|||
<i class="fas fa-cogs me-2"></i> |
|||
Task Configuration |
|||
</h5> |
|||
</div> |
|||
<div class="card-body"> |
|||
<p class="text-muted mb-3">Configure specific settings for each maintenance task type.</p> |
|||
<div class="list-group"> |
|||
for _, menuItem := range data.MenuItems { |
|||
<a href={templ.SafeURL(menuItem.Path)} class="list-group-item list-group-item-action"> |
|||
<div class="d-flex w-100 justify-content-between"> |
|||
<h6 class="mb-1"> |
|||
<i class={menuItem.Icon + " me-2"}></i> |
|||
{menuItem.DisplayName} |
|||
</h6> |
|||
if data.Config.Policy.IsTaskEnabled(menuItem.TaskType) { |
|||
<span class="badge bg-success">Enabled</span> |
|||
} else { |
|||
<span class="badge bg-secondary">Disabled</span> |
|||
} |
|||
</div> |
|||
<p class="mb-1 small text-muted">{menuItem.Description}</p> |
|||
</a> |
|||
} |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<!-- Statistics Overview --> |
|||
<div class="row mt-4"> |
|||
<div class="col-12"> |
|||
<div class="card"> |
|||
<div class="card-header"> |
|||
<h5 class="mb-0">System Statistics</h5> |
|||
</div> |
|||
<div class="card-body"> |
|||
<div class="row"> |
|||
<div class="col-md-3"> |
|||
<div class="text-center"> |
|||
<h6 class="text-muted">Last Scan</h6> |
|||
<p class="mb-0">{data.LastScanTime.Format("2006-01-02 15:04:05")}</p> |
|||
</div> |
|||
</div> |
|||
<div class="col-md-3"> |
|||
<div class="text-center"> |
|||
<h6 class="text-muted">Next Scan</h6> |
|||
<p class="mb-0">{data.NextScanTime.Format("2006-01-02 15:04:05")}</p> |
|||
</div> |
|||
</div> |
|||
<div class="col-md-3"> |
|||
<div class="text-center"> |
|||
<h6 class="text-muted">Total Tasks</h6> |
|||
<p class="mb-0">{fmt.Sprintf("%d", data.SystemStats.TotalTasks)}</p> |
|||
</div> |
|||
</div> |
|||
<div class="col-md-3"> |
|||
<div class="text-center"> |
|||
<h6 class="text-muted">Active Workers</h6> |
|||
<p class="mb-0">{fmt.Sprintf("%d", data.SystemStats.ActiveWorkers)}</p> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<script> |
|||
function saveConfiguration() { |
|||
const config = { |
|||
enabled: document.getElementById('enabled').checked, |
|||
scan_interval_seconds: parseInt(document.getElementById('scanInterval').value) * 60, // Convert to seconds |
|||
policy: { |
|||
vacuum_enabled: document.getElementById('vacuumEnabled').checked, |
|||
vacuum_garbage_ratio: parseFloat(document.getElementById('vacuumGarbageRatio').value) / 100, |
|||
replication_fix_enabled: document.getElementById('replicationFixEnabled').checked, |
|||
} |
|||
}; |
|||
|
|||
fetch('/api/maintenance/config', { |
|||
method: 'PUT', |
|||
headers: { |
|||
'Content-Type': 'application/json', |
|||
}, |
|||
body: JSON.stringify(config) |
|||
}) |
|||
.then(response => response.json()) |
|||
.then(data => { |
|||
if (data.success) { |
|||
alert('Configuration saved successfully'); |
|||
} else { |
|||
alert('Failed to save configuration: ' + (data.error || 'Unknown error')); |
|||
} |
|||
}) |
|||
.catch(error => { |
|||
alert('Error: ' + error.message); |
|||
}); |
|||
} |
|||
|
|||
function resetToDefaults() { |
|||
if (confirm('Are you sure you want to reset to default configuration? This will overwrite your current settings.')) { |
|||
// Reset form to defaults |
|||
document.getElementById('enabled').checked = false; |
|||
document.getElementById('scanInterval').value = '30'; |
|||
document.getElementById('vacuumEnabled').checked = false; |
|||
document.getElementById('vacuumGarbageRatio').value = '30'; |
|||
document.getElementById('replicationFixEnabled').checked = false; |
|||
} |
|||
} |
|||
</script> |
|||
} |
|||
@ -0,0 +1,280 @@ |
|||
// 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/maintenance" |
|||
) |
|||
|
|||
func MaintenanceConfig(data *maintenance.MaintenanceConfigData) 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 mb-4\"><div class=\"col-12\"><div class=\"d-flex justify-content-between align-items-center\"><h2 class=\"mb-0\"><i class=\"fas fa-cog me-2\"></i> Maintenance Configuration</h2><div class=\"btn-group\"><a href=\"/maintenance\" class=\"btn btn-outline-secondary\"><i class=\"fas fa-arrow-left me-1\"></i> Back to Queue</a></div></div></div></div><div class=\"row\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header\"><h5 class=\"mb-0\">System Settings</h5></div><div class=\"card-body\"><form><div class=\"mb-3\"><div class=\"form-check form-switch\"><input class=\"form-check-input\" type=\"checkbox\" id=\"enabled\"") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
if data.IsEnabled { |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 2, " checked") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 3, "> <label class=\"form-check-label\" for=\"enabled\"><strong>Enable Maintenance System</strong></label></div><small class=\"form-text text-muted\">When enabled, the system will automatically scan for and execute maintenance tasks.</small></div><div class=\"mb-3\"><label for=\"scanInterval\" class=\"form-label\">Scan Interval (minutes)</label> <input type=\"number\" class=\"form-control\" id=\"scanInterval\" value=\"") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var2 string |
|||
templ_7745c5c3_Var2, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%.0f", float64(data.Config.ScanIntervalSeconds)/60)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_config.templ`, Line: 50, Col: 110} |
|||
} |
|||
_, 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, 4, "\" min=\"1\" max=\"1440\"> <small class=\"form-text text-muted\">How often to scan for maintenance tasks (1-1440 minutes).</small></div><div class=\"mb-3\"><label for=\"workerTimeout\" class=\"form-label\">Worker Timeout (minutes)</label> <input type=\"number\" class=\"form-control\" id=\"workerTimeout\" value=\"") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var3 string |
|||
templ_7745c5c3_Var3, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%.0f", float64(data.Config.WorkerTimeoutSeconds)/60)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_config.templ`, Line: 59, Col: 111} |
|||
} |
|||
_, 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, "\" min=\"1\" max=\"60\"> <small class=\"form-text text-muted\">How long to wait for worker heartbeat before considering it inactive (1-60 minutes).</small></div><div class=\"mb-3\"><label for=\"taskTimeout\" class=\"form-label\">Task Timeout (hours)</label> <input type=\"number\" class=\"form-control\" id=\"taskTimeout\" value=\"") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var4 string |
|||
templ_7745c5c3_Var4, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%.0f", float64(data.Config.TaskTimeoutSeconds)/3600)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_config.templ`, Line: 68, Col: 111} |
|||
} |
|||
_, 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, "\" min=\"1\" max=\"24\"> <small class=\"form-text text-muted\">Maximum time allowed for a single task to complete (1-24 hours).</small></div><div class=\"mb-3\"><label for=\"globalMaxConcurrent\" class=\"form-label\">Global Concurrent Limit</label> <input type=\"number\" class=\"form-control\" id=\"globalMaxConcurrent\" value=\"") |
|||
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.Config.Policy.GlobalMaxConcurrent)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_config.templ`, Line: 77, Col: 103} |
|||
} |
|||
_, 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, "\" min=\"1\" max=\"20\"> <small class=\"form-text text-muted\">Maximum number of maintenance tasks that can run simultaneously across all workers (1-20).</small></div><div class=\"mb-3\"><label for=\"maxRetries\" class=\"form-label\">Default Max Retries</label> <input type=\"number\" class=\"form-control\" id=\"maxRetries\" value=\"") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var6 string |
|||
templ_7745c5c3_Var6, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.Config.MaxRetries)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_config.templ`, Line: 86, Col: 87} |
|||
} |
|||
_, 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, "\" min=\"0\" max=\"10\"> <small class=\"form-text text-muted\">Default number of times to retry failed tasks (0-10).</small></div><div class=\"mb-3\"><label for=\"retryDelay\" class=\"form-label\">Retry Delay (minutes)</label> <input type=\"number\" class=\"form-control\" id=\"retryDelay\" value=\"") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var7 string |
|||
templ_7745c5c3_Var7, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%.0f", float64(data.Config.RetryDelaySeconds)/60)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_config.templ`, Line: 95, Col: 108} |
|||
} |
|||
_, 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, 9, "\" min=\"1\" max=\"120\"> <small class=\"form-text text-muted\">Time to wait before retrying failed tasks (1-120 minutes).</small></div><div class=\"mb-3\"><label for=\"taskRetention\" class=\"form-label\">Task Retention (days)</label> <input type=\"number\" class=\"form-control\" id=\"taskRetention\" value=\"") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var8 string |
|||
templ_7745c5c3_Var8, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%.0f", float64(data.Config.TaskRetentionSeconds)/(24*3600))) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_config.templ`, Line: 104, Col: 118} |
|||
} |
|||
_, 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, 10, "\" min=\"1\" max=\"30\"> <small class=\"form-text text-muted\">How long to keep completed/failed task records (1-30 days).</small></div><div class=\"d-flex gap-2\"><button type=\"button\" class=\"btn btn-primary\" onclick=\"saveConfiguration()\"><i class=\"fas fa-save me-1\"></i> Save Configuration</button> <button type=\"button\" class=\"btn btn-secondary\" onclick=\"resetToDefaults()\"><i class=\"fas fa-undo me-1\"></i> Reset to Defaults</button></div></form></div></div></div></div><!-- Individual Task Configuration Menu --><div class=\"row mt-4\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header\"><h5 class=\"mb-0\"><i class=\"fas fa-cogs me-2\"></i> Task Configuration</h5></div><div class=\"card-body\"><p class=\"text-muted mb-3\">Configure specific settings for each maintenance task type.</p><div class=\"list-group\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
for _, menuItem := range data.MenuItems { |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 11, "<a href=\"") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var9 templ.SafeURL = templ.SafeURL(menuItem.Path) |
|||
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(string(templ_7745c5c3_Var9))) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 12, "\" class=\"list-group-item list-group-item-action\"><div class=\"d-flex w-100 justify-content-between\"><h6 class=\"mb-1\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var10 = []any{menuItem.Icon + " me-2"} |
|||
templ_7745c5c3_Err = templ.RenderCSSItems(ctx, templ_7745c5c3_Buffer, templ_7745c5c3_Var10...) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, "<i class=\"") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var11 string |
|||
templ_7745c5c3_Var11, templ_7745c5c3_Err = templ.JoinStringErrs(templ.CSSClasses(templ_7745c5c3_Var10).String()) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_config.templ`, Line: 1, Col: 0} |
|||
} |
|||
_, 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, 14, "\"></i> ") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var12 string |
|||
templ_7745c5c3_Var12, templ_7745c5c3_Err = templ.JoinStringErrs(menuItem.DisplayName) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_config.templ`, Line: 144, Col: 65} |
|||
} |
|||
_, 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, 15, "</h6>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
if data.Config.Policy.IsTaskEnabled(menuItem.TaskType) { |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, "<span class=\"badge bg-success\">Enabled</span>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
} else { |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, "<span class=\"badge bg-secondary\">Disabled</span>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 18, "</div><p class=\"mb-1 small text-muted\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var13 string |
|||
templ_7745c5c3_Var13, templ_7745c5c3_Err = templ.JoinStringErrs(menuItem.Description) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_config.templ`, Line: 152, Col: 90} |
|||
} |
|||
_, 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, "</p></a>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, "</div></div></div></div></div><!-- Statistics Overview --><div class=\"row mt-4\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header\"><h5 class=\"mb-0\">System Statistics</h5></div><div class=\"card-body\"><div class=\"row\"><div class=\"col-md-3\"><div class=\"text-center\"><h6 class=\"text-muted\">Last Scan</h6><p class=\"mb-0\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var14 string |
|||
templ_7745c5c3_Var14, templ_7745c5c3_Err = templ.JoinStringErrs(data.LastScanTime.Format("2006-01-02 15:04:05")) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_config.templ`, Line: 173, Col: 100} |
|||
} |
|||
_, 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, 21, "</p></div></div><div class=\"col-md-3\"><div class=\"text-center\"><h6 class=\"text-muted\">Next Scan</h6><p class=\"mb-0\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var15 string |
|||
templ_7745c5c3_Var15, templ_7745c5c3_Err = templ.JoinStringErrs(data.NextScanTime.Format("2006-01-02 15:04:05")) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_config.templ`, Line: 179, Col: 100} |
|||
} |
|||
_, 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, 22, "</p></div></div><div class=\"col-md-3\"><div class=\"text-center\"><h6 class=\"text-muted\">Total Tasks</h6><p class=\"mb-0\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var16 string |
|||
templ_7745c5c3_Var16, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.SystemStats.TotalTasks)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_config.templ`, Line: 185, Col: 99} |
|||
} |
|||
_, 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, 23, "</p></div></div><div class=\"col-md-3\"><div class=\"text-center\"><h6 class=\"text-muted\">Active Workers</h6><p class=\"mb-0\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var17 string |
|||
templ_7745c5c3_Var17, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.SystemStats.ActiveWorkers)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_config.templ`, Line: 191, Col: 102} |
|||
} |
|||
_, 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, 24, "</p></div></div></div></div></div></div></div></div><script>\n function saveConfiguration() {\n const config = {\n enabled: document.getElementById('enabled').checked,\n scan_interval_seconds: parseInt(document.getElementById('scanInterval').value) * 60, // Convert to seconds\n policy: {\n vacuum_enabled: document.getElementById('vacuumEnabled').checked,\n vacuum_garbage_ratio: parseFloat(document.getElementById('vacuumGarbageRatio').value) / 100,\n replication_fix_enabled: document.getElementById('replicationFixEnabled').checked,\n }\n };\n\n fetch('/api/maintenance/config', {\n method: 'PUT',\n headers: {\n 'Content-Type': 'application/json',\n },\n body: JSON.stringify(config)\n })\n .then(response => response.json())\n .then(data => {\n if (data.success) {\n alert('Configuration saved successfully');\n } else {\n alert('Failed to save configuration: ' + (data.error || 'Unknown error'));\n }\n })\n .catch(error => {\n alert('Error: ' + error.message);\n });\n }\n\n function resetToDefaults() {\n if (confirm('Are you sure you want to reset to default configuration? This will overwrite your current settings.')) {\n // Reset form to defaults\n document.getElementById('enabled').checked = false;\n document.getElementById('scanInterval').value = '30';\n document.getElementById('vacuumEnabled').checked = false;\n document.getElementById('vacuumGarbageRatio').value = '30';\n document.getElementById('replicationFixEnabled').checked = false;\n }\n }\n </script>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
return nil |
|||
}) |
|||
} |
|||
|
|||
var _ = templruntime.GeneratedTemplate |
|||
@ -0,0 +1,289 @@ |
|||
package app |
|||
|
|||
import ( |
|||
"fmt" |
|||
"time" |
|||
"github.com/seaweedfs/seaweedfs/weed/admin/maintenance" |
|||
) |
|||
|
|||
templ MaintenanceQueue(data *maintenance.MaintenanceQueueData) { |
|||
<div class="container-fluid"> |
|||
<!-- Header --> |
|||
<div class="row mb-4"> |
|||
<div class="col-12"> |
|||
<div class="d-flex justify-content-between align-items-center"> |
|||
<h2 class="mb-0"> |
|||
<i class="fas fa-tasks me-2"></i> |
|||
Maintenance Queue |
|||
</h2> |
|||
<div class="btn-group"> |
|||
<button type="button" class="btn btn-primary" onclick="triggerScan()"> |
|||
<i class="fas fa-search me-1"></i> |
|||
Trigger Scan |
|||
</button> |
|||
<button type="button" class="btn btn-secondary" onclick="refreshPage()"> |
|||
<i class="fas fa-sync-alt me-1"></i> |
|||
Refresh |
|||
</button> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<!-- Statistics Cards --> |
|||
<div class="row mb-4"> |
|||
<div class="col-md-3"> |
|||
<div class="card border-primary"> |
|||
<div class="card-body text-center"> |
|||
<i class="fas fa-clock fa-2x text-primary mb-2"></i> |
|||
<h4 class="mb-1">{fmt.Sprintf("%d", data.Stats.PendingTasks)}</h4> |
|||
<p class="text-muted mb-0">Pending Tasks</p> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
<div class="col-md-3"> |
|||
<div class="card border-warning"> |
|||
<div class="card-body text-center"> |
|||
<i class="fas fa-running fa-2x text-warning mb-2"></i> |
|||
<h4 class="mb-1">{fmt.Sprintf("%d", data.Stats.RunningTasks)}</h4> |
|||
<p class="text-muted mb-0">Running Tasks</p> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
<div class="col-md-3"> |
|||
<div class="card border-success"> |
|||
<div class="card-body text-center"> |
|||
<i class="fas fa-check-circle fa-2x text-success mb-2"></i> |
|||
<h4 class="mb-1">{fmt.Sprintf("%d", data.Stats.CompletedToday)}</h4> |
|||
<p class="text-muted mb-0">Completed Today</p> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
<div class="col-md-3"> |
|||
<div class="card border-danger"> |
|||
<div class="card-body text-center"> |
|||
<i class="fas fa-exclamation-triangle fa-2x text-danger mb-2"></i> |
|||
<h4 class="mb-1">{fmt.Sprintf("%d", data.Stats.FailedToday)}</h4> |
|||
<p class="text-muted mb-0">Failed Today</p> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<!-- Simple task queue display --> |
|||
<div class="row"> |
|||
<div class="col-12"> |
|||
<div class="card"> |
|||
<div class="card-header"> |
|||
<h5 class="mb-0">Task Queue</h5> |
|||
</div> |
|||
<div class="card-body"> |
|||
if len(data.Tasks) == 0 { |
|||
<div class="text-center text-muted py-4"> |
|||
<i class="fas fa-clipboard-list fa-3x mb-3"></i> |
|||
<p>No maintenance tasks in queue</p> |
|||
<small>Tasks will appear here when the system detects maintenance needs</small> |
|||
</div> |
|||
} else { |
|||
<div class="table-responsive"> |
|||
<table class="table table-hover"> |
|||
<thead> |
|||
<tr> |
|||
<th>ID</th> |
|||
<th>Type</th> |
|||
<th>Status</th> |
|||
<th>Volume</th> |
|||
<th>Server</th> |
|||
<th>Created</th> |
|||
</tr> |
|||
</thead> |
|||
<tbody> |
|||
for _, task := range data.Tasks { |
|||
<tr> |
|||
<td><code>{task.ID[:8]}...</code></td> |
|||
<td>{string(task.Type)}</td> |
|||
<td>{string(task.Status)}</td> |
|||
<td>{fmt.Sprintf("%d", task.VolumeID)}</td> |
|||
<td>{task.Server}</td> |
|||
<td>{task.CreatedAt.Format("2006-01-02 15:04")}</td> |
|||
</tr> |
|||
} |
|||
</tbody> |
|||
</table> |
|||
</div> |
|||
} |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<!-- Workers Summary --> |
|||
<div class="row mt-4"> |
|||
<div class="col-12"> |
|||
<div class="card"> |
|||
<div class="card-header"> |
|||
<h5 class="mb-0">Active Workers</h5> |
|||
</div> |
|||
<div class="card-body"> |
|||
if len(data.Workers) == 0 { |
|||
<div class="text-center text-muted py-4"> |
|||
<i class="fas fa-robot fa-3x mb-3"></i> |
|||
<p>No workers are currently active</p> |
|||
<small>Start workers using: <code>weed worker -admin=localhost:9333</code></small> |
|||
</div> |
|||
} else { |
|||
<div class="row"> |
|||
for _, worker := range data.Workers { |
|||
<div class="col-md-4 mb-3"> |
|||
<div class="card"> |
|||
<div class="card-body"> |
|||
<h6 class="card-title">{worker.ID}</h6> |
|||
<p class="card-text"> |
|||
<small class="text-muted">{worker.Address}</small><br/> |
|||
Status: {worker.Status}<br/> |
|||
Load: {fmt.Sprintf("%d/%d", worker.CurrentLoad, worker.MaxConcurrent)} |
|||
</p> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
} |
|||
</div> |
|||
} |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<script> |
|||
// Auto-refresh every 10 seconds |
|||
setInterval(function() { |
|||
if (!document.hidden) { |
|||
window.location.reload(); |
|||
} |
|||
}, 10000); |
|||
|
|||
function triggerScan() { |
|||
fetch('/api/maintenance/scan', { |
|||
method: 'POST', |
|||
headers: { |
|||
'Content-Type': 'application/json', |
|||
} |
|||
}) |
|||
.then(response => response.json()) |
|||
.then(data => { |
|||
if (data.success) { |
|||
alert('Maintenance scan triggered successfully'); |
|||
setTimeout(() => window.location.reload(), 2000); |
|||
} else { |
|||
alert('Failed to trigger scan: ' + (data.error || 'Unknown error')); |
|||
} |
|||
}) |
|||
.catch(error => { |
|||
alert('Error: ' + error.message); |
|||
}); |
|||
} |
|||
</script> |
|||
} |
|||
|
|||
// Helper components |
|||
templ TaskTypeIcon(taskType maintenance.MaintenanceTaskType) { |
|||
<i class={maintenance.GetTaskIcon(taskType) + " me-1"}></i> |
|||
} |
|||
|
|||
templ PriorityBadge(priority maintenance.MaintenanceTaskPriority) { |
|||
switch priority { |
|||
case maintenance.PriorityCritical: |
|||
<span class="badge bg-danger">Critical</span> |
|||
case maintenance.PriorityHigh: |
|||
<span class="badge bg-warning">High</span> |
|||
case maintenance.PriorityNormal: |
|||
<span class="badge bg-primary">Normal</span> |
|||
case maintenance.PriorityLow: |
|||
<span class="badge bg-secondary">Low</span> |
|||
default: |
|||
<span class="badge bg-light text-dark">Unknown</span> |
|||
} |
|||
} |
|||
|
|||
templ StatusBadge(status maintenance.MaintenanceTaskStatus) { |
|||
switch status { |
|||
case maintenance.TaskStatusPending: |
|||
<span class="badge bg-secondary">Pending</span> |
|||
case maintenance.TaskStatusAssigned: |
|||
<span class="badge bg-info">Assigned</span> |
|||
case maintenance.TaskStatusInProgress: |
|||
<span class="badge bg-warning">Running</span> |
|||
case maintenance.TaskStatusCompleted: |
|||
<span class="badge bg-success">Completed</span> |
|||
case maintenance.TaskStatusFailed: |
|||
<span class="badge bg-danger">Failed</span> |
|||
case maintenance.TaskStatusCancelled: |
|||
<span class="badge bg-light text-dark">Cancelled</span> |
|||
default: |
|||
<span class="badge bg-light text-dark">Unknown</span> |
|||
} |
|||
} |
|||
|
|||
templ ProgressBar(progress float64, status maintenance.MaintenanceTaskStatus) { |
|||
if status == maintenance.TaskStatusInProgress || status == maintenance.TaskStatusAssigned { |
|||
<div class="progress" style="height: 8px; min-width: 100px;"> |
|||
<div class="progress-bar" role="progressbar" style={fmt.Sprintf("width: %.1f%%", progress)}> |
|||
</div> |
|||
</div> |
|||
<small class="text-muted">{fmt.Sprintf("%.1f%%", progress)}</small> |
|||
} else if status == maintenance.TaskStatusCompleted { |
|||
<div class="progress" style="height: 8px; min-width: 100px;"> |
|||
<div class="progress-bar bg-success" role="progressbar" style="width: 100%"> |
|||
</div> |
|||
</div> |
|||
<small class="text-success">100%</small> |
|||
} else { |
|||
<span class="text-muted">-</span> |
|||
} |
|||
} |
|||
|
|||
templ WorkerStatusBadge(status string) { |
|||
switch status { |
|||
case "active": |
|||
<span class="badge bg-success">Active</span> |
|||
case "busy": |
|||
<span class="badge bg-warning">Busy</span> |
|||
case "inactive": |
|||
<span class="badge bg-secondary">Inactive</span> |
|||
default: |
|||
<span class="badge bg-light text-dark">Unknown</span> |
|||
} |
|||
} |
|||
|
|||
// Helper functions (would be defined in Go) |
|||
|
|||
|
|||
func getWorkerStatusColor(status string) string { |
|||
switch status { |
|||
case "active": |
|||
return "success" |
|||
case "busy": |
|||
return "warning" |
|||
case "inactive": |
|||
return "secondary" |
|||
default: |
|||
return "light" |
|||
} |
|||
} |
|||
|
|||
func formatTimeAgo(t time.Time) string { |
|||
duration := time.Since(t) |
|||
if duration < time.Minute { |
|||
return "just now" |
|||
} else if duration < time.Hour { |
|||
minutes := int(duration.Minutes()) |
|||
return fmt.Sprintf("%dm ago", minutes) |
|||
} else if duration < 24*time.Hour { |
|||
hours := int(duration.Hours()) |
|||
return fmt.Sprintf("%dh ago", hours) |
|||
} else { |
|||
days := int(duration.Hours() / 24) |
|||
return fmt.Sprintf("%dd ago", days) |
|||
} |
|||
} |
|||
@ -0,0 +1,585 @@ |
|||
// 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/maintenance" |
|||
"time" |
|||
) |
|||
|
|||
func MaintenanceQueue(data *maintenance.MaintenanceQueueData) 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\"><!-- Header --><div class=\"row mb-4\"><div class=\"col-12\"><div class=\"d-flex justify-content-between align-items-center\"><h2 class=\"mb-0\"><i class=\"fas fa-tasks me-2\"></i> Maintenance Queue</h2><div class=\"btn-group\"><button type=\"button\" class=\"btn btn-primary\" onclick=\"triggerScan()\"><i class=\"fas fa-search me-1\"></i> Trigger Scan</button> <button type=\"button\" class=\"btn btn-secondary\" onclick=\"refreshPage()\"><i class=\"fas fa-sync-alt me-1\"></i> Refresh</button></div></div></div></div><!-- Statistics Cards --><div class=\"row mb-4\"><div class=\"col-md-3\"><div class=\"card border-primary\"><div class=\"card-body text-center\"><i class=\"fas fa-clock fa-2x text-primary mb-2\"></i><h4 class=\"mb-1\">") |
|||
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.Stats.PendingTasks)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 39, Col: 84} |
|||
} |
|||
_, 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, "</h4><p class=\"text-muted mb-0\">Pending Tasks</p></div></div></div><div class=\"col-md-3\"><div class=\"card border-warning\"><div class=\"card-body text-center\"><i class=\"fas fa-running fa-2x text-warning mb-2\"></i><h4 class=\"mb-1\">") |
|||
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.Stats.RunningTasks)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 48, Col: 84} |
|||
} |
|||
_, 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, "</h4><p class=\"text-muted mb-0\">Running Tasks</p></div></div></div><div class=\"col-md-3\"><div class=\"card border-success\"><div class=\"card-body text-center\"><i class=\"fas fa-check-circle fa-2x text-success mb-2\"></i><h4 class=\"mb-1\">") |
|||
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.Stats.CompletedToday)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 57, Col: 86} |
|||
} |
|||
_, 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, "</h4><p class=\"text-muted mb-0\">Completed Today</p></div></div></div><div class=\"col-md-3\"><div class=\"card border-danger\"><div class=\"card-body text-center\"><i class=\"fas fa-exclamation-triangle fa-2x text-danger mb-2\"></i><h4 class=\"mb-1\">") |
|||
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.Stats.FailedToday)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 66, Col: 83} |
|||
} |
|||
_, 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, "</h4><p class=\"text-muted mb-0\">Failed Today</p></div></div></div></div><!-- Simple task queue display --><div class=\"row\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header\"><h5 class=\"mb-0\">Task Queue</h5></div><div class=\"card-body\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
if len(data.Tasks) == 0 { |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 6, "<div class=\"text-center text-muted py-4\"><i class=\"fas fa-clipboard-list fa-3x mb-3\"></i><p>No maintenance tasks in queue</p><small>Tasks will appear here when the system detects maintenance needs</small></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-hover\"><thead><tr><th>ID</th><th>Type</th><th>Status</th><th>Volume</th><th>Server</th><th>Created</th></tr></thead> <tbody>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
for _, task := range data.Tasks { |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 8, "<tr><td><code>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var6 string |
|||
templ_7745c5c3_Var6, templ_7745c5c3_Err = templ.JoinStringErrs(task.ID[:8]) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 103, Col: 70} |
|||
} |
|||
_, 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, "...</code></td><td>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var7 string |
|||
templ_7745c5c3_Var7, templ_7745c5c3_Err = templ.JoinStringErrs(string(task.Type)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 104, Col: 70} |
|||
} |
|||
_, 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, "</td><td>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var8 string |
|||
templ_7745c5c3_Var8, templ_7745c5c3_Err = templ.JoinStringErrs(string(task.Status)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 105, Col: 72} |
|||
} |
|||
_, 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 |
|||
} |
|||
var templ_7745c5c3_Var9 string |
|||
templ_7745c5c3_Var9, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", task.VolumeID)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 106, Col: 85} |
|||
} |
|||
_, 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, 12, "</td><td>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var10 string |
|||
templ_7745c5c3_Var10, templ_7745c5c3_Err = templ.JoinStringErrs(task.Server) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 107, Col: 64} |
|||
} |
|||
_, 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, 13, "</td><td>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var11 string |
|||
templ_7745c5c3_Var11, templ_7745c5c3_Err = templ.JoinStringErrs(task.CreatedAt.Format("2006-01-02 15:04")) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 108, Col: 94} |
|||
} |
|||
_, 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, 14, "</td></tr>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "</tbody></table></div>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, "</div></div></div></div><!-- Workers Summary --><div class=\"row mt-4\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header\"><h5 class=\"mb-0\">Active Workers</h5></div><div class=\"card-body\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
if len(data.Workers) == 0 { |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, "<div class=\"text-center text-muted py-4\"><i class=\"fas fa-robot fa-3x mb-3\"></i><p>No workers are currently active</p><small>Start workers using: <code>weed worker -admin=localhost:9333</code></small></div>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
} else { |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 18, "<div class=\"row\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
for _, worker := range data.Workers { |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, "<div class=\"col-md-4 mb-3\"><div class=\"card\"><div class=\"card-body\"><h6 class=\"card-title\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var12 string |
|||
templ_7745c5c3_Var12, templ_7745c5c3_Err = templ.JoinStringErrs(worker.ID) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 140, Col: 81} |
|||
} |
|||
_, 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, 20, "</h6><p class=\"card-text\"><small class=\"text-muted\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var13 string |
|||
templ_7745c5c3_Var13, templ_7745c5c3_Err = templ.JoinStringErrs(worker.Address) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 142, Col: 93} |
|||
} |
|||
_, 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, 21, "</small><br>Status: ") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var14 string |
|||
templ_7745c5c3_Var14, templ_7745c5c3_Err = templ.JoinStringErrs(worker.Status) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 143, Col: 74} |
|||
} |
|||
_, 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, 22, "<br>Load: ") |
|||
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", worker.CurrentLoad, worker.MaxConcurrent)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 144, Col: 121} |
|||
} |
|||
_, 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, 23, "</p></div></div></div>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, "</div>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 25, "</div></div></div></div></div><script>\n // Auto-refresh every 10 seconds\n setInterval(function() {\n if (!document.hidden) {\n window.location.reload();\n }\n }, 10000);\n\n function triggerScan() {\n fetch('/api/maintenance/scan', {\n method: 'POST',\n headers: {\n 'Content-Type': 'application/json',\n }\n })\n .then(response => response.json())\n .then(data => {\n if (data.success) {\n alert('Maintenance scan triggered successfully');\n setTimeout(() => window.location.reload(), 2000);\n } else {\n alert('Failed to trigger scan: ' + (data.error || 'Unknown error'));\n }\n })\n .catch(error => {\n alert('Error: ' + error.message);\n });\n }\n </script>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
return nil |
|||
}) |
|||
} |
|||
|
|||
// Helper components
|
|||
func TaskTypeIcon(taskType maintenance.MaintenanceTaskType) 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_Var16 := templ.GetChildren(ctx) |
|||
if templ_7745c5c3_Var16 == nil { |
|||
templ_7745c5c3_Var16 = templ.NopComponent |
|||
} |
|||
ctx = templ.ClearChildren(ctx) |
|||
var templ_7745c5c3_Var17 = []any{maintenance.GetTaskIcon(taskType) + " me-1"} |
|||
templ_7745c5c3_Err = templ.RenderCSSItems(ctx, templ_7745c5c3_Buffer, templ_7745c5c3_Var17...) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, "<i class=\"") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var18 string |
|||
templ_7745c5c3_Var18, templ_7745c5c3_Err = templ.JoinStringErrs(templ.CSSClasses(templ_7745c5c3_Var17).String()) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 1, Col: 0} |
|||
} |
|||
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var18)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, "\"></i>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
return nil |
|||
}) |
|||
} |
|||
|
|||
func PriorityBadge(priority maintenance.MaintenanceTaskPriority) 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_Var19 := templ.GetChildren(ctx) |
|||
if templ_7745c5c3_Var19 == nil { |
|||
templ_7745c5c3_Var19 = templ.NopComponent |
|||
} |
|||
ctx = templ.ClearChildren(ctx) |
|||
switch priority { |
|||
case maintenance.PriorityCritical: |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, "<span class=\"badge bg-danger\">Critical</span>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
case maintenance.PriorityHigh: |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, "<span class=\"badge bg-warning\">High</span>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
case maintenance.PriorityNormal: |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 30, "<span class=\"badge bg-primary\">Normal</span>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
case maintenance.PriorityLow: |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 31, "<span class=\"badge bg-secondary\">Low</span>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
default: |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 32, "<span class=\"badge bg-light text-dark\">Unknown</span>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
} |
|||
return nil |
|||
}) |
|||
} |
|||
|
|||
func StatusBadge(status maintenance.MaintenanceTaskStatus) 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_Var20 := templ.GetChildren(ctx) |
|||
if templ_7745c5c3_Var20 == nil { |
|||
templ_7745c5c3_Var20 = templ.NopComponent |
|||
} |
|||
ctx = templ.ClearChildren(ctx) |
|||
switch status { |
|||
case maintenance.TaskStatusPending: |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 33, "<span class=\"badge bg-secondary\">Pending</span>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
case maintenance.TaskStatusAssigned: |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 34, "<span class=\"badge bg-info\">Assigned</span>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
case maintenance.TaskStatusInProgress: |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 35, "<span class=\"badge bg-warning\">Running</span>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
case maintenance.TaskStatusCompleted: |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 36, "<span class=\"badge bg-success\">Completed</span>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
case maintenance.TaskStatusFailed: |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 37, "<span class=\"badge bg-danger\">Failed</span>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
case maintenance.TaskStatusCancelled: |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 38, "<span class=\"badge bg-light text-dark\">Cancelled</span>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
default: |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 39, "<span class=\"badge bg-light text-dark\">Unknown</span>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
} |
|||
return nil |
|||
}) |
|||
} |
|||
|
|||
func ProgressBar(progress float64, status maintenance.MaintenanceTaskStatus) 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_Var21 := templ.GetChildren(ctx) |
|||
if templ_7745c5c3_Var21 == nil { |
|||
templ_7745c5c3_Var21 = templ.NopComponent |
|||
} |
|||
ctx = templ.ClearChildren(ctx) |
|||
if status == maintenance.TaskStatusInProgress || status == maintenance.TaskStatusAssigned { |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 40, "<div class=\"progress\" style=\"height: 8px; min-width: 100px;\"><div class=\"progress-bar\" role=\"progressbar\" style=\"") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var22 string |
|||
templ_7745c5c3_Var22, templ_7745c5c3_Err = templruntime.SanitizeStyleAttributeValues(fmt.Sprintf("width: %.1f%%", progress)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 231, Col: 102} |
|||
} |
|||
_, 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, 41, "\"></div></div><small class=\"text-muted\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var23 string |
|||
templ_7745c5c3_Var23, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%.1f%%", progress)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 234, Col: 66} |
|||
} |
|||
_, 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, 42, "</small>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
} else if status == maintenance.TaskStatusCompleted { |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 43, "<div class=\"progress\" style=\"height: 8px; min-width: 100px;\"><div class=\"progress-bar bg-success\" role=\"progressbar\" style=\"width: 100%\"></div></div><small class=\"text-success\">100%</small>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
} else { |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 44, "<span class=\"text-muted\">-</span>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
} |
|||
return nil |
|||
}) |
|||
} |
|||
|
|||
func WorkerStatusBadge(status string) 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_Var24 := templ.GetChildren(ctx) |
|||
if templ_7745c5c3_Var24 == nil { |
|||
templ_7745c5c3_Var24 = templ.NopComponent |
|||
} |
|||
ctx = templ.ClearChildren(ctx) |
|||
switch status { |
|||
case "active": |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 45, "<span class=\"badge bg-success\">Active</span>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
case "busy": |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 46, "<span class=\"badge bg-warning\">Busy</span>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
case "inactive": |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 47, "<span class=\"badge bg-secondary\">Inactive</span>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
default: |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 48, "<span class=\"badge bg-light text-dark\">Unknown</span>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
} |
|||
return nil |
|||
}) |
|||
} |
|||
|
|||
// Helper functions (would be defined in Go)
|
|||
|
|||
func getWorkerStatusColor(status string) string { |
|||
switch status { |
|||
case "active": |
|||
return "success" |
|||
case "busy": |
|||
return "warning" |
|||
case "inactive": |
|||
return "secondary" |
|||
default: |
|||
return "light" |
|||
} |
|||
} |
|||
|
|||
func formatTimeAgo(t time.Time) string { |
|||
duration := time.Since(t) |
|||
if duration < time.Minute { |
|||
return "just now" |
|||
} else if duration < time.Hour { |
|||
minutes := int(duration.Minutes()) |
|||
return fmt.Sprintf("%dm ago", minutes) |
|||
} else if duration < 24*time.Hour { |
|||
hours := int(duration.Hours()) |
|||
return fmt.Sprintf("%dh ago", hours) |
|||
} else { |
|||
days := int(duration.Hours() / 24) |
|||
return fmt.Sprintf("%dd ago", days) |
|||
} |
|||
} |
|||
|
|||
var _ = templruntime.GeneratedTemplate |
|||
@ -0,0 +1,340 @@ |
|||
package app |
|||
|
|||
import ( |
|||
"fmt" |
|||
"github.com/seaweedfs/seaweedfs/weed/admin/dash" |
|||
"time" |
|||
) |
|||
|
|||
templ MaintenanceWorkers(data *dash.MaintenanceWorkersData) { |
|||
<div class="container-fluid"> |
|||
<div class="row"> |
|||
<div class="col-12"> |
|||
<div class="d-flex justify-content-between align-items-center mb-4"> |
|||
<div> |
|||
<h1 class="h3 mb-0 text-gray-800">Maintenance Workers</h1> |
|||
<p class="text-muted">Monitor and manage maintenance workers</p> |
|||
</div> |
|||
<div class="text-end"> |
|||
<small class="text-muted">Last updated: { data.LastUpdated.Format("2006-01-02 15:04:05") }</small> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<!-- Summary Cards --> |
|||
<div class="row mb-4"> |
|||
<div class="col-xl-3 col-md-6 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 Workers |
|||
</div> |
|||
<div class="h5 mb-0 font-weight-bold text-gray-800">{ fmt.Sprintf("%d", len(data.Workers)) }</div> |
|||
</div> |
|||
<div class="col-auto"> |
|||
<i class="fas fa-users fa-2x text-gray-300"></i> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<div class="col-xl-3 col-md-6 mb-4"> |
|||
<div class="card border-left-success 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-success text-uppercase mb-1"> |
|||
Active Workers |
|||
</div> |
|||
<div class="h5 mb-0 font-weight-bold text-gray-800"> |
|||
{ fmt.Sprintf("%d", data.ActiveWorkers) } |
|||
</div> |
|||
</div> |
|||
<div class="col-auto"> |
|||
<i class="fas fa-check-circle fa-2x text-gray-300"></i> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<div class="col-xl-3 col-md-6 mb-4"> |
|||
<div class="card border-left-info 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-info text-uppercase mb-1"> |
|||
Busy Workers |
|||
</div> |
|||
<div class="h5 mb-0 font-weight-bold text-gray-800"> |
|||
{ fmt.Sprintf("%d", data.BusyWorkers) } |
|||
</div> |
|||
</div> |
|||
<div class="col-auto"> |
|||
<i class="fas fa-spinner fa-2x text-gray-300"></i> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<div class="col-xl-3 col-md-6 mb-4"> |
|||
<div class="card border-left-warning 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-warning text-uppercase mb-1"> |
|||
Total Load |
|||
</div> |
|||
<div class="h5 mb-0 font-weight-bold text-gray-800"> |
|||
{ fmt.Sprintf("%d", data.TotalLoad) } |
|||
</div> |
|||
</div> |
|||
<div class="col-auto"> |
|||
<i class="fas fa-tasks fa-2x text-gray-300"></i> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<!-- Workers Table --> |
|||
<div class="row"> |
|||
<div class="col-12"> |
|||
<div class="card shadow mb-4"> |
|||
<div class="card-header py-3"> |
|||
<h6 class="m-0 font-weight-bold text-primary">Worker Details</h6> |
|||
</div> |
|||
<div class="card-body"> |
|||
if len(data.Workers) == 0 { |
|||
<div class="text-center py-4"> |
|||
<i class="fas fa-users fa-3x text-gray-300 mb-3"></i> |
|||
<h5 class="text-gray-600">No Workers Found</h5> |
|||
<p class="text-muted">No maintenance workers are currently registered.</p> |
|||
<div class="alert alert-info mt-3"> |
|||
<strong>💡 Tip:</strong> To start a worker, run: |
|||
<br><code>weed worker -admin=<admin_server> -capabilities=vacuum,ec,replication</code> |
|||
</div> |
|||
</div> |
|||
} else { |
|||
<div class="table-responsive"> |
|||
<table class="table table-bordered table-hover" id="workersTable"> |
|||
<thead class="table-light"> |
|||
<tr> |
|||
<th>Worker ID</th> |
|||
<th>Address</th> |
|||
<th>Status</th> |
|||
<th>Capabilities</th> |
|||
<th>Load</th> |
|||
<th>Current Tasks</th> |
|||
<th>Performance</th> |
|||
<th>Last Heartbeat</th> |
|||
<th>Actions</th> |
|||
</tr> |
|||
</thead> |
|||
<tbody> |
|||
for _, worker := range data.Workers { |
|||
<tr> |
|||
<td> |
|||
<code>{ worker.Worker.ID }</code> |
|||
</td> |
|||
<td> |
|||
<code>{ worker.Worker.Address }</code> |
|||
</td> |
|||
<td> |
|||
if worker.Worker.Status == "active" { |
|||
<span class="badge bg-success">Active</span> |
|||
} else if worker.Worker.Status == "busy" { |
|||
<span class="badge bg-warning">Busy</span> |
|||
} else { |
|||
<span class="badge bg-danger">Inactive</span> |
|||
} |
|||
</td> |
|||
<td> |
|||
<div class="d-flex flex-wrap gap-1"> |
|||
for _, capability := range worker.Worker.Capabilities { |
|||
<span class="badge bg-secondary rounded-pill">{ string(capability) }</span> |
|||
} |
|||
</div> |
|||
</td> |
|||
<td> |
|||
<div class="progress" style="height: 20px;"> |
|||
if worker.Worker.MaxConcurrent > 0 { |
|||
<div class="progress-bar" role="progressbar" |
|||
style={ fmt.Sprintf("width: %d%%", (worker.Worker.CurrentLoad*100)/worker.Worker.MaxConcurrent) } |
|||
aria-valuenow={ fmt.Sprintf("%d", worker.Worker.CurrentLoad) } |
|||
aria-valuemin="0" |
|||
aria-valuemax={ fmt.Sprintf("%d", worker.Worker.MaxConcurrent) }> |
|||
{ fmt.Sprintf("%d/%d", worker.Worker.CurrentLoad, worker.Worker.MaxConcurrent) } |
|||
</div> |
|||
} else { |
|||
<div class="progress-bar" role="progressbar" style="width: 0%">0/0</div> |
|||
} |
|||
</div> |
|||
</td> |
|||
<td> |
|||
{ fmt.Sprintf("%d", len(worker.CurrentTasks)) } |
|||
</td> |
|||
<td> |
|||
<small> |
|||
<div>✅ { fmt.Sprintf("%d", worker.Performance.TasksCompleted) }</div> |
|||
<div>❌ { fmt.Sprintf("%d", worker.Performance.TasksFailed) }</div> |
|||
<div>📊 { fmt.Sprintf("%.1f%%", worker.Performance.SuccessRate) }</div> |
|||
</small> |
|||
</td> |
|||
<td> |
|||
if time.Since(worker.Worker.LastHeartbeat) < 2*time.Minute { |
|||
<span class="text-success"> |
|||
<i class="fas fa-heartbeat"></i> |
|||
{ worker.Worker.LastHeartbeat.Format("15:04:05") } |
|||
</span> |
|||
} else { |
|||
<span class="text-danger"> |
|||
<i class="fas fa-exclamation-triangle"></i> |
|||
{ worker.Worker.LastHeartbeat.Format("15:04:05") } |
|||
</span> |
|||
} |
|||
</td> |
|||
<td> |
|||
<div class="btn-group btn-group-sm" role="group"> |
|||
<button type="button" class="btn btn-outline-info" onclick={ templ.ComponentScript{Call: "showWorkerDetails"} } data-worker-id={ worker.Worker.ID }> |
|||
<i class="fas fa-info-circle"></i> |
|||
</button> |
|||
if worker.Worker.Status == "active" { |
|||
<button type="button" class="btn btn-outline-warning" onclick={ templ.ComponentScript{Call: "pauseWorker"} } data-worker-id={ worker.Worker.ID }> |
|||
<i class="fas fa-pause"></i> |
|||
</button> |
|||
} |
|||
</div> |
|||
</td> |
|||
</tr> |
|||
} |
|||
</tbody> |
|||
</table> |
|||
</div> |
|||
} |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<!-- Worker Details Modal --> |
|||
<div class="modal fade" id="workerDetailsModal" tabindex="-1" aria-labelledby="workerDetailsModalLabel" aria-hidden="true"> |
|||
<div class="modal-dialog modal-lg"> |
|||
<div class="modal-content"> |
|||
<div class="modal-header"> |
|||
<h5 class="modal-title" id="workerDetailsModalLabel">Worker Details</h5> |
|||
<button type="button" class="btn-close" data-bs-dismiss="modal" aria-label="Close"></button> |
|||
</div> |
|||
<div class="modal-body" id="workerDetailsContent"> |
|||
<!-- Content will be loaded dynamically --> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<script> |
|||
function showWorkerDetails(event) { |
|||
const workerID = event.target.closest('button').getAttribute('data-worker-id'); |
|||
|
|||
// Show modal |
|||
var modal = new bootstrap.Modal(document.getElementById('workerDetailsModal')); |
|||
|
|||
// Load worker details |
|||
fetch(`/api/maintenance/workers/${workerID}`) |
|||
.then(response => response.json()) |
|||
.then(data => { |
|||
const content = document.getElementById('workerDetailsContent'); |
|||
content.innerHTML = ` |
|||
<div class="row"> |
|||
<div class="col-md-6"> |
|||
<h6>Worker Information</h6> |
|||
<ul class="list-unstyled"> |
|||
<li><strong>ID:</strong> ${data.worker.id}</li> |
|||
<li><strong>Address:</strong> ${data.worker.address}</li> |
|||
<li><strong>Status:</strong> ${data.worker.status}</li> |
|||
<li><strong>Max Concurrent:</strong> ${data.worker.max_concurrent}</li> |
|||
<li><strong>Current Load:</strong> ${data.worker.current_load}</li> |
|||
</ul> |
|||
</div> |
|||
<div class="col-md-6"> |
|||
<h6>Performance Metrics</h6> |
|||
<ul class="list-unstyled"> |
|||
<li><strong>Tasks Completed:</strong> ${data.performance.tasks_completed}</li> |
|||
<li><strong>Tasks Failed:</strong> ${data.performance.tasks_failed}</li> |
|||
<li><strong>Success Rate:</strong> ${data.performance.success_rate.toFixed(1)}%</li> |
|||
<li><strong>Average Task Time:</strong> ${formatDuration(data.performance.average_task_time)}</li> |
|||
<li><strong>Uptime:</strong> ${formatDuration(data.performance.uptime)}</li> |
|||
</ul> |
|||
</div> |
|||
</div> |
|||
<hr> |
|||
<h6>Current Tasks</h6> |
|||
${data.current_tasks.length === 0 ? |
|||
'<p class="text-muted">No current tasks</p>' : |
|||
data.current_tasks.map(task => ` |
|||
<div class="card mb-2"> |
|||
<div class="card-body py-2"> |
|||
<div class="d-flex justify-content-between"> |
|||
<span><strong>${task.type}</strong> - Volume ${task.volume_id}</span> |
|||
<span class="badge bg-info">${task.status}</span> |
|||
</div> |
|||
<small class="text-muted">${task.reason}</small> |
|||
</div> |
|||
</div> |
|||
`).join('') |
|||
} |
|||
`; |
|||
modal.show(); |
|||
}) |
|||
.catch(error => { |
|||
console.error('Error loading worker details:', error); |
|||
const content = document.getElementById('workerDetailsContent'); |
|||
content.innerHTML = '<div class="alert alert-danger">Failed to load worker details</div>'; |
|||
modal.show(); |
|||
}); |
|||
} |
|||
|
|||
function pauseWorker(event) { |
|||
const workerID = event.target.closest('button').getAttribute('data-worker-id'); |
|||
|
|||
if (confirm('Are you sure you want to pause this worker?')) { |
|||
fetch(`/api/maintenance/workers/${workerID}/pause`, { |
|||
method: 'POST' |
|||
}) |
|||
.then(response => response.json()) |
|||
.then(data => { |
|||
if (data.success) { |
|||
location.reload(); |
|||
} else { |
|||
alert('Failed to pause worker: ' + data.error); |
|||
} |
|||
}) |
|||
.catch(error => { |
|||
console.error('Error pausing worker:', error); |
|||
alert('Failed to pause worker'); |
|||
}); |
|||
} |
|||
} |
|||
|
|||
function formatDuration(nanoseconds) { |
|||
const seconds = Math.floor(nanoseconds / 1000000000); |
|||
const minutes = Math.floor(seconds / 60); |
|||
const hours = Math.floor(minutes / 60); |
|||
|
|||
if (hours > 0) { |
|||
return `${hours}h ${minutes % 60}m`; |
|||
} else if (minutes > 0) { |
|||
return `${minutes}m ${seconds % 60}s`; |
|||
} else { |
|||
return `${seconds}s`; |
|||
} |
|||
} |
|||
</script> |
|||
} |
|||
431
weed/admin/view/app/maintenance_workers_templ.go
File diff suppressed because it is too large
View File
File diff suppressed because it is too large
View File
@ -0,0 +1,160 @@ |
|||
package app |
|||
|
|||
import ( |
|||
"github.com/seaweedfs/seaweedfs/weed/admin/maintenance" |
|||
) |
|||
|
|||
templ TaskConfig(data *maintenance.TaskConfigData) { |
|||
<div class="container-fluid"> |
|||
<div class="row mb-4"> |
|||
<div class="col-12"> |
|||
<div class="d-flex justify-content-between align-items-center"> |
|||
<h2 class="mb-0"> |
|||
<i class={data.TaskIcon + " me-2"}></i> |
|||
{data.TaskName} Configuration |
|||
</h2> |
|||
<div class="btn-group"> |
|||
<a href="/maintenance/config" class="btn btn-outline-secondary"> |
|||
<i class="fas fa-arrow-left me-1"></i> |
|||
Back to Configuration |
|||
</a> |
|||
<a href="/maintenance" class="btn btn-outline-primary"> |
|||
<i class="fas fa-list me-1"></i> |
|||
View Queue |
|||
</a> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<div class="row"> |
|||
<div class="col-12"> |
|||
<div class="card"> |
|||
<div class="card-header"> |
|||
<h5 class="mb-0"> |
|||
<i class={data.TaskIcon + " me-2"}></i> |
|||
{data.TaskName} Settings |
|||
</h5> |
|||
</div> |
|||
<div class="card-body"> |
|||
<p class="text-muted mb-4">{data.Description}</p> |
|||
|
|||
<!-- Task-specific configuration form --> |
|||
<form method="POST"> |
|||
<div class="task-config-form"> |
|||
@templ.Raw(string(data.ConfigFormHTML)) |
|||
</div> |
|||
|
|||
<hr class="my-4"> |
|||
|
|||
<div class="d-flex gap-2"> |
|||
<button type="submit" class="btn btn-primary"> |
|||
<i class="fas fa-save me-1"></i> |
|||
Save Configuration |
|||
</button> |
|||
<button type="button" class="btn btn-secondary" onclick="resetForm()"> |
|||
<i class="fas fa-undo me-1"></i> |
|||
Reset to Defaults |
|||
</button> |
|||
<a href="/maintenance/config" class="btn btn-outline-secondary"> |
|||
<i class="fas fa-times me-1"></i> |
|||
Cancel |
|||
</a> |
|||
</div> |
|||
</form> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<!-- Task Information --> |
|||
<div class="row mt-4"> |
|||
<div class="col-12"> |
|||
<div class="card"> |
|||
<div class="card-header"> |
|||
<h5 class="mb-0"> |
|||
<i class="fas fa-info-circle me-2"></i> |
|||
Task Information |
|||
</h5> |
|||
</div> |
|||
<div class="card-body"> |
|||
<div class="row"> |
|||
<div class="col-md-6"> |
|||
<h6 class="text-muted">Task Type</h6> |
|||
<p class="mb-3"> |
|||
<span class="badge bg-secondary">{string(data.TaskType)}</span> |
|||
</p> |
|||
</div> |
|||
<div class="col-md-6"> |
|||
<h6 class="text-muted">Display Name</h6> |
|||
<p class="mb-3">{data.TaskName}</p> |
|||
</div> |
|||
</div> |
|||
<div class="row"> |
|||
<div class="col-12"> |
|||
<h6 class="text-muted">Description</h6> |
|||
<p class="mb-0">{data.Description}</p> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<script> |
|||
function resetForm() { |
|||
if (confirm('Are you sure you want to reset all settings to their default values?')) { |
|||
// Find all form inputs and reset them |
|||
const form = document.querySelector('form'); |
|||
if (form) { |
|||
form.reset(); |
|||
} |
|||
} |
|||
} |
|||
|
|||
// Auto-save form data to localStorage for recovery |
|||
document.addEventListener('DOMContentLoaded', function() { |
|||
const form = document.querySelector('form'); |
|||
if (form) { |
|||
const taskType = '{string(data.TaskType)}'; |
|||
const storageKey = 'taskConfig_' + taskType; |
|||
|
|||
// Load saved data |
|||
const savedData = localStorage.getItem(storageKey); |
|||
if (savedData) { |
|||
try { |
|||
const data = JSON.parse(savedData); |
|||
Object.keys(data).forEach(key => { |
|||
const input = form.querySelector(`[name="${key}"]`); |
|||
if (input) { |
|||
if (input.type === 'checkbox') { |
|||
input.checked = data[key]; |
|||
} else { |
|||
input.value = data[key]; |
|||
} |
|||
} |
|||
}); |
|||
} catch (e) { |
|||
console.warn('Failed to load saved configuration:', e); |
|||
} |
|||
} |
|||
|
|||
// Save data on input change |
|||
form.addEventListener('input', function() { |
|||
const formData = new FormData(form); |
|||
const data = {}; |
|||
for (let [key, value] of formData.entries()) { |
|||
data[key] = value; |
|||
} |
|||
localStorage.setItem(storageKey, JSON.stringify(data)); |
|||
}); |
|||
|
|||
// Clear saved data on successful submit |
|||
form.addEventListener('submit', function() { |
|||
localStorage.removeItem(storageKey); |
|||
}); |
|||
} |
|||
}); |
|||
</script> |
|||
} |
|||
@ -0,0 +1,174 @@ |
|||
// 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 ( |
|||
"github.com/seaweedfs/seaweedfs/weed/admin/maintenance" |
|||
) |
|||
|
|||
func TaskConfig(data *maintenance.TaskConfigData) 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 mb-4\"><div class=\"col-12\"><div class=\"d-flex justify-content-between align-items-center\"><h2 class=\"mb-0\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var2 = []any{data.TaskIcon + " me-2"} |
|||
templ_7745c5c3_Err = templ.RenderCSSItems(ctx, templ_7745c5c3_Buffer, templ_7745c5c3_Var2...) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 2, "<i class=\"") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var3 string |
|||
templ_7745c5c3_Var3, templ_7745c5c3_Err = templ.JoinStringErrs(templ.CSSClasses(templ_7745c5c3_Var2).String()) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config.templ`, Line: 1, Col: 0} |
|||
} |
|||
_, 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, "\"></i> ") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var4 string |
|||
templ_7745c5c3_Var4, templ_7745c5c3_Err = templ.JoinStringErrs(data.TaskName) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config.templ`, Line: 14, Col: 38} |
|||
} |
|||
_, 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, " Configuration</h2><div class=\"btn-group\"><a href=\"/maintenance/config\" class=\"btn btn-outline-secondary\"><i class=\"fas fa-arrow-left me-1\"></i> Back to Configuration</a> <a href=\"/maintenance\" class=\"btn btn-outline-primary\"><i class=\"fas fa-list me-1\"></i> View Queue</a></div></div></div></div><div class=\"row\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header\"><h5 class=\"mb-0\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var5 = []any{data.TaskIcon + " me-2"} |
|||
templ_7745c5c3_Err = templ.RenderCSSItems(ctx, templ_7745c5c3_Buffer, templ_7745c5c3_Var5...) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 5, "<i class=\"") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var6 string |
|||
templ_7745c5c3_Var6, templ_7745c5c3_Err = templ.JoinStringErrs(templ.CSSClasses(templ_7745c5c3_Var5).String()) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config.templ`, Line: 1, Col: 0} |
|||
} |
|||
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var6)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 6, "\"></i> ") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var7 string |
|||
templ_7745c5c3_Var7, templ_7745c5c3_Err = templ.JoinStringErrs(data.TaskName) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config.templ`, Line: 36, Col: 42} |
|||
} |
|||
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var7)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 7, " Settings</h5></div><div class=\"card-body\"><p class=\"text-muted mb-4\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var8 string |
|||
templ_7745c5c3_Var8, templ_7745c5c3_Err = templ.JoinStringErrs(data.Description) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config.templ`, Line: 40, Col: 68} |
|||
} |
|||
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var8)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 8, "</p><!-- Task-specific configuration form --><form method=\"POST\"><div class=\"task-config-form\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
templ_7745c5c3_Err = templ.Raw(string(data.ConfigFormHTML)).Render(ctx, templ_7745c5c3_Buffer) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 9, "</div><hr class=\"my-4\"><div class=\"d-flex gap-2\"><button type=\"submit\" class=\"btn btn-primary\"><i class=\"fas fa-save me-1\"></i> Save Configuration</button> <button type=\"button\" class=\"btn btn-secondary\" onclick=\"resetForm()\"><i class=\"fas fa-undo me-1\"></i> Reset to Defaults</button> <a href=\"/maintenance/config\" class=\"btn btn-outline-secondary\"><i class=\"fas fa-times me-1\"></i> Cancel</a></div></form></div></div></div></div><!-- Task Information --><div class=\"row mt-4\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header\"><h5 class=\"mb-0\"><i class=\"fas fa-info-circle me-2\"></i> Task Information</h5></div><div class=\"card-body\"><div class=\"row\"><div class=\"col-md-6\"><h6 class=\"text-muted\">Task Type</h6><p class=\"mb-3\"><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(string(data.TaskType)) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config.templ`, Line: 85, Col: 91} |
|||
} |
|||
_, 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, 10, "</span></p></div><div class=\"col-md-6\"><h6 class=\"text-muted\">Display Name</h6><p class=\"mb-3\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var10 string |
|||
templ_7745c5c3_Var10, templ_7745c5c3_Err = templ.JoinStringErrs(data.TaskName) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config.templ`, Line: 90, Col: 62} |
|||
} |
|||
_, 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, 11, "</p></div></div><div class=\"row\"><div class=\"col-12\"><h6 class=\"text-muted\">Description</h6><p class=\"mb-0\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var11 string |
|||
templ_7745c5c3_Var11, templ_7745c5c3_Err = templ.JoinStringErrs(data.Description) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config.templ`, Line: 96, Col: 65} |
|||
} |
|||
_, 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, 12, "</p></div></div></div></div></div></div></div><script>\n function resetForm() {\n if (confirm('Are you sure you want to reset all settings to their default values?')) {\n // Find all form inputs and reset them\n const form = document.querySelector('form');\n if (form) {\n form.reset();\n }\n }\n }\n\n // Auto-save form data to localStorage for recovery\n document.addEventListener('DOMContentLoaded', function() {\n const form = document.querySelector('form');\n if (form) {\n const taskType = '{string(data.TaskType)}';\n const storageKey = 'taskConfig_' + taskType;\n\n // Load saved data\n const savedData = localStorage.getItem(storageKey);\n if (savedData) {\n try {\n const data = JSON.parse(savedData);\n Object.keys(data).forEach(key => {\n const input = form.querySelector(`[name=\"${key}\"]`);\n if (input) {\n if (input.type === 'checkbox') {\n input.checked = data[key];\n } else {\n input.value = data[key];\n }\n }\n });\n } catch (e) {\n console.warn('Failed to load saved configuration:', e);\n }\n }\n\n // Save data on input change\n form.addEventListener('input', function() {\n const formData = new FormData(form);\n const data = {};\n for (let [key, value] of formData.entries()) {\n data[key] = value;\n }\n localStorage.setItem(storageKey, JSON.stringify(data));\n });\n\n // Clear saved data on successful submit\n form.addEventListener('submit', function() {\n localStorage.removeItem(storageKey);\n });\n }\n });\n </script>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
return nil |
|||
}) |
|||
} |
|||
|
|||
var _ = templruntime.GeneratedTemplate |
|||
@ -0,0 +1,160 @@ |
|||
package app |
|||
|
|||
import ( |
|||
"github.com/seaweedfs/seaweedfs/weed/admin/maintenance" |
|||
"github.com/seaweedfs/seaweedfs/weed/admin/view/components" |
|||
) |
|||
|
|||
// TaskConfigTemplData represents data for templ-based task configuration |
|||
type TaskConfigTemplData struct { |
|||
TaskType maintenance.MaintenanceTaskType |
|||
TaskName string |
|||
TaskIcon string |
|||
Description string |
|||
ConfigSections []components.ConfigSectionData |
|||
} |
|||
|
|||
templ TaskConfigTempl(data *TaskConfigTemplData) { |
|||
<div class="container-fluid"> |
|||
<div class="row mb-4"> |
|||
<div class="col-12"> |
|||
<div class="d-flex justify-content-between align-items-center"> |
|||
<h2 class="mb-0"> |
|||
<i class={data.TaskIcon + " me-2"}></i> |
|||
{data.TaskName} Configuration |
|||
</h2> |
|||
<div class="btn-group"> |
|||
<a href="/maintenance/config" class="btn btn-outline-secondary"> |
|||
<i class="fas fa-arrow-left me-1"></i> |
|||
Back to Configuration |
|||
</a> |
|||
<a href="/maintenance/queue" class="btn btn-outline-info"> |
|||
<i class="fas fa-list me-1"></i> |
|||
View Queue |
|||
</a> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<div class="row mb-4"> |
|||
<div class="col-12"> |
|||
<div class="alert alert-info" role="alert"> |
|||
<i class="fas fa-info-circle me-2"></i> |
|||
{data.Description} |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<form method="POST" class="needs-validation" novalidate> |
|||
<!-- Render all configuration sections --> |
|||
for _, section := range data.ConfigSections { |
|||
@components.ConfigSection(section) |
|||
} |
|||
|
|||
<!-- Form actions --> |
|||
<div class="row"> |
|||
<div class="col-12"> |
|||
<div class="card"> |
|||
<div class="card-body"> |
|||
<div class="d-flex justify-content-between"> |
|||
<div> |
|||
<button type="submit" class="btn btn-primary"> |
|||
<i class="fas fa-save me-1"></i> |
|||
Save Configuration |
|||
</button> |
|||
<button type="button" class="btn btn-outline-secondary ms-2" onclick="resetForm()"> |
|||
<i class="fas fa-undo me-1"></i> |
|||
Reset |
|||
</button> |
|||
</div> |
|||
<div> |
|||
<button type="button" class="btn btn-outline-info" onclick="testConfiguration()"> |
|||
<i class="fas fa-play me-1"></i> |
|||
Test Configuration |
|||
</button> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</form> |
|||
</div> |
|||
|
|||
<script> |
|||
// Form validation |
|||
(function() { |
|||
'use strict'; |
|||
window.addEventListener('load', function() { |
|||
var forms = document.getElementsByClassName('needs-validation'); |
|||
var validation = Array.prototype.filter.call(forms, function(form) { |
|||
form.addEventListener('submit', function(event) { |
|||
if (form.checkValidity() === false) { |
|||
event.preventDefault(); |
|||
event.stopPropagation(); |
|||
} |
|||
form.classList.add('was-validated'); |
|||
}, false); |
|||
}); |
|||
}, false); |
|||
})(); |
|||
|
|||
// Auto-save functionality |
|||
let autoSaveTimeout; |
|||
function autoSave() { |
|||
clearTimeout(autoSaveTimeout); |
|||
autoSaveTimeout = setTimeout(function() { |
|||
const formData = new FormData(document.querySelector('form')); |
|||
localStorage.setItem('task_config_' + '{data.TaskType}', JSON.stringify(Object.fromEntries(formData))); |
|||
}, 1000); |
|||
} |
|||
|
|||
// Add auto-save listeners to all form inputs |
|||
document.addEventListener('DOMContentLoaded', function() { |
|||
const form = document.querySelector('form'); |
|||
if (form) { |
|||
form.addEventListener('input', autoSave); |
|||
form.addEventListener('change', autoSave); |
|||
} |
|||
}); |
|||
|
|||
// Reset form function |
|||
function resetForm() { |
|||
if (confirm('Are you sure you want to reset all changes?')) { |
|||
location.reload(); |
|||
} |
|||
} |
|||
|
|||
// Test configuration function |
|||
function testConfiguration() { |
|||
const formData = new FormData(document.querySelector('form')); |
|||
|
|||
// Show loading state |
|||
const testBtn = document.querySelector('button[onclick="testConfiguration()"]'); |
|||
const originalContent = testBtn.innerHTML; |
|||
testBtn.innerHTML = '<i class="fas fa-spinner fa-spin me-1"></i>Testing...'; |
|||
testBtn.disabled = true; |
|||
|
|||
fetch('/maintenance/config/{data.TaskType}/test', { |
|||
method: 'POST', |
|||
body: formData |
|||
}) |
|||
.then(response => response.json()) |
|||
.then(data => { |
|||
if (data.success) { |
|||
alert('Configuration test successful!'); |
|||
} else { |
|||
alert('Configuration test failed: ' + data.error); |
|||
} |
|||
}) |
|||
.catch(error => { |
|||
alert('Test failed: ' + error); |
|||
}) |
|||
.finally(() => { |
|||
testBtn.innerHTML = originalContent; |
|||
testBtn.disabled = false; |
|||
}); |
|||
} |
|||
</script> |
|||
} |
|||
@ -0,0 +1,112 @@ |
|||
// 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 ( |
|||
"github.com/seaweedfs/seaweedfs/weed/admin/maintenance" |
|||
"github.com/seaweedfs/seaweedfs/weed/admin/view/components" |
|||
) |
|||
|
|||
// TaskConfigTemplData represents data for templ-based task configuration
|
|||
type TaskConfigTemplData struct { |
|||
TaskType maintenance.MaintenanceTaskType |
|||
TaskName string |
|||
TaskIcon string |
|||
Description string |
|||
ConfigSections []components.ConfigSectionData |
|||
} |
|||
|
|||
func TaskConfigTempl(data *TaskConfigTemplData) 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 mb-4\"><div class=\"col-12\"><div class=\"d-flex justify-content-between align-items-center\"><h2 class=\"mb-0\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var2 = []any{data.TaskIcon + " me-2"} |
|||
templ_7745c5c3_Err = templ.RenderCSSItems(ctx, templ_7745c5c3_Buffer, templ_7745c5c3_Var2...) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 2, "<i class=\"") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var3 string |
|||
templ_7745c5c3_Var3, templ_7745c5c3_Err = templ.JoinStringErrs(templ.CSSClasses(templ_7745c5c3_Var2).String()) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_templ.templ`, Line: 1, Col: 0} |
|||
} |
|||
_, 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, "\"></i> ") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var4 string |
|||
templ_7745c5c3_Var4, templ_7745c5c3_Err = templ.JoinStringErrs(data.TaskName) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_templ.templ`, Line: 24, Col: 38} |
|||
} |
|||
_, 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, " Configuration</h2><div class=\"btn-group\"><a href=\"/maintenance/config\" class=\"btn btn-outline-secondary\"><i class=\"fas fa-arrow-left me-1\"></i> Back to Configuration</a> <a href=\"/maintenance/queue\" class=\"btn btn-outline-info\"><i class=\"fas fa-list me-1\"></i> View Queue</a></div></div></div></div><div class=\"row mb-4\"><div class=\"col-12\"><div class=\"alert alert-info\" role=\"alert\"><i class=\"fas fa-info-circle me-2\"></i> ") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var5 string |
|||
templ_7745c5c3_Var5, templ_7745c5c3_Err = templ.JoinStringErrs(data.Description) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_templ.templ`, Line: 44, Col: 37} |
|||
} |
|||
_, 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, "</div></div></div><form method=\"POST\" class=\"needs-validation\" novalidate><!-- Render all configuration sections -->") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
for _, section := range data.ConfigSections { |
|||
templ_7745c5c3_Err = components.ConfigSection(section).Render(ctx, templ_7745c5c3_Buffer) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 6, "<!-- Form actions --><div class=\"row\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-body\"><div class=\"d-flex justify-content-between\"><div><button type=\"submit\" class=\"btn btn-primary\"><i class=\"fas fa-save me-1\"></i> Save Configuration</button> <button type=\"button\" class=\"btn btn-outline-secondary ms-2\" onclick=\"resetForm()\"><i class=\"fas fa-undo me-1\"></i> Reset</button></div><div><button type=\"button\" class=\"btn btn-outline-info\" onclick=\"testConfiguration()\"><i class=\"fas fa-play me-1\"></i> Test Configuration</button></div></div></div></div></div></div></form></div><script>\n // Form validation\n (function() {\n 'use strict';\n window.addEventListener('load', function() {\n var forms = document.getElementsByClassName('needs-validation');\n var validation = Array.prototype.filter.call(forms, function(form) {\n form.addEventListener('submit', function(event) {\n if (form.checkValidity() === false) {\n event.preventDefault();\n event.stopPropagation();\n }\n form.classList.add('was-validated');\n }, false);\n });\n }, false);\n })();\n\n // Auto-save functionality\n let autoSaveTimeout;\n function autoSave() {\n clearTimeout(autoSaveTimeout);\n autoSaveTimeout = setTimeout(function() {\n const formData = new FormData(document.querySelector('form'));\n localStorage.setItem('task_config_' + '{data.TaskType}', JSON.stringify(Object.fromEntries(formData)));\n }, 1000);\n }\n\n // Add auto-save listeners to all form inputs\n document.addEventListener('DOMContentLoaded', function() {\n const form = document.querySelector('form');\n if (form) {\n form.addEventListener('input', autoSave);\n form.addEventListener('change', autoSave);\n }\n });\n\n // Reset form function\n function resetForm() {\n if (confirm('Are you sure you want to reset all changes?')) {\n location.reload();\n }\n }\n\n // Test configuration function\n function testConfiguration() {\n const formData = new FormData(document.querySelector('form'));\n \n // Show loading state\n const testBtn = document.querySelector('button[onclick=\"testConfiguration()\"]');\n const originalContent = testBtn.innerHTML;\n testBtn.innerHTML = '<i class=\"fas fa-spinner fa-spin me-1\"></i>Testing...';\n testBtn.disabled = true;\n \n fetch('/maintenance/config/{data.TaskType}/test', {\n method: 'POST',\n body: formData\n })\n .then(response => response.json())\n .then(data => {\n if (data.success) {\n alert('Configuration test successful!');\n } else {\n alert('Configuration test failed: ' + data.error);\n }\n })\n .catch(error => {\n alert('Test failed: ' + error);\n })\n .finally(() => {\n testBtn.innerHTML = originalContent;\n testBtn.disabled = false;\n });\n }\n </script>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
return nil |
|||
}) |
|||
} |
|||
|
|||
var _ = templruntime.GeneratedTemplate |
|||
@ -0,0 +1,83 @@ |
|||
package components |
|||
|
|||
|
|||
|
|||
// ConfigSectionData represents data for a configuration section |
|||
type ConfigSectionData struct { |
|||
Title string |
|||
Icon string |
|||
Description string |
|||
Fields []interface{} // Will hold field data structures |
|||
} |
|||
|
|||
// InfoSectionData represents data for an informational section |
|||
type InfoSectionData struct { |
|||
Title string |
|||
Icon string |
|||
Type string // "info", "warning", "success", "danger" |
|||
Content string |
|||
} |
|||
|
|||
// ConfigSection renders a Bootstrap card for configuration settings |
|||
templ ConfigSection(data ConfigSectionData) { |
|||
<div class="row"> |
|||
<div class="col-12"> |
|||
<div class="card mb-4"> |
|||
<div class="card-header"> |
|||
<h5 class="mb-0"> |
|||
if data.Icon != "" { |
|||
<i class={ data.Icon + " me-2" }></i> |
|||
} |
|||
{ data.Title } |
|||
</h5> |
|||
if data.Description != "" { |
|||
<small class="text-muted">{ data.Description }</small> |
|||
} |
|||
</div> |
|||
<div class="card-body"> |
|||
for _, field := range data.Fields { |
|||
switch v := field.(type) { |
|||
case TextFieldData: |
|||
@TextField(v) |
|||
case NumberFieldData: |
|||
@NumberField(v) |
|||
case CheckboxFieldData: |
|||
@CheckboxField(v) |
|||
case SelectFieldData: |
|||
@SelectField(v) |
|||
case DurationFieldData: |
|||
@DurationField(v) |
|||
case DurationInputFieldData: |
|||
@DurationInputField(v) |
|||
} |
|||
} |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
} |
|||
|
|||
// InfoSection renders a Bootstrap alert section for informational content |
|||
templ InfoSection(data InfoSectionData) { |
|||
<div class="row"> |
|||
<div class="col-12"> |
|||
<div class="card mb-3"> |
|||
<div class="card-header"> |
|||
<h5 class="mb-0"> |
|||
if data.Icon != "" { |
|||
<i class={ data.Icon + " me-2" }></i> |
|||
} |
|||
{ data.Title } |
|||
</h5> |
|||
</div> |
|||
<div class="card-body"> |
|||
<div class={ "alert alert-" + data.Type } role="alert"> |
|||
{data.Content} |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
} |
|||
|
|||
|
|||
@ -0,0 +1,257 @@ |
|||
// Code generated by templ - DO NOT EDIT.
|
|||
|
|||
// templ: version: v0.3.833
|
|||
package components |
|||
|
|||
//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" |
|||
|
|||
// ConfigSectionData represents data for a configuration section
|
|||
type ConfigSectionData struct { |
|||
Title string |
|||
Icon string |
|||
Description string |
|||
Fields []interface{} // Will hold field data structures
|
|||
} |
|||
|
|||
// InfoSectionData represents data for an informational section
|
|||
type InfoSectionData struct { |
|||
Title string |
|||
Icon string |
|||
Type string // "info", "warning", "success", "danger"
|
|||
Content string |
|||
} |
|||
|
|||
// ConfigSection renders a Bootstrap card for configuration settings
|
|||
func ConfigSection(data ConfigSectionData) 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=\"row\"><div class=\"col-12\"><div class=\"card mb-4\"><div class=\"card-header\"><h5 class=\"mb-0\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
if data.Icon != "" { |
|||
var templ_7745c5c3_Var2 = []any{data.Icon + " me-2"} |
|||
templ_7745c5c3_Err = templ.RenderCSSItems(ctx, templ_7745c5c3_Buffer, templ_7745c5c3_Var2...) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 2, "<i class=\"") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var3 string |
|||
templ_7745c5c3_Var3, templ_7745c5c3_Err = templ.JoinStringErrs(templ.CSSClasses(templ_7745c5c3_Var2).String()) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/components/config_sections.templ`, Line: 1, Col: 0} |
|||
} |
|||
_, 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, "\"></i> ") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
} |
|||
var templ_7745c5c3_Var4 string |
|||
templ_7745c5c3_Var4, templ_7745c5c3_Err = templ.JoinStringErrs(data.Title) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/components/config_sections.templ`, Line: 31, Col: 36} |
|||
} |
|||
_, 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, "</h5>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
if data.Description != "" { |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 5, "<small class=\"text-muted\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var5 string |
|||
templ_7745c5c3_Var5, templ_7745c5c3_Err = templ.JoinStringErrs(data.Description) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/components/config_sections.templ`, Line: 34, Col: 68} |
|||
} |
|||
_, 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, 6, "</small>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 7, "</div><div class=\"card-body\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
for _, field := range data.Fields { |
|||
switch v := field.(type) { |
|||
case TextFieldData: |
|||
templ_7745c5c3_Err = TextField(v).Render(ctx, templ_7745c5c3_Buffer) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
case NumberFieldData: |
|||
templ_7745c5c3_Err = NumberField(v).Render(ctx, templ_7745c5c3_Buffer) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
case CheckboxFieldData: |
|||
templ_7745c5c3_Err = CheckboxField(v).Render(ctx, templ_7745c5c3_Buffer) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
case SelectFieldData: |
|||
templ_7745c5c3_Err = SelectField(v).Render(ctx, templ_7745c5c3_Buffer) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
case DurationFieldData: |
|||
templ_7745c5c3_Err = DurationField(v).Render(ctx, templ_7745c5c3_Buffer) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
case DurationInputFieldData: |
|||
templ_7745c5c3_Err = DurationInputField(v).Render(ctx, templ_7745c5c3_Buffer) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
} |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 8, "</div></div></div></div>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
return nil |
|||
}) |
|||
} |
|||
|
|||
// InfoSection renders a Bootstrap alert section for informational content
|
|||
func InfoSection(data InfoSectionData) 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_Var6 := templ.GetChildren(ctx) |
|||
if templ_7745c5c3_Var6 == nil { |
|||
templ_7745c5c3_Var6 = templ.NopComponent |
|||
} |
|||
ctx = templ.ClearChildren(ctx) |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 9, "<div class=\"row\"><div class=\"col-12\"><div class=\"card mb-3\"><div class=\"card-header\"><h5 class=\"mb-0\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
if data.Icon != "" { |
|||
var templ_7745c5c3_Var7 = []any{data.Icon + " me-2"} |
|||
templ_7745c5c3_Err = templ.RenderCSSItems(ctx, templ_7745c5c3_Buffer, templ_7745c5c3_Var7...) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 10, "<i class=\"") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var8 string |
|||
templ_7745c5c3_Var8, templ_7745c5c3_Err = templ.JoinStringErrs(templ.CSSClasses(templ_7745c5c3_Var7).String()) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/components/config_sections.templ`, Line: 1, Col: 0} |
|||
} |
|||
_, 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, "\"></i> ") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
} |
|||
var templ_7745c5c3_Var9 string |
|||
templ_7745c5c3_Var9, templ_7745c5c3_Err = templ.JoinStringErrs(data.Title) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/components/config_sections.templ`, Line: 70, Col: 36} |
|||
} |
|||
_, 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, 12, "</h5></div><div class=\"card-body\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var10 = []any{"alert alert-" + data.Type} |
|||
templ_7745c5c3_Err = templ.RenderCSSItems(ctx, templ_7745c5c3_Buffer, templ_7745c5c3_Var10...) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, "<div class=\"") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var11 string |
|||
templ_7745c5c3_Var11, templ_7745c5c3_Err = templ.JoinStringErrs(templ.CSSClasses(templ_7745c5c3_Var10).String()) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/components/config_sections.templ`, Line: 1, Col: 0} |
|||
} |
|||
_, 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, 14, "\" role=\"alert\">") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
var templ_7745c5c3_Var12 string |
|||
templ_7745c5c3_Var12, templ_7745c5c3_Err = templ.JoinStringErrs(data.Content) |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/components/config_sections.templ`, Line: 75, Col: 37} |
|||
} |
|||
_, 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, 15, "</div></div></div></div></div>") |
|||
if templ_7745c5c3_Err != nil { |
|||
return templ_7745c5c3_Err |
|||
} |
|||
return nil |
|||
}) |
|||
} |
|||
|
|||
var _ = templruntime.GeneratedTemplate |
|||
@ -0,0 +1,306 @@ |
|||
package components |
|||
|
|||
import "fmt" |
|||
|
|||
// FormFieldData represents common form field data |
|||
type FormFieldData struct { |
|||
Name string |
|||
Label string |
|||
Description string |
|||
Required bool |
|||
} |
|||
|
|||
// TextFieldData represents text input field data |
|||
type TextFieldData struct { |
|||
FormFieldData |
|||
Value string |
|||
Placeholder string |
|||
} |
|||
|
|||
// NumberFieldData represents number input field data |
|||
type NumberFieldData struct { |
|||
FormFieldData |
|||
Value float64 |
|||
Step string |
|||
Min *float64 |
|||
Max *float64 |
|||
} |
|||
|
|||
// CheckboxFieldData represents checkbox field data |
|||
type CheckboxFieldData struct { |
|||
FormFieldData |
|||
Checked bool |
|||
} |
|||
|
|||
// SelectFieldData represents select field data |
|||
type SelectFieldData struct { |
|||
FormFieldData |
|||
Value string |
|||
Options []SelectOption |
|||
} |
|||
|
|||
type SelectOption struct { |
|||
Value string |
|||
Label string |
|||
} |
|||
|
|||
// DurationFieldData represents duration input field data |
|||
type DurationFieldData struct { |
|||
FormFieldData |
|||
Value string |
|||
Placeholder string |
|||
} |
|||
|
|||
// DurationInputFieldData represents duration input with number + unit dropdown |
|||
type DurationInputFieldData struct { |
|||
FormFieldData |
|||
Seconds int // The duration value in seconds |
|||
} |
|||
|
|||
// TextField renders a Bootstrap text input field |
|||
templ TextField(data TextFieldData) { |
|||
<div class="mb-3"> |
|||
<label for={ data.Name } class="form-label"> |
|||
{ data.Label } |
|||
if data.Required { |
|||
<span class="text-danger">*</span> |
|||
} |
|||
</label> |
|||
<input |
|||
type="text" |
|||
class="form-control" |
|||
id={ data.Name } |
|||
name={ data.Name } |
|||
value={ data.Value } |
|||
if data.Placeholder != "" { |
|||
placeholder={ data.Placeholder } |
|||
} |
|||
if data.Required { |
|||
required |
|||
} |
|||
/> |
|||
if data.Description != "" { |
|||
<div class="form-text text-muted">{ data.Description }</div> |
|||
} |
|||
</div> |
|||
} |
|||
|
|||
// NumberField renders a Bootstrap number input field |
|||
templ NumberField(data NumberFieldData) { |
|||
<div class="mb-3"> |
|||
<label for={ data.Name } class="form-label"> |
|||
{ data.Label } |
|||
if data.Required { |
|||
<span class="text-danger">*</span> |
|||
} |
|||
</label> |
|||
<input |
|||
type="number" |
|||
class="form-control" |
|||
id={ data.Name } |
|||
name={ data.Name } |
|||
value={ fmt.Sprintf("%.6g", data.Value) } |
|||
if data.Step != "" { |
|||
step={ data.Step } |
|||
} else { |
|||
step="any" |
|||
} |
|||
if data.Min != nil { |
|||
min={ fmt.Sprintf("%.6g", *data.Min) } |
|||
} |
|||
if data.Max != nil { |
|||
max={ fmt.Sprintf("%.6g", *data.Max) } |
|||
} |
|||
if data.Required { |
|||
required |
|||
} |
|||
/> |
|||
if data.Description != "" { |
|||
<div class="form-text text-muted">{ data.Description }</div> |
|||
} |
|||
</div> |
|||
} |
|||
|
|||
// CheckboxField renders a Bootstrap checkbox field |
|||
templ CheckboxField(data CheckboxFieldData) { |
|||
<div class="mb-3"> |
|||
<div class="form-check"> |
|||
<input |
|||
type="checkbox" |
|||
class="form-check-input" |
|||
id={ data.Name } |
|||
name={ data.Name } |
|||
if data.Checked { |
|||
checked |
|||
} |
|||
/> |
|||
<label class="form-check-label" for={ data.Name }> |
|||
{ data.Label } |
|||
</label> |
|||
</div> |
|||
if data.Description != "" { |
|||
<div class="form-text text-muted">{ data.Description }</div> |
|||
} |
|||
</div> |
|||
} |
|||
|
|||
// SelectField renders a Bootstrap select field |
|||
templ SelectField(data SelectFieldData) { |
|||
<div class="mb-3"> |
|||
<label for={ data.Name } class="form-label"> |
|||
{ data.Label } |
|||
if data.Required { |
|||
<span class="text-danger">*</span> |
|||
} |
|||
</label> |
|||
<select |
|||
class="form-select" |
|||
id={ data.Name } |
|||
name={ data.Name } |
|||
if data.Required { |
|||
required |
|||
} |
|||
> |
|||
for _, option := range data.Options { |
|||
<option |
|||
value={ option.Value } |
|||
if option.Value == data.Value { |
|||
selected |
|||
} |
|||
> |
|||
{ option.Label } |
|||
</option> |
|||
} |
|||
</select> |
|||
if data.Description != "" { |
|||
<div class="form-text text-muted">{ data.Description }</div> |
|||
} |
|||
</div> |
|||
} |
|||
|
|||
// DurationField renders a Bootstrap duration input field |
|||
templ DurationField(data DurationFieldData) { |
|||
<div class="mb-3"> |
|||
<label for={ data.Name } class="form-label"> |
|||
{ data.Label } |
|||
if data.Required { |
|||
<span class="text-danger">*</span> |
|||
} |
|||
</label> |
|||
<input |
|||
type="text" |
|||
class="form-control" |
|||
id={ data.Name } |
|||
name={ data.Name } |
|||
value={ data.Value } |
|||
if data.Placeholder != "" { |
|||
placeholder={ data.Placeholder } |
|||
} else { |
|||
placeholder="e.g., 30m, 2h, 24h" |
|||
} |
|||
if data.Required { |
|||
required |
|||
} |
|||
/> |
|||
if data.Description != "" { |
|||
<div class="form-text text-muted">{ data.Description }</div> |
|||
} |
|||
</div> |
|||
} |
|||
|
|||
// DurationInputField renders a Bootstrap duration input with number + unit dropdown |
|||
templ DurationInputField(data DurationInputFieldData) { |
|||
<div class="mb-3"> |
|||
<label for={ data.Name } class="form-label"> |
|||
{ data.Label } |
|||
if data.Required { |
|||
<span class="text-danger">*</span> |
|||
} |
|||
</label> |
|||
<div class="input-group"> |
|||
<input |
|||
type="number" |
|||
class="form-control" |
|||
id={ data.Name } |
|||
name={ data.Name } |
|||
value={ fmt.Sprintf("%.0f", convertSecondsToValue(data.Seconds, convertSecondsToUnit(data.Seconds))) } |
|||
step="1" |
|||
min="1" |
|||
if data.Required { |
|||
required |
|||
} |
|||
/> |
|||
<select |
|||
class="form-select" |
|||
id={ data.Name + "_unit" } |
|||
name={ data.Name + "_unit" } |
|||
style="max-width: 120px;" |
|||
> |
|||
<option |
|||
value="minutes" |
|||
if convertSecondsToUnit(data.Seconds) == "minutes" { |
|||
selected |
|||
} |
|||
> |
|||
Minutes |
|||
</option> |
|||
<option |
|||
value="hours" |
|||
if convertSecondsToUnit(data.Seconds) == "hours" { |
|||
selected |
|||
} |
|||
> |
|||
Hours |
|||
</option> |
|||
<option |
|||
value="days" |
|||
if convertSecondsToUnit(data.Seconds) == "days" { |
|||
selected |
|||
} |
|||
> |
|||
Days |
|||
</option> |
|||
</select> |
|||
</div> |
|||
if data.Description != "" { |
|||
<div class="form-text text-muted">{ data.Description }</div> |
|||
} |
|||
</div> |
|||
} |
|||
|
|||
// Helper functions for duration conversion (used by DurationInputField) |
|||
func convertSecondsToUnit(seconds int) string { |
|||
if seconds == 0 { |
|||
return "minutes" |
|||
} |
|||
|
|||
// Try days first |
|||
if seconds%(24*3600) == 0 && seconds >= 24*3600 { |
|||
return "days" |
|||
} |
|||
|
|||
// Try hours |
|||
if seconds%3600 == 0 && seconds >= 3600 { |
|||
return "hours" |
|||
} |
|||
|
|||
// Default to minutes |
|||
return "minutes" |
|||
} |
|||
|
|||
func convertSecondsToValue(seconds int, unit string) float64 { |
|||
if seconds == 0 { |
|||
return 0 |
|||
} |
|||
|
|||
switch unit { |
|||
case "days": |
|||
return float64(seconds / (24 * 3600)) |
|||
case "hours": |
|||
return float64(seconds / 3600) |
|||
case "minutes": |
|||
return float64(seconds / 60) |
|||
default: |
|||
return float64(seconds / 60) // Default to minutes |
|||
} |
|||
} |
|||
1104
weed/admin/view/components/form_fields_templ.go
File diff suppressed because it is too large
View File
File diff suppressed because it is too large
View File
@ -0,0 +1,47 @@ |
|||
package layout |
|||
|
|||
import ( |
|||
"github.com/seaweedfs/seaweedfs/weed/admin/maintenance" |
|||
|
|||
// Import task packages to trigger their auto-registration
|
|||
_ "github.com/seaweedfs/seaweedfs/weed/worker/tasks/balance" |
|||
_ "github.com/seaweedfs/seaweedfs/weed/worker/tasks/erasure_coding" |
|||
_ "github.com/seaweedfs/seaweedfs/weed/worker/tasks/vacuum" |
|||
) |
|||
|
|||
// MenuItemData represents a menu item
|
|||
type MenuItemData struct { |
|||
Name string |
|||
URL string |
|||
Icon string |
|||
Description string |
|||
} |
|||
|
|||
// GetConfigurationMenuItems returns the dynamic configuration menu items
|
|||
func GetConfigurationMenuItems() []*MenuItemData { |
|||
var menuItems []*MenuItemData |
|||
|
|||
// Add system configuration item
|
|||
menuItems = append(menuItems, &MenuItemData{ |
|||
Name: "System", |
|||
URL: "/maintenance/config", |
|||
Icon: "fas fa-cogs", |
|||
Description: "System-level configuration", |
|||
}) |
|||
|
|||
// Get all registered task types and add them as submenu items
|
|||
registeredTypes := maintenance.GetRegisteredMaintenanceTaskTypes() |
|||
|
|||
for _, taskType := range registeredTypes { |
|||
menuItem := &MenuItemData{ |
|||
Name: maintenance.GetTaskDisplayName(taskType), |
|||
URL: "/maintenance/config/" + string(taskType), |
|||
Icon: maintenance.GetTaskIcon(taskType), |
|||
Description: maintenance.GetTaskDescription(taskType), |
|||
} |
|||
|
|||
menuItems = append(menuItems, menuItem) |
|||
} |
|||
|
|||
return menuItems |
|||
} |
|||
@ -0,0 +1,182 @@ |
|||
package command |
|||
|
|||
import ( |
|||
"os" |
|||
"os/signal" |
|||
"strings" |
|||
"syscall" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/security" |
|||
"github.com/seaweedfs/seaweedfs/weed/util" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/tasks" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
|
|||
// Import task packages to trigger their auto-registration
|
|||
_ "github.com/seaweedfs/seaweedfs/weed/worker/tasks/balance" |
|||
_ "github.com/seaweedfs/seaweedfs/weed/worker/tasks/erasure_coding" |
|||
_ "github.com/seaweedfs/seaweedfs/weed/worker/tasks/vacuum" |
|||
) |
|||
|
|||
var cmdWorker = &Command{ |
|||
UsageLine: "worker -admin=<admin_server> [-capabilities=<task_types>] [-maxConcurrent=<num>]", |
|||
Short: "start a maintenance worker to process cluster maintenance tasks", |
|||
Long: `Start a maintenance worker that connects to an admin server to process |
|||
maintenance tasks like vacuum, erasure coding, remote upload, and replication fixes. |
|||
|
|||
The worker ID and address are automatically generated. |
|||
The worker connects to the admin server via gRPC (admin HTTP port + 10000). |
|||
|
|||
Examples: |
|||
weed worker -admin=localhost:23646 |
|||
weed worker -admin=admin.example.com:23646 |
|||
weed worker -admin=localhost:23646 -capabilities=vacuum,replication |
|||
weed worker -admin=localhost:23646 -maxConcurrent=4 |
|||
`, |
|||
} |
|||
|
|||
var ( |
|||
workerAdminServer = cmdWorker.Flag.String("admin", "localhost:23646", "admin server address") |
|||
workerCapabilities = cmdWorker.Flag.String("capabilities", "vacuum,ec,remote,replication,balance", "comma-separated list of task types this worker can handle") |
|||
workerMaxConcurrent = cmdWorker.Flag.Int("maxConcurrent", 2, "maximum number of concurrent tasks") |
|||
workerHeartbeatInterval = cmdWorker.Flag.Duration("heartbeat", 30*time.Second, "heartbeat interval") |
|||
workerTaskRequestInterval = cmdWorker.Flag.Duration("taskInterval", 5*time.Second, "task request interval") |
|||
) |
|||
|
|||
func init() { |
|||
cmdWorker.Run = runWorker |
|||
|
|||
// Set default capabilities from registered task types
|
|||
// This happens after package imports have triggered auto-registration
|
|||
tasks.SetDefaultCapabilitiesFromRegistry() |
|||
} |
|||
|
|||
func runWorker(cmd *Command, args []string) bool { |
|||
util.LoadConfiguration("security", false) |
|||
|
|||
glog.Infof("Starting maintenance worker") |
|||
glog.Infof("Admin server: %s", *workerAdminServer) |
|||
glog.Infof("Capabilities: %s", *workerCapabilities) |
|||
|
|||
// Parse capabilities
|
|||
capabilities := parseCapabilities(*workerCapabilities) |
|||
if len(capabilities) == 0 { |
|||
glog.Fatalf("No valid capabilities specified") |
|||
return false |
|||
} |
|||
|
|||
// Create worker configuration
|
|||
config := &types.WorkerConfig{ |
|||
AdminServer: *workerAdminServer, |
|||
Capabilities: capabilities, |
|||
MaxConcurrent: *workerMaxConcurrent, |
|||
HeartbeatInterval: *workerHeartbeatInterval, |
|||
TaskRequestInterval: *workerTaskRequestInterval, |
|||
} |
|||
|
|||
// Create worker instance
|
|||
workerInstance, err := worker.NewWorker(config) |
|||
if err != nil { |
|||
glog.Fatalf("Failed to create worker: %v", err) |
|||
return false |
|||
} |
|||
|
|||
// Create admin client with LoadClientTLS
|
|||
grpcDialOption := security.LoadClientTLS(util.GetViper(), "grpc.worker") |
|||
adminClient, err := worker.CreateAdminClient(*workerAdminServer, workerInstance.ID(), grpcDialOption) |
|||
if err != nil { |
|||
glog.Fatalf("Failed to create admin client: %v", err) |
|||
return false |
|||
} |
|||
|
|||
// Set admin client
|
|||
workerInstance.SetAdminClient(adminClient) |
|||
|
|||
// Start the worker
|
|||
err = workerInstance.Start() |
|||
if err != nil { |
|||
glog.Fatalf("Failed to start worker: %v", err) |
|||
return false |
|||
} |
|||
|
|||
// Set up signal handling
|
|||
sigChan := make(chan os.Signal, 1) |
|||
signal.Notify(sigChan, syscall.SIGINT, syscall.SIGTERM) |
|||
|
|||
glog.Infof("Maintenance worker %s started successfully", workerInstance.ID()) |
|||
glog.Infof("Press Ctrl+C to stop the worker") |
|||
|
|||
// Wait for shutdown signal
|
|||
<-sigChan |
|||
glog.Infof("Shutdown signal received, stopping worker...") |
|||
|
|||
// Gracefully stop the worker
|
|||
err = workerInstance.Stop() |
|||
if err != nil { |
|||
glog.Errorf("Error stopping worker: %v", err) |
|||
} |
|||
glog.Infof("Worker stopped") |
|||
|
|||
return true |
|||
} |
|||
|
|||
// parseCapabilities converts comma-separated capability string to task types
|
|||
func parseCapabilities(capabilityStr string) []types.TaskType { |
|||
if capabilityStr == "" { |
|||
return nil |
|||
} |
|||
|
|||
capabilityMap := map[string]types.TaskType{} |
|||
|
|||
// Populate capabilityMap with registered task types
|
|||
typesRegistry := tasks.GetGlobalTypesRegistry() |
|||
for taskType := range typesRegistry.GetAllDetectors() { |
|||
// Use the task type string directly as the key
|
|||
capabilityMap[strings.ToLower(string(taskType))] = taskType |
|||
} |
|||
|
|||
// Add common aliases for convenience
|
|||
if taskType, exists := capabilityMap["erasure_coding"]; exists { |
|||
capabilityMap["ec"] = taskType |
|||
} |
|||
if taskType, exists := capabilityMap["remote_upload"]; exists { |
|||
capabilityMap["remote"] = taskType |
|||
} |
|||
if taskType, exists := capabilityMap["fix_replication"]; exists { |
|||
capabilityMap["replication"] = taskType |
|||
} |
|||
|
|||
var capabilities []types.TaskType |
|||
parts := strings.Split(capabilityStr, ",") |
|||
|
|||
for _, part := range parts { |
|||
part = strings.TrimSpace(part) |
|||
if taskType, exists := capabilityMap[part]; exists { |
|||
capabilities = append(capabilities, taskType) |
|||
} else { |
|||
glog.Warningf("Unknown capability: %s", part) |
|||
} |
|||
} |
|||
|
|||
return capabilities |
|||
} |
|||
|
|||
// Legacy compatibility types for backward compatibility
|
|||
// These will be deprecated in future versions
|
|||
|
|||
// WorkerStatus represents the current status of a worker (deprecated)
|
|||
type WorkerStatus struct { |
|||
WorkerID string `json:"worker_id"` |
|||
Address string `json:"address"` |
|||
Status string `json:"status"` |
|||
Capabilities []types.TaskType `json:"capabilities"` |
|||
MaxConcurrent int `json:"max_concurrent"` |
|||
CurrentLoad int `json:"current_load"` |
|||
LastHeartbeat time.Time `json:"last_heartbeat"` |
|||
CurrentTasks []types.Task `json:"current_tasks"` |
|||
Uptime time.Duration `json:"uptime"` |
|||
TasksCompleted int `json:"tasks_completed"` |
|||
TasksFailed int `json:"tasks_failed"` |
|||
} |
|||
@ -0,0 +1,142 @@ |
|||
syntax = "proto3"; |
|||
|
|||
package worker_pb; |
|||
|
|||
option go_package = "github.com/seaweedfs/seaweedfs/weed/pb/worker_pb"; |
|||
|
|||
// WorkerService provides bidirectional communication between admin and worker |
|||
service WorkerService { |
|||
// WorkerStream maintains a bidirectional stream for worker communication |
|||
rpc WorkerStream(stream WorkerMessage) returns (stream AdminMessage); |
|||
} |
|||
|
|||
// WorkerMessage represents messages from worker to admin |
|||
message WorkerMessage { |
|||
string worker_id = 1; |
|||
int64 timestamp = 2; |
|||
|
|||
oneof message { |
|||
WorkerRegistration registration = 3; |
|||
WorkerHeartbeat heartbeat = 4; |
|||
TaskRequest task_request = 5; |
|||
TaskUpdate task_update = 6; |
|||
TaskComplete task_complete = 7; |
|||
WorkerShutdown shutdown = 8; |
|||
} |
|||
} |
|||
|
|||
// AdminMessage represents messages from admin to worker |
|||
message AdminMessage { |
|||
string admin_id = 1; |
|||
int64 timestamp = 2; |
|||
|
|||
oneof message { |
|||
RegistrationResponse registration_response = 3; |
|||
HeartbeatResponse heartbeat_response = 4; |
|||
TaskAssignment task_assignment = 5; |
|||
TaskCancellation task_cancellation = 6; |
|||
AdminShutdown admin_shutdown = 7; |
|||
} |
|||
} |
|||
|
|||
// WorkerRegistration message when worker connects |
|||
message WorkerRegistration { |
|||
string worker_id = 1; |
|||
string address = 2; |
|||
repeated string capabilities = 3; |
|||
int32 max_concurrent = 4; |
|||
map<string, string> metadata = 5; |
|||
} |
|||
|
|||
// RegistrationResponse confirms worker registration |
|||
message RegistrationResponse { |
|||
bool success = 1; |
|||
string message = 2; |
|||
string assigned_worker_id = 3; |
|||
} |
|||
|
|||
// WorkerHeartbeat sent periodically by worker |
|||
message WorkerHeartbeat { |
|||
string worker_id = 1; |
|||
string status = 2; |
|||
int32 current_load = 3; |
|||
int32 max_concurrent = 4; |
|||
repeated string current_task_ids = 5; |
|||
int32 tasks_completed = 6; |
|||
int32 tasks_failed = 7; |
|||
int64 uptime_seconds = 8; |
|||
} |
|||
|
|||
// HeartbeatResponse acknowledges heartbeat |
|||
message HeartbeatResponse { |
|||
bool success = 1; |
|||
string message = 2; |
|||
} |
|||
|
|||
// TaskRequest from worker asking for new tasks |
|||
message TaskRequest { |
|||
string worker_id = 1; |
|||
repeated string capabilities = 2; |
|||
int32 available_slots = 3; |
|||
} |
|||
|
|||
// TaskAssignment from admin to worker |
|||
message TaskAssignment { |
|||
string task_id = 1; |
|||
string task_type = 2; |
|||
TaskParams params = 3; |
|||
int32 priority = 4; |
|||
int64 created_time = 5; |
|||
map<string, string> metadata = 6; |
|||
} |
|||
|
|||
// TaskParams contains task-specific parameters |
|||
message TaskParams { |
|||
uint32 volume_id = 1; |
|||
string server = 2; |
|||
string collection = 3; |
|||
string data_center = 4; |
|||
string rack = 5; |
|||
repeated string replicas = 6; |
|||
map<string, string> parameters = 7; |
|||
} |
|||
|
|||
// TaskUpdate reports task progress |
|||
message TaskUpdate { |
|||
string task_id = 1; |
|||
string worker_id = 2; |
|||
string status = 3; |
|||
float progress = 4; |
|||
string message = 5; |
|||
map<string, string> metadata = 6; |
|||
} |
|||
|
|||
// TaskComplete reports task completion |
|||
message TaskComplete { |
|||
string task_id = 1; |
|||
string worker_id = 2; |
|||
bool success = 3; |
|||
string error_message = 4; |
|||
int64 completion_time = 5; |
|||
map<string, string> result_metadata = 6; |
|||
} |
|||
|
|||
// TaskCancellation from admin to cancel a task |
|||
message TaskCancellation { |
|||
string task_id = 1; |
|||
string reason = 2; |
|||
bool force = 3; |
|||
} |
|||
|
|||
// WorkerShutdown notifies admin that worker is shutting down |
|||
message WorkerShutdown { |
|||
string worker_id = 1; |
|||
string reason = 2; |
|||
repeated string pending_task_ids = 3; |
|||
} |
|||
|
|||
// AdminShutdown notifies worker that admin is shutting down |
|||
message AdminShutdown { |
|||
string reason = 1; |
|||
int32 graceful_shutdown_seconds = 2; |
|||
} |
|||
1724
weed/pb/worker_pb/worker.pb.go
File diff suppressed because it is too large
View File
File diff suppressed because it is too large
View File
@ -0,0 +1,121 @@ |
|||
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
|
|||
// versions:
|
|||
// - protoc-gen-go-grpc v1.5.1
|
|||
// - protoc v5.29.3
|
|||
// source: worker.proto
|
|||
|
|||
package worker_pb |
|||
|
|||
import ( |
|||
context "context" |
|||
grpc "google.golang.org/grpc" |
|||
codes "google.golang.org/grpc/codes" |
|||
status "google.golang.org/grpc/status" |
|||
) |
|||
|
|||
// This is a compile-time assertion to ensure that this generated file
|
|||
// is compatible with the grpc package it is being compiled against.
|
|||
// Requires gRPC-Go v1.64.0 or later.
|
|||
const _ = grpc.SupportPackageIsVersion9 |
|||
|
|||
const ( |
|||
WorkerService_WorkerStream_FullMethodName = "/worker_pb.WorkerService/WorkerStream" |
|||
) |
|||
|
|||
// WorkerServiceClient is the client API for WorkerService service.
|
|||
//
|
|||
// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream.
|
|||
//
|
|||
// WorkerService provides bidirectional communication between admin and worker
|
|||
type WorkerServiceClient interface { |
|||
// WorkerStream maintains a bidirectional stream for worker communication
|
|||
WorkerStream(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[WorkerMessage, AdminMessage], error) |
|||
} |
|||
|
|||
type workerServiceClient struct { |
|||
cc grpc.ClientConnInterface |
|||
} |
|||
|
|||
func NewWorkerServiceClient(cc grpc.ClientConnInterface) WorkerServiceClient { |
|||
return &workerServiceClient{cc} |
|||
} |
|||
|
|||
func (c *workerServiceClient) WorkerStream(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[WorkerMessage, AdminMessage], error) { |
|||
cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) |
|||
stream, err := c.cc.NewStream(ctx, &WorkerService_ServiceDesc.Streams[0], WorkerService_WorkerStream_FullMethodName, cOpts...) |
|||
if err != nil { |
|||
return nil, err |
|||
} |
|||
x := &grpc.GenericClientStream[WorkerMessage, AdminMessage]{ClientStream: stream} |
|||
return x, nil |
|||
} |
|||
|
|||
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
|
|||
type WorkerService_WorkerStreamClient = grpc.BidiStreamingClient[WorkerMessage, AdminMessage] |
|||
|
|||
// WorkerServiceServer is the server API for WorkerService service.
|
|||
// All implementations must embed UnimplementedWorkerServiceServer
|
|||
// for forward compatibility.
|
|||
//
|
|||
// WorkerService provides bidirectional communication between admin and worker
|
|||
type WorkerServiceServer interface { |
|||
// WorkerStream maintains a bidirectional stream for worker communication
|
|||
WorkerStream(grpc.BidiStreamingServer[WorkerMessage, AdminMessage]) error |
|||
mustEmbedUnimplementedWorkerServiceServer() |
|||
} |
|||
|
|||
// UnimplementedWorkerServiceServer must be embedded to have
|
|||
// forward compatible implementations.
|
|||
//
|
|||
// NOTE: this should be embedded by value instead of pointer to avoid a nil
|
|||
// pointer dereference when methods are called.
|
|||
type UnimplementedWorkerServiceServer struct{} |
|||
|
|||
func (UnimplementedWorkerServiceServer) WorkerStream(grpc.BidiStreamingServer[WorkerMessage, AdminMessage]) error { |
|||
return status.Errorf(codes.Unimplemented, "method WorkerStream not implemented") |
|||
} |
|||
func (UnimplementedWorkerServiceServer) mustEmbedUnimplementedWorkerServiceServer() {} |
|||
func (UnimplementedWorkerServiceServer) testEmbeddedByValue() {} |
|||
|
|||
// UnsafeWorkerServiceServer may be embedded to opt out of forward compatibility for this service.
|
|||
// Use of this interface is not recommended, as added methods to WorkerServiceServer will
|
|||
// result in compilation errors.
|
|||
type UnsafeWorkerServiceServer interface { |
|||
mustEmbedUnimplementedWorkerServiceServer() |
|||
} |
|||
|
|||
func RegisterWorkerServiceServer(s grpc.ServiceRegistrar, srv WorkerServiceServer) { |
|||
// If the following call pancis, it indicates UnimplementedWorkerServiceServer was
|
|||
// embedded by pointer and is nil. This will cause panics if an
|
|||
// unimplemented method is ever invoked, so we test this at initialization
|
|||
// time to prevent it from happening at runtime later due to I/O.
|
|||
if t, ok := srv.(interface{ testEmbeddedByValue() }); ok { |
|||
t.testEmbeddedByValue() |
|||
} |
|||
s.RegisterService(&WorkerService_ServiceDesc, srv) |
|||
} |
|||
|
|||
func _WorkerService_WorkerStream_Handler(srv interface{}, stream grpc.ServerStream) error { |
|||
return srv.(WorkerServiceServer).WorkerStream(&grpc.GenericServerStream[WorkerMessage, AdminMessage]{ServerStream: stream}) |
|||
} |
|||
|
|||
// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name.
|
|||
type WorkerService_WorkerStreamServer = grpc.BidiStreamingServer[WorkerMessage, AdminMessage] |
|||
|
|||
// WorkerService_ServiceDesc is the grpc.ServiceDesc for WorkerService service.
|
|||
// It's only intended for direct use with grpc.RegisterService,
|
|||
// and not to be introspected or modified (even as a copy)
|
|||
var WorkerService_ServiceDesc = grpc.ServiceDesc{ |
|||
ServiceName: "worker_pb.WorkerService", |
|||
HandlerType: (*WorkerServiceServer)(nil), |
|||
Methods: []grpc.MethodDesc{}, |
|||
Streams: []grpc.StreamDesc{ |
|||
{ |
|||
StreamName: "WorkerStream", |
|||
Handler: _WorkerService_WorkerStream_Handler, |
|||
ServerStreams: true, |
|||
ClientStreams: true, |
|||
}, |
|||
}, |
|||
Metadata: "worker.proto", |
|||
} |
|||
@ -0,0 +1,761 @@ |
|||
package worker |
|||
|
|||
import ( |
|||
"context" |
|||
"fmt" |
|||
"io" |
|||
"sync" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/pb" |
|||
"github.com/seaweedfs/seaweedfs/weed/pb/worker_pb" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
"google.golang.org/grpc" |
|||
) |
|||
|
|||
// GrpcAdminClient implements AdminClient using gRPC bidirectional streaming
|
|||
type GrpcAdminClient struct { |
|||
adminAddress string |
|||
workerID string |
|||
dialOption grpc.DialOption |
|||
|
|||
conn *grpc.ClientConn |
|||
client worker_pb.WorkerServiceClient |
|||
stream worker_pb.WorkerService_WorkerStreamClient |
|||
streamCtx context.Context |
|||
streamCancel context.CancelFunc |
|||
|
|||
connected bool |
|||
reconnecting bool |
|||
shouldReconnect bool |
|||
mutex sync.RWMutex |
|||
|
|||
// Reconnection parameters
|
|||
maxReconnectAttempts int |
|||
reconnectBackoff time.Duration |
|||
maxReconnectBackoff time.Duration |
|||
reconnectMultiplier float64 |
|||
|
|||
// Worker registration info for re-registration after reconnection
|
|||
lastWorkerInfo *types.Worker |
|||
|
|||
// Channels for communication
|
|||
outgoing chan *worker_pb.WorkerMessage |
|||
incoming chan *worker_pb.AdminMessage |
|||
responseChans map[string]chan *worker_pb.AdminMessage |
|||
responsesMutex sync.RWMutex |
|||
|
|||
// Shutdown channel
|
|||
shutdownChan chan struct{} |
|||
} |
|||
|
|||
// NewGrpcAdminClient creates a new gRPC admin client
|
|||
func NewGrpcAdminClient(adminAddress string, workerID string, dialOption grpc.DialOption) *GrpcAdminClient { |
|||
// Admin uses HTTP port + 10000 as gRPC port
|
|||
grpcAddress := pb.ServerToGrpcAddress(adminAddress) |
|||
|
|||
return &GrpcAdminClient{ |
|||
adminAddress: grpcAddress, |
|||
workerID: workerID, |
|||
dialOption: dialOption, |
|||
shouldReconnect: true, |
|||
maxReconnectAttempts: 0, // 0 means infinite attempts
|
|||
reconnectBackoff: 1 * time.Second, |
|||
maxReconnectBackoff: 30 * time.Second, |
|||
reconnectMultiplier: 1.5, |
|||
outgoing: make(chan *worker_pb.WorkerMessage, 100), |
|||
incoming: make(chan *worker_pb.AdminMessage, 100), |
|||
responseChans: make(map[string]chan *worker_pb.AdminMessage), |
|||
shutdownChan: make(chan struct{}), |
|||
} |
|||
} |
|||
|
|||
// Connect establishes gRPC connection to admin server with TLS detection
|
|||
func (c *GrpcAdminClient) Connect() error { |
|||
c.mutex.Lock() |
|||
defer c.mutex.Unlock() |
|||
|
|||
if c.connected { |
|||
return fmt.Errorf("already connected") |
|||
} |
|||
|
|||
// Detect TLS support and create appropriate connection
|
|||
conn, err := c.createConnection() |
|||
if err != nil { |
|||
return fmt.Errorf("failed to connect to admin server: %v", err) |
|||
} |
|||
|
|||
c.conn = conn |
|||
c.client = worker_pb.NewWorkerServiceClient(conn) |
|||
|
|||
// Create bidirectional stream
|
|||
c.streamCtx, c.streamCancel = context.WithCancel(context.Background()) |
|||
stream, err := c.client.WorkerStream(c.streamCtx) |
|||
if err != nil { |
|||
c.conn.Close() |
|||
return fmt.Errorf("failed to create worker stream: %v", err) |
|||
} |
|||
|
|||
c.stream = stream |
|||
c.connected = true |
|||
|
|||
// Start stream handlers and reconnection loop
|
|||
go c.handleOutgoing() |
|||
go c.handleIncoming() |
|||
go c.reconnectionLoop() |
|||
|
|||
glog.Infof("Connected to admin server at %s", c.adminAddress) |
|||
return nil |
|||
} |
|||
|
|||
// createConnection attempts to connect using the provided dial option
|
|||
func (c *GrpcAdminClient) createConnection() (*grpc.ClientConn, error) { |
|||
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) |
|||
defer cancel() |
|||
|
|||
conn, err := pb.GrpcDial(ctx, c.adminAddress, false, c.dialOption) |
|||
if err != nil { |
|||
return nil, fmt.Errorf("failed to connect to admin server: %v", err) |
|||
} |
|||
|
|||
glog.Infof("Connected to admin server at %s", c.adminAddress) |
|||
return conn, nil |
|||
} |
|||
|
|||
// Disconnect closes the gRPC connection
|
|||
func (c *GrpcAdminClient) Disconnect() error { |
|||
c.mutex.Lock() |
|||
defer c.mutex.Unlock() |
|||
|
|||
if !c.connected { |
|||
return nil |
|||
} |
|||
|
|||
c.connected = false |
|||
c.shouldReconnect = false |
|||
|
|||
// Send shutdown signal to stop reconnection loop
|
|||
select { |
|||
case c.shutdownChan <- struct{}{}: |
|||
default: |
|||
} |
|||
|
|||
// Send shutdown message
|
|||
shutdownMsg := &worker_pb.WorkerMessage{ |
|||
WorkerId: c.workerID, |
|||
Timestamp: time.Now().Unix(), |
|||
Message: &worker_pb.WorkerMessage_Shutdown{ |
|||
Shutdown: &worker_pb.WorkerShutdown{ |
|||
WorkerId: c.workerID, |
|||
Reason: "normal shutdown", |
|||
}, |
|||
}, |
|||
} |
|||
|
|||
select { |
|||
case c.outgoing <- shutdownMsg: |
|||
case <-time.After(time.Second): |
|||
glog.Warningf("Failed to send shutdown message") |
|||
} |
|||
|
|||
// Cancel stream context
|
|||
if c.streamCancel != nil { |
|||
c.streamCancel() |
|||
} |
|||
|
|||
// Close stream
|
|||
if c.stream != nil { |
|||
c.stream.CloseSend() |
|||
} |
|||
|
|||
// Close connection
|
|||
if c.conn != nil { |
|||
c.conn.Close() |
|||
} |
|||
|
|||
// Close channels
|
|||
close(c.outgoing) |
|||
close(c.incoming) |
|||
|
|||
glog.Infof("Disconnected from admin server") |
|||
return nil |
|||
} |
|||
|
|||
// reconnectionLoop handles automatic reconnection with exponential backoff
|
|||
func (c *GrpcAdminClient) reconnectionLoop() { |
|||
backoff := c.reconnectBackoff |
|||
attempts := 0 |
|||
|
|||
for { |
|||
select { |
|||
case <-c.shutdownChan: |
|||
return |
|||
default: |
|||
} |
|||
|
|||
c.mutex.RLock() |
|||
shouldReconnect := c.shouldReconnect && !c.connected && !c.reconnecting |
|||
c.mutex.RUnlock() |
|||
|
|||
if !shouldReconnect { |
|||
time.Sleep(time.Second) |
|||
continue |
|||
} |
|||
|
|||
c.mutex.Lock() |
|||
c.reconnecting = true |
|||
c.mutex.Unlock() |
|||
|
|||
glog.Infof("Attempting to reconnect to admin server (attempt %d)", attempts+1) |
|||
|
|||
// Attempt to reconnect
|
|||
if err := c.reconnect(); err != nil { |
|||
attempts++ |
|||
glog.Errorf("Reconnection attempt %d failed: %v", attempts, err) |
|||
|
|||
// Reset reconnecting flag
|
|||
c.mutex.Lock() |
|||
c.reconnecting = false |
|||
c.mutex.Unlock() |
|||
|
|||
// Check if we should give up
|
|||
if c.maxReconnectAttempts > 0 && attempts >= c.maxReconnectAttempts { |
|||
glog.Errorf("Max reconnection attempts (%d) reached, giving up", c.maxReconnectAttempts) |
|||
c.mutex.Lock() |
|||
c.shouldReconnect = false |
|||
c.mutex.Unlock() |
|||
return |
|||
} |
|||
|
|||
// Wait with exponential backoff
|
|||
glog.Infof("Waiting %v before next reconnection attempt", backoff) |
|||
|
|||
select { |
|||
case <-c.shutdownChan: |
|||
return |
|||
case <-time.After(backoff): |
|||
} |
|||
|
|||
// Increase backoff
|
|||
backoff = time.Duration(float64(backoff) * c.reconnectMultiplier) |
|||
if backoff > c.maxReconnectBackoff { |
|||
backoff = c.maxReconnectBackoff |
|||
} |
|||
} else { |
|||
// Successful reconnection
|
|||
attempts = 0 |
|||
backoff = c.reconnectBackoff |
|||
glog.Infof("Successfully reconnected to admin server") |
|||
|
|||
c.mutex.Lock() |
|||
c.reconnecting = false |
|||
c.mutex.Unlock() |
|||
} |
|||
} |
|||
} |
|||
|
|||
// reconnect attempts to re-establish the connection
|
|||
func (c *GrpcAdminClient) reconnect() error { |
|||
// Clean up existing connection completely
|
|||
c.mutex.Lock() |
|||
if c.streamCancel != nil { |
|||
c.streamCancel() |
|||
} |
|||
if c.stream != nil { |
|||
c.stream.CloseSend() |
|||
} |
|||
if c.conn != nil { |
|||
c.conn.Close() |
|||
} |
|||
c.mutex.Unlock() |
|||
|
|||
// Create new connection
|
|||
conn, err := c.createConnection() |
|||
if err != nil { |
|||
return fmt.Errorf("failed to create connection: %v", err) |
|||
} |
|||
|
|||
client := worker_pb.NewWorkerServiceClient(conn) |
|||
|
|||
// Create new stream
|
|||
streamCtx, streamCancel := context.WithCancel(context.Background()) |
|||
stream, err := client.WorkerStream(streamCtx) |
|||
if err != nil { |
|||
conn.Close() |
|||
streamCancel() |
|||
return fmt.Errorf("failed to create stream: %v", err) |
|||
} |
|||
|
|||
// Update client state
|
|||
c.mutex.Lock() |
|||
c.conn = conn |
|||
c.client = client |
|||
c.stream = stream |
|||
c.streamCtx = streamCtx |
|||
c.streamCancel = streamCancel |
|||
c.connected = true |
|||
c.mutex.Unlock() |
|||
|
|||
// Restart stream handlers
|
|||
go c.handleOutgoing() |
|||
go c.handleIncoming() |
|||
|
|||
// Re-register worker if we have previous registration info
|
|||
c.mutex.RLock() |
|||
workerInfo := c.lastWorkerInfo |
|||
c.mutex.RUnlock() |
|||
|
|||
if workerInfo != nil { |
|||
glog.Infof("Re-registering worker after reconnection...") |
|||
if err := c.sendRegistration(workerInfo); err != nil { |
|||
glog.Errorf("Failed to re-register worker: %v", err) |
|||
// Don't fail the reconnection because of registration failure
|
|||
// The registration will be retried on next heartbeat or operation
|
|||
} |
|||
} |
|||
|
|||
return nil |
|||
} |
|||
|
|||
// handleOutgoing processes outgoing messages to admin
|
|||
func (c *GrpcAdminClient) handleOutgoing() { |
|||
for msg := range c.outgoing { |
|||
c.mutex.RLock() |
|||
connected := c.connected |
|||
stream := c.stream |
|||
c.mutex.RUnlock() |
|||
|
|||
if !connected { |
|||
break |
|||
} |
|||
|
|||
if err := stream.Send(msg); err != nil { |
|||
glog.Errorf("Failed to send message to admin: %v", err) |
|||
c.mutex.Lock() |
|||
c.connected = false |
|||
c.mutex.Unlock() |
|||
break |
|||
} |
|||
} |
|||
} |
|||
|
|||
// handleIncoming processes incoming messages from admin
|
|||
func (c *GrpcAdminClient) handleIncoming() { |
|||
for { |
|||
c.mutex.RLock() |
|||
connected := c.connected |
|||
stream := c.stream |
|||
c.mutex.RUnlock() |
|||
|
|||
if !connected { |
|||
break |
|||
} |
|||
|
|||
msg, err := stream.Recv() |
|||
if err != nil { |
|||
if err == io.EOF { |
|||
glog.Infof("Admin server closed the stream") |
|||
} else { |
|||
glog.Errorf("Failed to receive message from admin: %v", err) |
|||
} |
|||
c.mutex.Lock() |
|||
c.connected = false |
|||
c.mutex.Unlock() |
|||
break |
|||
} |
|||
|
|||
// Route message to waiting goroutines or general handler
|
|||
select { |
|||
case c.incoming <- msg: |
|||
case <-time.After(time.Second): |
|||
glog.Warningf("Incoming message buffer full, dropping message") |
|||
} |
|||
} |
|||
} |
|||
|
|||
// RegisterWorker registers the worker with the admin server
|
|||
func (c *GrpcAdminClient) RegisterWorker(worker *types.Worker) error { |
|||
if !c.connected { |
|||
return fmt.Errorf("not connected to admin server") |
|||
} |
|||
|
|||
// Store worker info for re-registration after reconnection
|
|||
c.mutex.Lock() |
|||
c.lastWorkerInfo = worker |
|||
c.mutex.Unlock() |
|||
|
|||
return c.sendRegistration(worker) |
|||
} |
|||
|
|||
// sendRegistration sends the registration message and waits for response
|
|||
func (c *GrpcAdminClient) sendRegistration(worker *types.Worker) error { |
|||
capabilities := make([]string, len(worker.Capabilities)) |
|||
for i, cap := range worker.Capabilities { |
|||
capabilities[i] = string(cap) |
|||
} |
|||
|
|||
msg := &worker_pb.WorkerMessage{ |
|||
WorkerId: c.workerID, |
|||
Timestamp: time.Now().Unix(), |
|||
Message: &worker_pb.WorkerMessage_Registration{ |
|||
Registration: &worker_pb.WorkerRegistration{ |
|||
WorkerId: c.workerID, |
|||
Address: worker.Address, |
|||
Capabilities: capabilities, |
|||
MaxConcurrent: int32(worker.MaxConcurrent), |
|||
Metadata: make(map[string]string), |
|||
}, |
|||
}, |
|||
} |
|||
|
|||
select { |
|||
case c.outgoing <- msg: |
|||
case <-time.After(5 * time.Second): |
|||
return fmt.Errorf("failed to send registration message: timeout") |
|||
} |
|||
|
|||
// Wait for registration response
|
|||
timeout := time.NewTimer(10 * time.Second) |
|||
defer timeout.Stop() |
|||
|
|||
for { |
|||
select { |
|||
case response := <-c.incoming: |
|||
if regResp := response.GetRegistrationResponse(); regResp != nil { |
|||
if regResp.Success { |
|||
glog.Infof("Worker registered successfully: %s", regResp.Message) |
|||
return nil |
|||
} |
|||
return fmt.Errorf("registration failed: %s", regResp.Message) |
|||
} |
|||
case <-timeout.C: |
|||
return fmt.Errorf("registration timeout") |
|||
} |
|||
} |
|||
} |
|||
|
|||
// SendHeartbeat sends heartbeat to admin server
|
|||
func (c *GrpcAdminClient) SendHeartbeat(workerID string, status *types.WorkerStatus) error { |
|||
if !c.connected { |
|||
// Wait for reconnection for a short time
|
|||
if err := c.waitForConnection(10 * time.Second); err != nil { |
|||
return fmt.Errorf("not connected to admin server: %v", err) |
|||
} |
|||
} |
|||
|
|||
taskIds := make([]string, len(status.CurrentTasks)) |
|||
for i, task := range status.CurrentTasks { |
|||
taskIds[i] = task.ID |
|||
} |
|||
|
|||
msg := &worker_pb.WorkerMessage{ |
|||
WorkerId: c.workerID, |
|||
Timestamp: time.Now().Unix(), |
|||
Message: &worker_pb.WorkerMessage_Heartbeat{ |
|||
Heartbeat: &worker_pb.WorkerHeartbeat{ |
|||
WorkerId: c.workerID, |
|||
Status: status.Status, |
|||
CurrentLoad: int32(status.CurrentLoad), |
|||
MaxConcurrent: int32(status.MaxConcurrent), |
|||
CurrentTaskIds: taskIds, |
|||
TasksCompleted: int32(status.TasksCompleted), |
|||
TasksFailed: int32(status.TasksFailed), |
|||
UptimeSeconds: int64(status.Uptime.Seconds()), |
|||
}, |
|||
}, |
|||
} |
|||
|
|||
select { |
|||
case c.outgoing <- msg: |
|||
return nil |
|||
case <-time.After(time.Second): |
|||
return fmt.Errorf("failed to send heartbeat: timeout") |
|||
} |
|||
} |
|||
|
|||
// RequestTask requests a new task from admin server
|
|||
func (c *GrpcAdminClient) RequestTask(workerID string, capabilities []types.TaskType) (*types.Task, error) { |
|||
if !c.connected { |
|||
// Wait for reconnection for a short time
|
|||
if err := c.waitForConnection(5 * time.Second); err != nil { |
|||
return nil, fmt.Errorf("not connected to admin server: %v", err) |
|||
} |
|||
} |
|||
|
|||
caps := make([]string, len(capabilities)) |
|||
for i, cap := range capabilities { |
|||
caps[i] = string(cap) |
|||
} |
|||
|
|||
msg := &worker_pb.WorkerMessage{ |
|||
WorkerId: c.workerID, |
|||
Timestamp: time.Now().Unix(), |
|||
Message: &worker_pb.WorkerMessage_TaskRequest{ |
|||
TaskRequest: &worker_pb.TaskRequest{ |
|||
WorkerId: c.workerID, |
|||
Capabilities: caps, |
|||
AvailableSlots: 1, // Request one task
|
|||
}, |
|||
}, |
|||
} |
|||
|
|||
select { |
|||
case c.outgoing <- msg: |
|||
case <-time.After(time.Second): |
|||
return nil, fmt.Errorf("failed to send task request: timeout") |
|||
} |
|||
|
|||
// Wait for task assignment
|
|||
timeout := time.NewTimer(5 * time.Second) |
|||
defer timeout.Stop() |
|||
|
|||
for { |
|||
select { |
|||
case response := <-c.incoming: |
|||
if taskAssign := response.GetTaskAssignment(); taskAssign != nil { |
|||
// Convert parameters map[string]string to map[string]interface{}
|
|||
parameters := make(map[string]interface{}) |
|||
for k, v := range taskAssign.Params.Parameters { |
|||
parameters[k] = v |
|||
} |
|||
|
|||
// Convert to our task type
|
|||
task := &types.Task{ |
|||
ID: taskAssign.TaskId, |
|||
Type: types.TaskType(taskAssign.TaskType), |
|||
Status: types.TaskStatusAssigned, |
|||
VolumeID: taskAssign.Params.VolumeId, |
|||
Server: taskAssign.Params.Server, |
|||
Collection: taskAssign.Params.Collection, |
|||
Priority: types.TaskPriority(taskAssign.Priority), |
|||
CreatedAt: time.Unix(taskAssign.CreatedTime, 0), |
|||
Parameters: parameters, |
|||
} |
|||
return task, nil |
|||
} |
|||
case <-timeout.C: |
|||
return nil, nil // No task available
|
|||
} |
|||
} |
|||
} |
|||
|
|||
// CompleteTask reports task completion to admin server
|
|||
func (c *GrpcAdminClient) CompleteTask(taskID string, success bool, errorMsg string) error { |
|||
if !c.connected { |
|||
// Wait for reconnection for a short time
|
|||
if err := c.waitForConnection(5 * time.Second); err != nil { |
|||
return fmt.Errorf("not connected to admin server: %v", err) |
|||
} |
|||
} |
|||
|
|||
msg := &worker_pb.WorkerMessage{ |
|||
WorkerId: c.workerID, |
|||
Timestamp: time.Now().Unix(), |
|||
Message: &worker_pb.WorkerMessage_TaskComplete{ |
|||
TaskComplete: &worker_pb.TaskComplete{ |
|||
TaskId: taskID, |
|||
WorkerId: c.workerID, |
|||
Success: success, |
|||
ErrorMessage: errorMsg, |
|||
CompletionTime: time.Now().Unix(), |
|||
}, |
|||
}, |
|||
} |
|||
|
|||
select { |
|||
case c.outgoing <- msg: |
|||
return nil |
|||
case <-time.After(time.Second): |
|||
return fmt.Errorf("failed to send task completion: timeout") |
|||
} |
|||
} |
|||
|
|||
// UpdateTaskProgress updates task progress to admin server
|
|||
func (c *GrpcAdminClient) UpdateTaskProgress(taskID string, progress float64) error { |
|||
if !c.connected { |
|||
// Wait for reconnection for a short time
|
|||
if err := c.waitForConnection(5 * time.Second); err != nil { |
|||
return fmt.Errorf("not connected to admin server: %v", err) |
|||
} |
|||
} |
|||
|
|||
msg := &worker_pb.WorkerMessage{ |
|||
WorkerId: c.workerID, |
|||
Timestamp: time.Now().Unix(), |
|||
Message: &worker_pb.WorkerMessage_TaskUpdate{ |
|||
TaskUpdate: &worker_pb.TaskUpdate{ |
|||
TaskId: taskID, |
|||
WorkerId: c.workerID, |
|||
Status: "in_progress", |
|||
Progress: float32(progress), |
|||
}, |
|||
}, |
|||
} |
|||
|
|||
select { |
|||
case c.outgoing <- msg: |
|||
return nil |
|||
case <-time.After(time.Second): |
|||
return fmt.Errorf("failed to send task progress: timeout") |
|||
} |
|||
} |
|||
|
|||
// IsConnected returns whether the client is connected
|
|||
func (c *GrpcAdminClient) IsConnected() bool { |
|||
c.mutex.RLock() |
|||
defer c.mutex.RUnlock() |
|||
return c.connected |
|||
} |
|||
|
|||
// IsReconnecting returns whether the client is currently attempting to reconnect
|
|||
func (c *GrpcAdminClient) IsReconnecting() bool { |
|||
c.mutex.RLock() |
|||
defer c.mutex.RUnlock() |
|||
return c.reconnecting |
|||
} |
|||
|
|||
// SetReconnectionSettings allows configuration of reconnection behavior
|
|||
func (c *GrpcAdminClient) SetReconnectionSettings(maxAttempts int, initialBackoff, maxBackoff time.Duration, multiplier float64) { |
|||
c.mutex.Lock() |
|||
defer c.mutex.Unlock() |
|||
c.maxReconnectAttempts = maxAttempts |
|||
c.reconnectBackoff = initialBackoff |
|||
c.maxReconnectBackoff = maxBackoff |
|||
c.reconnectMultiplier = multiplier |
|||
} |
|||
|
|||
// StopReconnection stops the reconnection loop
|
|||
func (c *GrpcAdminClient) StopReconnection() { |
|||
c.mutex.Lock() |
|||
defer c.mutex.Unlock() |
|||
c.shouldReconnect = false |
|||
} |
|||
|
|||
// StartReconnection starts the reconnection loop
|
|||
func (c *GrpcAdminClient) StartReconnection() { |
|||
c.mutex.Lock() |
|||
defer c.mutex.Unlock() |
|||
c.shouldReconnect = true |
|||
} |
|||
|
|||
// waitForConnection waits for the connection to be established or timeout
|
|||
func (c *GrpcAdminClient) waitForConnection(timeout time.Duration) error { |
|||
deadline := time.Now().Add(timeout) |
|||
|
|||
for time.Now().Before(deadline) { |
|||
c.mutex.RLock() |
|||
connected := c.connected |
|||
shouldReconnect := c.shouldReconnect |
|||
c.mutex.RUnlock() |
|||
|
|||
if connected { |
|||
return nil |
|||
} |
|||
|
|||
if !shouldReconnect { |
|||
return fmt.Errorf("reconnection is disabled") |
|||
} |
|||
|
|||
time.Sleep(100 * time.Millisecond) |
|||
} |
|||
|
|||
return fmt.Errorf("timeout waiting for connection") |
|||
} |
|||
|
|||
// MockAdminClient provides a mock implementation for testing
|
|||
type MockAdminClient struct { |
|||
workerID string |
|||
connected bool |
|||
tasks []*types.Task |
|||
mutex sync.RWMutex |
|||
} |
|||
|
|||
// NewMockAdminClient creates a new mock admin client
|
|||
func NewMockAdminClient() *MockAdminClient { |
|||
return &MockAdminClient{ |
|||
connected: true, |
|||
tasks: make([]*types.Task, 0), |
|||
} |
|||
} |
|||
|
|||
// Connect mock implementation
|
|||
func (m *MockAdminClient) Connect() error { |
|||
m.mutex.Lock() |
|||
defer m.mutex.Unlock() |
|||
m.connected = true |
|||
return nil |
|||
} |
|||
|
|||
// Disconnect mock implementation
|
|||
func (m *MockAdminClient) Disconnect() error { |
|||
m.mutex.Lock() |
|||
defer m.mutex.Unlock() |
|||
m.connected = false |
|||
return nil |
|||
} |
|||
|
|||
// RegisterWorker mock implementation
|
|||
func (m *MockAdminClient) RegisterWorker(worker *types.Worker) error { |
|||
m.workerID = worker.ID |
|||
glog.Infof("Mock: Worker %s registered with capabilities: %v", worker.ID, worker.Capabilities) |
|||
return nil |
|||
} |
|||
|
|||
// SendHeartbeat mock implementation
|
|||
func (m *MockAdminClient) SendHeartbeat(workerID string, status *types.WorkerStatus) error { |
|||
glog.V(2).Infof("Mock: Heartbeat from worker %s, status: %s, load: %d/%d", |
|||
workerID, status.Status, status.CurrentLoad, status.MaxConcurrent) |
|||
return nil |
|||
} |
|||
|
|||
// RequestTask mock implementation
|
|||
func (m *MockAdminClient) RequestTask(workerID string, capabilities []types.TaskType) (*types.Task, error) { |
|||
m.mutex.Lock() |
|||
defer m.mutex.Unlock() |
|||
|
|||
if len(m.tasks) > 0 { |
|||
task := m.tasks[0] |
|||
m.tasks = m.tasks[1:] |
|||
glog.Infof("Mock: Assigned task %s to worker %s", task.ID, workerID) |
|||
return task, nil |
|||
} |
|||
|
|||
// No tasks available
|
|||
return nil, nil |
|||
} |
|||
|
|||
// CompleteTask mock implementation
|
|||
func (m *MockAdminClient) CompleteTask(taskID string, success bool, errorMsg string) error { |
|||
if success { |
|||
glog.Infof("Mock: Task %s completed successfully", taskID) |
|||
} else { |
|||
glog.Infof("Mock: Task %s failed: %s", taskID, errorMsg) |
|||
} |
|||
return nil |
|||
} |
|||
|
|||
// UpdateTaskProgress mock implementation
|
|||
func (m *MockAdminClient) UpdateTaskProgress(taskID string, progress float64) error { |
|||
glog.V(2).Infof("Mock: Task %s progress: %.1f%%", taskID, progress) |
|||
return nil |
|||
} |
|||
|
|||
// IsConnected mock implementation
|
|||
func (m *MockAdminClient) IsConnected() bool { |
|||
m.mutex.RLock() |
|||
defer m.mutex.RUnlock() |
|||
return m.connected |
|||
} |
|||
|
|||
// AddMockTask adds a mock task for testing
|
|||
func (m *MockAdminClient) AddMockTask(task *types.Task) { |
|||
m.mutex.Lock() |
|||
defer m.mutex.Unlock() |
|||
m.tasks = append(m.tasks, task) |
|||
} |
|||
|
|||
// CreateAdminClient creates an admin client with the provided dial option
|
|||
func CreateAdminClient(adminServer string, workerID string, dialOption grpc.DialOption) (AdminClient, error) { |
|||
return NewGrpcAdminClient(adminServer, workerID, dialOption), nil |
|||
} |
|||
@ -0,0 +1,111 @@ |
|||
package worker |
|||
|
|||
import ( |
|||
"context" |
|||
"testing" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/pb" |
|||
"google.golang.org/grpc" |
|||
"google.golang.org/grpc/credentials/insecure" |
|||
) |
|||
|
|||
func TestGrpcConnection(t *testing.T) { |
|||
// Test that we can create a gRPC connection with insecure credentials
|
|||
// This tests the connection setup without requiring a running server
|
|||
adminAddress := "localhost:33646" // gRPC port for admin server on port 23646
|
|||
|
|||
// This should not fail with transport security errors
|
|||
conn, err := pb.GrpcDial(context.Background(), adminAddress, false, grpc.WithTransportCredentials(insecure.NewCredentials())) |
|||
if err != nil { |
|||
// Connection failure is expected when no server is running
|
|||
// But it should NOT be a transport security error
|
|||
if err.Error() == "grpc: no transport security set" { |
|||
t.Fatalf("Transport security error should not occur with insecure credentials: %v", err) |
|||
} |
|||
t.Logf("Connection failed as expected (no server running): %v", err) |
|||
} else { |
|||
// If connection succeeds, clean up
|
|||
conn.Close() |
|||
t.Log("Connection succeeded") |
|||
} |
|||
} |
|||
|
|||
func TestGrpcAdminClient_Connect(t *testing.T) { |
|||
// Test that the GrpcAdminClient can be created and attempt connection
|
|||
dialOption := grpc.WithTransportCredentials(insecure.NewCredentials()) |
|||
client := NewGrpcAdminClient("localhost:23646", "test-worker", dialOption) |
|||
|
|||
// This should not fail with transport security errors
|
|||
err := client.Connect() |
|||
if err != nil { |
|||
// Connection failure is expected when no server is running
|
|||
// But it should NOT be a transport security error
|
|||
if err.Error() == "grpc: no transport security set" { |
|||
t.Fatalf("Transport security error should not occur with insecure credentials: %v", err) |
|||
} |
|||
t.Logf("Connection failed as expected (no server running): %v", err) |
|||
} else { |
|||
// If connection succeeds, clean up
|
|||
client.Disconnect() |
|||
t.Log("Connection succeeded") |
|||
} |
|||
} |
|||
|
|||
func TestAdminAddressToGrpcAddress(t *testing.T) { |
|||
tests := []struct { |
|||
adminAddress string |
|||
expected string |
|||
}{ |
|||
{"localhost:9333", "localhost:19333"}, |
|||
{"localhost:23646", "localhost:33646"}, |
|||
{"admin.example.com:9333", "admin.example.com:19333"}, |
|||
{"127.0.0.1:8080", "127.0.0.1:18080"}, |
|||
} |
|||
|
|||
for _, test := range tests { |
|||
dialOption := grpc.WithTransportCredentials(insecure.NewCredentials()) |
|||
client := NewGrpcAdminClient(test.adminAddress, "test-worker", dialOption) |
|||
result := client.adminAddress |
|||
if result != test.expected { |
|||
t.Errorf("For admin address %s, expected gRPC address %s, got %s", |
|||
test.adminAddress, test.expected, result) |
|||
} |
|||
} |
|||
} |
|||
|
|||
func TestMockAdminClient(t *testing.T) { |
|||
// Test that the mock client works correctly
|
|||
client := NewMockAdminClient() |
|||
|
|||
// Should be able to connect/disconnect without errors
|
|||
err := client.Connect() |
|||
if err != nil { |
|||
t.Fatalf("Mock client connect failed: %v", err) |
|||
} |
|||
|
|||
if !client.IsConnected() { |
|||
t.Error("Mock client should be connected") |
|||
} |
|||
|
|||
err = client.Disconnect() |
|||
if err != nil { |
|||
t.Fatalf("Mock client disconnect failed: %v", err) |
|||
} |
|||
|
|||
if client.IsConnected() { |
|||
t.Error("Mock client should be disconnected") |
|||
} |
|||
} |
|||
|
|||
func TestCreateAdminClient(t *testing.T) { |
|||
// Test client creation
|
|||
dialOption := grpc.WithTransportCredentials(insecure.NewCredentials()) |
|||
client, err := CreateAdminClient("localhost:9333", "test-worker", dialOption) |
|||
if err != nil { |
|||
t.Fatalf("Failed to create admin client: %v", err) |
|||
} |
|||
|
|||
if client == nil { |
|||
t.Fatal("Client should not be nil") |
|||
} |
|||
} |
|||
@ -0,0 +1,146 @@ |
|||
package worker |
|||
|
|||
import ( |
|||
"strings" |
|||
"testing" |
|||
"time" |
|||
|
|||
"google.golang.org/grpc" |
|||
"google.golang.org/grpc/credentials/insecure" |
|||
) |
|||
|
|||
func TestGrpcClientTLSDetection(t *testing.T) { |
|||
// Test that the client can be created with a dial option
|
|||
dialOption := grpc.WithTransportCredentials(insecure.NewCredentials()) |
|||
client := NewGrpcAdminClient("localhost:33646", "test-worker", dialOption) |
|||
|
|||
// Test that the client has the correct dial option
|
|||
if client.dialOption == nil { |
|||
t.Error("Client should have a dial option") |
|||
} |
|||
|
|||
t.Logf("Client created successfully with dial option") |
|||
} |
|||
|
|||
func TestCreateAdminClientGrpc(t *testing.T) { |
|||
// Test client creation - admin server port gets transformed to gRPC port
|
|||
dialOption := grpc.WithTransportCredentials(insecure.NewCredentials()) |
|||
client, err := CreateAdminClient("localhost:23646", "test-worker", dialOption) |
|||
if err != nil { |
|||
t.Fatalf("Failed to create admin client: %v", err) |
|||
} |
|||
|
|||
if client == nil { |
|||
t.Fatal("Client should not be nil") |
|||
} |
|||
|
|||
// Verify it's the correct type
|
|||
grpcClient, ok := client.(*GrpcAdminClient) |
|||
if !ok { |
|||
t.Fatal("Client should be GrpcAdminClient type") |
|||
} |
|||
|
|||
// The admin address should be transformed to the gRPC port (HTTP + 10000)
|
|||
expectedAddress := "localhost:33646" // 23646 + 10000
|
|||
if grpcClient.adminAddress != expectedAddress { |
|||
t.Errorf("Expected admin address %s, got %s", expectedAddress, grpcClient.adminAddress) |
|||
} |
|||
|
|||
if grpcClient.workerID != "test-worker" { |
|||
t.Errorf("Expected worker ID test-worker, got %s", grpcClient.workerID) |
|||
} |
|||
} |
|||
|
|||
func TestConnectionTimeouts(t *testing.T) { |
|||
// Test that connections have proper timeouts
|
|||
// Use localhost with a port that's definitely closed
|
|||
dialOption := grpc.WithTransportCredentials(insecure.NewCredentials()) |
|||
client := NewGrpcAdminClient("localhost:1", "test-worker", dialOption) // Port 1 is reserved and won't be open
|
|||
|
|||
// Test that the connection creation fails when actually trying to use it
|
|||
start := time.Now() |
|||
err := client.Connect() // This should fail when trying to establish the stream
|
|||
duration := time.Since(start) |
|||
|
|||
if err == nil { |
|||
t.Error("Expected connection to closed port to fail") |
|||
} else { |
|||
t.Logf("Connection failed as expected: %v", err) |
|||
} |
|||
|
|||
// Should fail quickly but not too quickly
|
|||
if duration > 10*time.Second { |
|||
t.Errorf("Connection attempt took too long: %v", duration) |
|||
} |
|||
} |
|||
|
|||
func TestConnectionWithDialOption(t *testing.T) { |
|||
// Test that the connection uses the provided dial option
|
|||
dialOption := grpc.WithTransportCredentials(insecure.NewCredentials()) |
|||
client := NewGrpcAdminClient("localhost:1", "test-worker", dialOption) // Port 1 is reserved and won't be open
|
|||
|
|||
// Test the actual connection
|
|||
err := client.Connect() |
|||
if err == nil { |
|||
t.Error("Expected connection to closed port to fail") |
|||
client.Disconnect() // Clean up if it somehow succeeded
|
|||
} else { |
|||
t.Logf("Connection failed as expected: %v", err) |
|||
} |
|||
|
|||
// The error should indicate a connection failure
|
|||
if err != nil && err.Error() != "" { |
|||
t.Logf("Connection error message: %s", err.Error()) |
|||
// The error should contain connection-related terms
|
|||
if !strings.Contains(err.Error(), "connection") && !strings.Contains(err.Error(), "dial") { |
|||
t.Logf("Error message doesn't indicate connection issues: %s", err.Error()) |
|||
} |
|||
} |
|||
} |
|||
|
|||
func TestClientWithSecureDialOption(t *testing.T) { |
|||
// Test that the client correctly uses a secure dial option
|
|||
// This would normally use LoadClientTLS, but for testing we'll use insecure
|
|||
dialOption := grpc.WithTransportCredentials(insecure.NewCredentials()) |
|||
client := NewGrpcAdminClient("localhost:33646", "test-worker", dialOption) |
|||
|
|||
if client.dialOption == nil { |
|||
t.Error("Client should have a dial option") |
|||
} |
|||
|
|||
t.Logf("Client created successfully with dial option") |
|||
} |
|||
|
|||
func TestConnectionWithRealAddress(t *testing.T) { |
|||
// Test connection behavior with a real address that doesn't support gRPC
|
|||
dialOption := grpc.WithTransportCredentials(insecure.NewCredentials()) |
|||
client := NewGrpcAdminClient("www.google.com:80", "test-worker", dialOption) // HTTP port, not gRPC
|
|||
|
|||
err := client.Connect() |
|||
if err == nil { |
|||
t.Log("Connection succeeded unexpectedly") |
|||
client.Disconnect() |
|||
} else { |
|||
t.Logf("Connection failed as expected: %v", err) |
|||
} |
|||
} |
|||
|
|||
func TestDialOptionUsage(t *testing.T) { |
|||
// Test that the provided dial option is used for connections
|
|||
dialOption := grpc.WithTransportCredentials(insecure.NewCredentials()) |
|||
client := NewGrpcAdminClient("localhost:1", "test-worker", dialOption) // Port 1 won't support gRPC at all
|
|||
|
|||
// Verify the dial option is stored
|
|||
if client.dialOption == nil { |
|||
t.Error("Dial option should be stored in client") |
|||
} |
|||
|
|||
// Test connection fails appropriately
|
|||
err := client.Connect() |
|||
if err == nil { |
|||
t.Error("Connection should fail to non-gRPC port") |
|||
client.Disconnect() |
|||
} else { |
|||
t.Logf("Connection failed as expected: %v", err) |
|||
} |
|||
} |
|||
@ -0,0 +1,348 @@ |
|||
package worker |
|||
|
|||
import ( |
|||
"fmt" |
|||
"sync" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// Registry manages workers and their statistics
|
|||
type Registry struct { |
|||
workers map[string]*types.Worker |
|||
stats *types.RegistryStats |
|||
mutex sync.RWMutex |
|||
} |
|||
|
|||
// NewRegistry creates a new worker registry
|
|||
func NewRegistry() *Registry { |
|||
return &Registry{ |
|||
workers: make(map[string]*types.Worker), |
|||
stats: &types.RegistryStats{ |
|||
TotalWorkers: 0, |
|||
ActiveWorkers: 0, |
|||
BusyWorkers: 0, |
|||
IdleWorkers: 0, |
|||
TotalTasks: 0, |
|||
CompletedTasks: 0, |
|||
FailedTasks: 0, |
|||
StartTime: time.Now(), |
|||
}, |
|||
} |
|||
} |
|||
|
|||
// RegisterWorker registers a new worker
|
|||
func (r *Registry) RegisterWorker(worker *types.Worker) error { |
|||
r.mutex.Lock() |
|||
defer r.mutex.Unlock() |
|||
|
|||
if _, exists := r.workers[worker.ID]; exists { |
|||
return fmt.Errorf("worker %s already registered", worker.ID) |
|||
} |
|||
|
|||
r.workers[worker.ID] = worker |
|||
r.updateStats() |
|||
return nil |
|||
} |
|||
|
|||
// UnregisterWorker removes a worker from the registry
|
|||
func (r *Registry) UnregisterWorker(workerID string) error { |
|||
r.mutex.Lock() |
|||
defer r.mutex.Unlock() |
|||
|
|||
if _, exists := r.workers[workerID]; !exists { |
|||
return fmt.Errorf("worker %s not found", workerID) |
|||
} |
|||
|
|||
delete(r.workers, workerID) |
|||
r.updateStats() |
|||
return nil |
|||
} |
|||
|
|||
// GetWorker returns a worker by ID
|
|||
func (r *Registry) GetWorker(workerID string) (*types.Worker, bool) { |
|||
r.mutex.RLock() |
|||
defer r.mutex.RUnlock() |
|||
|
|||
worker, exists := r.workers[workerID] |
|||
return worker, exists |
|||
} |
|||
|
|||
// ListWorkers returns all registered workers
|
|||
func (r *Registry) ListWorkers() []*types.Worker { |
|||
r.mutex.RLock() |
|||
defer r.mutex.RUnlock() |
|||
|
|||
workers := make([]*types.Worker, 0, len(r.workers)) |
|||
for _, worker := range r.workers { |
|||
workers = append(workers, worker) |
|||
} |
|||
return workers |
|||
} |
|||
|
|||
// GetWorkersByCapability returns workers that support a specific capability
|
|||
func (r *Registry) GetWorkersByCapability(capability types.TaskType) []*types.Worker { |
|||
r.mutex.RLock() |
|||
defer r.mutex.RUnlock() |
|||
|
|||
var workers []*types.Worker |
|||
for _, worker := range r.workers { |
|||
for _, cap := range worker.Capabilities { |
|||
if cap == capability { |
|||
workers = append(workers, worker) |
|||
break |
|||
} |
|||
} |
|||
} |
|||
return workers |
|||
} |
|||
|
|||
// GetAvailableWorkers returns workers that are available for new tasks
|
|||
func (r *Registry) GetAvailableWorkers() []*types.Worker { |
|||
r.mutex.RLock() |
|||
defer r.mutex.RUnlock() |
|||
|
|||
var workers []*types.Worker |
|||
for _, worker := range r.workers { |
|||
if worker.Status == "active" && worker.CurrentLoad < worker.MaxConcurrent { |
|||
workers = append(workers, worker) |
|||
} |
|||
} |
|||
return workers |
|||
} |
|||
|
|||
// GetBestWorkerForTask returns the best worker for a specific task
|
|||
func (r *Registry) GetBestWorkerForTask(taskType types.TaskType) *types.Worker { |
|||
r.mutex.RLock() |
|||
defer r.mutex.RUnlock() |
|||
|
|||
var bestWorker *types.Worker |
|||
var bestScore float64 |
|||
|
|||
for _, worker := range r.workers { |
|||
// Check if worker supports this task type
|
|||
supportsTask := false |
|||
for _, cap := range worker.Capabilities { |
|||
if cap == taskType { |
|||
supportsTask = true |
|||
break |
|||
} |
|||
} |
|||
|
|||
if !supportsTask { |
|||
continue |
|||
} |
|||
|
|||
// Check if worker is available
|
|||
if worker.Status != "active" || worker.CurrentLoad >= worker.MaxConcurrent { |
|||
continue |
|||
} |
|||
|
|||
// Calculate score based on current load and capacity
|
|||
score := float64(worker.MaxConcurrent-worker.CurrentLoad) / float64(worker.MaxConcurrent) |
|||
if bestWorker == nil || score > bestScore { |
|||
bestWorker = worker |
|||
bestScore = score |
|||
} |
|||
} |
|||
|
|||
return bestWorker |
|||
} |
|||
|
|||
// UpdateWorkerHeartbeat updates the last heartbeat time for a worker
|
|||
func (r *Registry) UpdateWorkerHeartbeat(workerID string) error { |
|||
r.mutex.Lock() |
|||
defer r.mutex.Unlock() |
|||
|
|||
worker, exists := r.workers[workerID] |
|||
if !exists { |
|||
return fmt.Errorf("worker %s not found", workerID) |
|||
} |
|||
|
|||
worker.LastHeartbeat = time.Now() |
|||
return nil |
|||
} |
|||
|
|||
// UpdateWorkerLoad updates the current load for a worker
|
|||
func (r *Registry) UpdateWorkerLoad(workerID string, load int) error { |
|||
r.mutex.Lock() |
|||
defer r.mutex.Unlock() |
|||
|
|||
worker, exists := r.workers[workerID] |
|||
if !exists { |
|||
return fmt.Errorf("worker %s not found", workerID) |
|||
} |
|||
|
|||
worker.CurrentLoad = load |
|||
if load >= worker.MaxConcurrent { |
|||
worker.Status = "busy" |
|||
} else { |
|||
worker.Status = "active" |
|||
} |
|||
|
|||
r.updateStats() |
|||
return nil |
|||
} |
|||
|
|||
// UpdateWorkerStatus updates the status of a worker
|
|||
func (r *Registry) UpdateWorkerStatus(workerID string, status string) error { |
|||
r.mutex.Lock() |
|||
defer r.mutex.Unlock() |
|||
|
|||
worker, exists := r.workers[workerID] |
|||
if !exists { |
|||
return fmt.Errorf("worker %s not found", workerID) |
|||
} |
|||
|
|||
worker.Status = status |
|||
r.updateStats() |
|||
return nil |
|||
} |
|||
|
|||
// CleanupStaleWorkers removes workers that haven't sent heartbeats recently
|
|||
func (r *Registry) CleanupStaleWorkers(timeout time.Duration) int { |
|||
r.mutex.Lock() |
|||
defer r.mutex.Unlock() |
|||
|
|||
var removedCount int |
|||
cutoff := time.Now().Add(-timeout) |
|||
|
|||
for workerID, worker := range r.workers { |
|||
if worker.LastHeartbeat.Before(cutoff) { |
|||
delete(r.workers, workerID) |
|||
removedCount++ |
|||
} |
|||
} |
|||
|
|||
if removedCount > 0 { |
|||
r.updateStats() |
|||
} |
|||
|
|||
return removedCount |
|||
} |
|||
|
|||
// GetStats returns current registry statistics
|
|||
func (r *Registry) GetStats() *types.RegistryStats { |
|||
r.mutex.RLock() |
|||
defer r.mutex.RUnlock() |
|||
|
|||
// Create a copy of the stats to avoid race conditions
|
|||
stats := *r.stats |
|||
return &stats |
|||
} |
|||
|
|||
// updateStats updates the registry statistics (must be called with lock held)
|
|||
func (r *Registry) updateStats() { |
|||
r.stats.TotalWorkers = len(r.workers) |
|||
r.stats.ActiveWorkers = 0 |
|||
r.stats.BusyWorkers = 0 |
|||
r.stats.IdleWorkers = 0 |
|||
|
|||
for _, worker := range r.workers { |
|||
switch worker.Status { |
|||
case "active": |
|||
if worker.CurrentLoad > 0 { |
|||
r.stats.ActiveWorkers++ |
|||
} else { |
|||
r.stats.IdleWorkers++ |
|||
} |
|||
case "busy": |
|||
r.stats.BusyWorkers++ |
|||
} |
|||
} |
|||
|
|||
r.stats.Uptime = time.Since(r.stats.StartTime) |
|||
r.stats.LastUpdated = time.Now() |
|||
} |
|||
|
|||
// GetTaskCapabilities returns all task capabilities available in the registry
|
|||
func (r *Registry) GetTaskCapabilities() []types.TaskType { |
|||
r.mutex.RLock() |
|||
defer r.mutex.RUnlock() |
|||
|
|||
capabilitySet := make(map[types.TaskType]bool) |
|||
for _, worker := range r.workers { |
|||
for _, cap := range worker.Capabilities { |
|||
capabilitySet[cap] = true |
|||
} |
|||
} |
|||
|
|||
var capabilities []types.TaskType |
|||
for cap := range capabilitySet { |
|||
capabilities = append(capabilities, cap) |
|||
} |
|||
|
|||
return capabilities |
|||
} |
|||
|
|||
// GetWorkersByStatus returns workers filtered by status
|
|||
func (r *Registry) GetWorkersByStatus(status string) []*types.Worker { |
|||
r.mutex.RLock() |
|||
defer r.mutex.RUnlock() |
|||
|
|||
var workers []*types.Worker |
|||
for _, worker := range r.workers { |
|||
if worker.Status == status { |
|||
workers = append(workers, worker) |
|||
} |
|||
} |
|||
return workers |
|||
} |
|||
|
|||
// GetWorkerCount returns the total number of registered workers
|
|||
func (r *Registry) GetWorkerCount() int { |
|||
r.mutex.RLock() |
|||
defer r.mutex.RUnlock() |
|||
return len(r.workers) |
|||
} |
|||
|
|||
// GetWorkerIDs returns all worker IDs
|
|||
func (r *Registry) GetWorkerIDs() []string { |
|||
r.mutex.RLock() |
|||
defer r.mutex.RUnlock() |
|||
|
|||
ids := make([]string, 0, len(r.workers)) |
|||
for id := range r.workers { |
|||
ids = append(ids, id) |
|||
} |
|||
return ids |
|||
} |
|||
|
|||
// GetWorkerSummary returns a summary of all workers
|
|||
func (r *Registry) GetWorkerSummary() *types.WorkerSummary { |
|||
r.mutex.RLock() |
|||
defer r.mutex.RUnlock() |
|||
|
|||
summary := &types.WorkerSummary{ |
|||
TotalWorkers: len(r.workers), |
|||
ByStatus: make(map[string]int), |
|||
ByCapability: make(map[types.TaskType]int), |
|||
TotalLoad: 0, |
|||
MaxCapacity: 0, |
|||
} |
|||
|
|||
for _, worker := range r.workers { |
|||
summary.ByStatus[worker.Status]++ |
|||
summary.TotalLoad += worker.CurrentLoad |
|||
summary.MaxCapacity += worker.MaxConcurrent |
|||
|
|||
for _, cap := range worker.Capabilities { |
|||
summary.ByCapability[cap]++ |
|||
} |
|||
} |
|||
|
|||
return summary |
|||
} |
|||
|
|||
// Default global registry instance
|
|||
var defaultRegistry *Registry |
|||
var registryOnce sync.Once |
|||
|
|||
// GetDefaultRegistry returns the default global registry
|
|||
func GetDefaultRegistry() *Registry { |
|||
registryOnce.Do(func() { |
|||
defaultRegistry = NewRegistry() |
|||
}) |
|||
return defaultRegistry |
|||
} |
|||
@ -0,0 +1,82 @@ |
|||
package balance |
|||
|
|||
import ( |
|||
"fmt" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/tasks" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// Task implements balance operation to redistribute volumes across volume servers
|
|||
type Task struct { |
|||
*tasks.BaseTask |
|||
server string |
|||
volumeID uint32 |
|||
collection string |
|||
} |
|||
|
|||
// NewTask creates a new balance task instance
|
|||
func NewTask(server string, volumeID uint32, collection string) *Task { |
|||
task := &Task{ |
|||
BaseTask: tasks.NewBaseTask(types.TaskTypeBalance), |
|||
server: server, |
|||
volumeID: volumeID, |
|||
collection: collection, |
|||
} |
|||
return task |
|||
} |
|||
|
|||
// Execute executes the balance task
|
|||
func (t *Task) Execute(params types.TaskParams) error { |
|||
glog.Infof("Starting balance task for volume %d on server %s (collection: %s)", t.volumeID, t.server, t.collection) |
|||
|
|||
// Simulate balance operation with progress updates
|
|||
steps := []struct { |
|||
name string |
|||
duration time.Duration |
|||
progress float64 |
|||
}{ |
|||
{"Analyzing cluster state", 2 * time.Second, 15}, |
|||
{"Identifying optimal placement", 3 * time.Second, 35}, |
|||
{"Moving volume data", 6 * time.Second, 75}, |
|||
{"Updating cluster metadata", 2 * time.Second, 95}, |
|||
{"Verifying balance", 1 * time.Second, 100}, |
|||
} |
|||
|
|||
for _, step := range steps { |
|||
if t.IsCancelled() { |
|||
return fmt.Errorf("balance task cancelled") |
|||
} |
|||
|
|||
glog.V(1).Infof("Balance task step: %s", step.name) |
|||
t.SetProgress(step.progress) |
|||
|
|||
// Simulate work
|
|||
time.Sleep(step.duration) |
|||
} |
|||
|
|||
glog.Infof("Balance task completed for volume %d on server %s", t.volumeID, t.server) |
|||
return nil |
|||
} |
|||
|
|||
// Validate validates the task parameters
|
|||
func (t *Task) Validate(params types.TaskParams) error { |
|||
if params.VolumeID == 0 { |
|||
return fmt.Errorf("volume_id is required") |
|||
} |
|||
if params.Server == "" { |
|||
return fmt.Errorf("server is required") |
|||
} |
|||
return nil |
|||
} |
|||
|
|||
// EstimateTime estimates the time needed for the task
|
|||
func (t *Task) EstimateTime(params types.TaskParams) time.Duration { |
|||
// Base time for balance operation
|
|||
baseTime := 35 * time.Second |
|||
|
|||
// Could adjust based on volume size or cluster state
|
|||
return baseTime |
|||
} |
|||
@ -0,0 +1,171 @@ |
|||
package balance |
|||
|
|||
import ( |
|||
"fmt" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// BalanceDetector implements TaskDetector for balance tasks
|
|||
type BalanceDetector struct { |
|||
enabled bool |
|||
threshold float64 // Imbalance threshold (0.1 = 10%)
|
|||
minCheckInterval time.Duration |
|||
minVolumeCount int |
|||
lastCheck time.Time |
|||
} |
|||
|
|||
// Compile-time interface assertions
|
|||
var ( |
|||
_ types.TaskDetector = (*BalanceDetector)(nil) |
|||
) |
|||
|
|||
// NewBalanceDetector creates a new balance detector
|
|||
func NewBalanceDetector() *BalanceDetector { |
|||
return &BalanceDetector{ |
|||
enabled: true, |
|||
threshold: 0.1, // 10% imbalance threshold
|
|||
minCheckInterval: 1 * time.Hour, |
|||
minVolumeCount: 10, // Don't balance small clusters
|
|||
lastCheck: time.Time{}, |
|||
} |
|||
} |
|||
|
|||
// GetTaskType returns the task type
|
|||
func (d *BalanceDetector) GetTaskType() types.TaskType { |
|||
return types.TaskTypeBalance |
|||
} |
|||
|
|||
// ScanForTasks checks if cluster balance is needed
|
|||
func (d *BalanceDetector) ScanForTasks(volumeMetrics []*types.VolumeHealthMetrics, clusterInfo *types.ClusterInfo) ([]*types.TaskDetectionResult, error) { |
|||
if !d.enabled { |
|||
return nil, nil |
|||
} |
|||
|
|||
glog.V(2).Infof("Scanning for balance tasks...") |
|||
|
|||
// Don't check too frequently
|
|||
if time.Since(d.lastCheck) < d.minCheckInterval { |
|||
return nil, nil |
|||
} |
|||
d.lastCheck = time.Now() |
|||
|
|||
// Skip if cluster is too small
|
|||
if len(volumeMetrics) < d.minVolumeCount { |
|||
glog.V(2).Infof("Cluster too small for balance (%d volumes < %d minimum)", len(volumeMetrics), d.minVolumeCount) |
|||
return nil, nil |
|||
} |
|||
|
|||
// Analyze volume distribution across servers
|
|||
serverVolumeCounts := make(map[string]int) |
|||
for _, metric := range volumeMetrics { |
|||
serverVolumeCounts[metric.Server]++ |
|||
} |
|||
|
|||
if len(serverVolumeCounts) < 2 { |
|||
glog.V(2).Infof("Not enough servers for balance (%d servers)", len(serverVolumeCounts)) |
|||
return nil, nil |
|||
} |
|||
|
|||
// Calculate balance metrics
|
|||
totalVolumes := len(volumeMetrics) |
|||
avgVolumesPerServer := float64(totalVolumes) / float64(len(serverVolumeCounts)) |
|||
|
|||
maxVolumes := 0 |
|||
minVolumes := totalVolumes |
|||
maxServer := "" |
|||
minServer := "" |
|||
|
|||
for server, count := range serverVolumeCounts { |
|||
if count > maxVolumes { |
|||
maxVolumes = count |
|||
maxServer = server |
|||
} |
|||
if count < minVolumes { |
|||
minVolumes = count |
|||
minServer = server |
|||
} |
|||
} |
|||
|
|||
// Check if imbalance exceeds threshold
|
|||
imbalanceRatio := float64(maxVolumes-minVolumes) / avgVolumesPerServer |
|||
if imbalanceRatio <= d.threshold { |
|||
glog.V(2).Infof("Cluster is balanced (imbalance ratio: %.2f <= %.2f)", imbalanceRatio, d.threshold) |
|||
return nil, nil |
|||
} |
|||
|
|||
// Create balance task
|
|||
reason := fmt.Sprintf("Cluster imbalance detected: %.1f%% (max: %d on %s, min: %d on %s, avg: %.1f)", |
|||
imbalanceRatio*100, maxVolumes, maxServer, minVolumes, minServer, avgVolumesPerServer) |
|||
|
|||
task := &types.TaskDetectionResult{ |
|||
TaskType: types.TaskTypeBalance, |
|||
Priority: types.TaskPriorityNormal, |
|||
Reason: reason, |
|||
ScheduleAt: time.Now(), |
|||
Parameters: map[string]interface{}{ |
|||
"imbalance_ratio": imbalanceRatio, |
|||
"threshold": d.threshold, |
|||
"max_volumes": maxVolumes, |
|||
"min_volumes": minVolumes, |
|||
"avg_volumes_per_server": avgVolumesPerServer, |
|||
"max_server": maxServer, |
|||
"min_server": minServer, |
|||
"total_servers": len(serverVolumeCounts), |
|||
}, |
|||
} |
|||
|
|||
glog.V(1).Infof("🔄 Found balance task: %s", reason) |
|||
return []*types.TaskDetectionResult{task}, nil |
|||
} |
|||
|
|||
// ScanInterval returns how often to scan
|
|||
func (d *BalanceDetector) ScanInterval() time.Duration { |
|||
return d.minCheckInterval |
|||
} |
|||
|
|||
// IsEnabled returns whether the detector is enabled
|
|||
func (d *BalanceDetector) IsEnabled() bool { |
|||
return d.enabled |
|||
} |
|||
|
|||
// SetEnabled sets whether the detector is enabled
|
|||
func (d *BalanceDetector) SetEnabled(enabled bool) { |
|||
d.enabled = enabled |
|||
glog.V(1).Infof("🔄 Balance detector enabled: %v", enabled) |
|||
} |
|||
|
|||
// SetThreshold sets the imbalance threshold
|
|||
func (d *BalanceDetector) SetThreshold(threshold float64) { |
|||
d.threshold = threshold |
|||
glog.V(1).Infof("🔄 Balance threshold set to: %.1f%%", threshold*100) |
|||
} |
|||
|
|||
// SetMinCheckInterval sets the minimum time between balance checks
|
|||
func (d *BalanceDetector) SetMinCheckInterval(interval time.Duration) { |
|||
d.minCheckInterval = interval |
|||
glog.V(1).Infof("🔄 Balance check interval set to: %v", interval) |
|||
} |
|||
|
|||
// SetMinVolumeCount sets the minimum volume count for balance operations
|
|||
func (d *BalanceDetector) SetMinVolumeCount(count int) { |
|||
d.minVolumeCount = count |
|||
glog.V(1).Infof("🔄 Balance minimum volume count set to: %d", count) |
|||
} |
|||
|
|||
// GetThreshold returns the current imbalance threshold
|
|||
func (d *BalanceDetector) GetThreshold() float64 { |
|||
return d.threshold |
|||
} |
|||
|
|||
// GetMinCheckInterval returns the minimum check interval
|
|||
func (d *BalanceDetector) GetMinCheckInterval() time.Duration { |
|||
return d.minCheckInterval |
|||
} |
|||
|
|||
// GetMinVolumeCount returns the minimum volume count
|
|||
func (d *BalanceDetector) GetMinVolumeCount() int { |
|||
return d.minVolumeCount |
|||
} |
|||
@ -0,0 +1,81 @@ |
|||
package balance |
|||
|
|||
import ( |
|||
"fmt" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/worker/tasks" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// Factory creates balance task instances
|
|||
type Factory struct { |
|||
*tasks.BaseTaskFactory |
|||
} |
|||
|
|||
// NewFactory creates a new balance task factory
|
|||
func NewFactory() *Factory { |
|||
return &Factory{ |
|||
BaseTaskFactory: tasks.NewBaseTaskFactory( |
|||
types.TaskTypeBalance, |
|||
[]string{"balance", "storage", "optimization"}, |
|||
"Balance data across volume servers for optimal performance", |
|||
), |
|||
} |
|||
} |
|||
|
|||
// Create creates a new balance task instance
|
|||
func (f *Factory) Create(params types.TaskParams) (types.TaskInterface, error) { |
|||
// Validate parameters
|
|||
if params.VolumeID == 0 { |
|||
return nil, fmt.Errorf("volume_id is required") |
|||
} |
|||
if params.Server == "" { |
|||
return nil, fmt.Errorf("server is required") |
|||
} |
|||
|
|||
task := NewTask(params.Server, params.VolumeID, params.Collection) |
|||
task.SetEstimatedDuration(task.EstimateTime(params)) |
|||
|
|||
return task, nil |
|||
} |
|||
|
|||
// Shared detector and scheduler instances
|
|||
var ( |
|||
sharedDetector *BalanceDetector |
|||
sharedScheduler *BalanceScheduler |
|||
) |
|||
|
|||
// getSharedInstances returns the shared detector and scheduler instances
|
|||
func getSharedInstances() (*BalanceDetector, *BalanceScheduler) { |
|||
if sharedDetector == nil { |
|||
sharedDetector = NewBalanceDetector() |
|||
} |
|||
if sharedScheduler == nil { |
|||
sharedScheduler = NewBalanceScheduler() |
|||
} |
|||
return sharedDetector, sharedScheduler |
|||
} |
|||
|
|||
// GetSharedInstances returns the shared detector and scheduler instances (public access)
|
|||
func GetSharedInstances() (*BalanceDetector, *BalanceScheduler) { |
|||
return getSharedInstances() |
|||
} |
|||
|
|||
// Auto-register this task when the package is imported
|
|||
func init() { |
|||
factory := NewFactory() |
|||
tasks.AutoRegister(types.TaskTypeBalance, factory) |
|||
|
|||
// Get shared instances for all registrations
|
|||
detector, scheduler := getSharedInstances() |
|||
|
|||
// Register with types registry
|
|||
tasks.AutoRegisterTypes(func(registry *types.TaskRegistry) { |
|||
registry.RegisterTask(detector, scheduler) |
|||
}) |
|||
|
|||
// Register with UI registry using the same instances
|
|||
tasks.AutoRegisterUI(func(uiRegistry *types.UIRegistry) { |
|||
RegisterUI(uiRegistry, detector, scheduler) |
|||
}) |
|||
} |
|||
@ -0,0 +1,197 @@ |
|||
package balance |
|||
|
|||
import ( |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// BalanceScheduler implements TaskScheduler for balance tasks
|
|||
type BalanceScheduler struct { |
|||
enabled bool |
|||
maxConcurrent int |
|||
minInterval time.Duration |
|||
lastScheduled map[string]time.Time // track when we last scheduled a balance for each task type
|
|||
minServerCount int |
|||
moveDuringOffHours bool |
|||
offHoursStart string |
|||
offHoursEnd string |
|||
} |
|||
|
|||
// Compile-time interface assertions
|
|||
var ( |
|||
_ types.TaskScheduler = (*BalanceScheduler)(nil) |
|||
) |
|||
|
|||
// NewBalanceScheduler creates a new balance scheduler
|
|||
func NewBalanceScheduler() *BalanceScheduler { |
|||
return &BalanceScheduler{ |
|||
enabled: true, |
|||
maxConcurrent: 1, // Only run one balance at a time
|
|||
minInterval: 6 * time.Hour, |
|||
lastScheduled: make(map[string]time.Time), |
|||
minServerCount: 3, |
|||
moveDuringOffHours: true, |
|||
offHoursStart: "23:00", |
|||
offHoursEnd: "06:00", |
|||
} |
|||
} |
|||
|
|||
// GetTaskType returns the task type
|
|||
func (s *BalanceScheduler) GetTaskType() types.TaskType { |
|||
return types.TaskTypeBalance |
|||
} |
|||
|
|||
// CanScheduleNow determines if a balance task can be scheduled
|
|||
func (s *BalanceScheduler) CanScheduleNow(task *types.Task, runningTasks []*types.Task, availableWorkers []*types.Worker) bool { |
|||
if !s.enabled { |
|||
return false |
|||
} |
|||
|
|||
// Count running balance tasks
|
|||
runningBalanceCount := 0 |
|||
for _, runningTask := range runningTasks { |
|||
if runningTask.Type == types.TaskTypeBalance { |
|||
runningBalanceCount++ |
|||
} |
|||
} |
|||
|
|||
// Check concurrency limit
|
|||
if runningBalanceCount >= s.maxConcurrent { |
|||
glog.V(3).Infof("⏸️ Balance task blocked: too many running (%d >= %d)", runningBalanceCount, s.maxConcurrent) |
|||
return false |
|||
} |
|||
|
|||
// Check minimum interval between balance operations
|
|||
if lastTime, exists := s.lastScheduled["balance"]; exists { |
|||
if time.Since(lastTime) < s.minInterval { |
|||
timeLeft := s.minInterval - time.Since(lastTime) |
|||
glog.V(3).Infof("⏸️ Balance task blocked: too soon (wait %v)", timeLeft) |
|||
return false |
|||
} |
|||
} |
|||
|
|||
// Check if we have available workers
|
|||
availableWorkerCount := 0 |
|||
for _, worker := range availableWorkers { |
|||
for _, capability := range worker.Capabilities { |
|||
if capability == types.TaskTypeBalance { |
|||
availableWorkerCount++ |
|||
break |
|||
} |
|||
} |
|||
} |
|||
|
|||
if availableWorkerCount == 0 { |
|||
glog.V(3).Infof("⏸️ Balance task blocked: no available workers") |
|||
return false |
|||
} |
|||
|
|||
// All checks passed - can schedule
|
|||
s.lastScheduled["balance"] = time.Now() |
|||
glog.V(2).Infof("✅ Balance task can be scheduled (running: %d/%d, workers: %d)", |
|||
runningBalanceCount, s.maxConcurrent, availableWorkerCount) |
|||
return true |
|||
} |
|||
|
|||
// GetPriority returns the priority for balance tasks
|
|||
func (s *BalanceScheduler) GetPriority(task *types.Task) types.TaskPriority { |
|||
// Balance is typically normal priority - not urgent but important for optimization
|
|||
return types.TaskPriorityNormal |
|||
} |
|||
|
|||
// GetMaxConcurrent returns the maximum concurrent balance tasks
|
|||
func (s *BalanceScheduler) GetMaxConcurrent() int { |
|||
return s.maxConcurrent |
|||
} |
|||
|
|||
// GetDefaultRepeatInterval returns the default interval to wait before repeating balance tasks
|
|||
func (s *BalanceScheduler) GetDefaultRepeatInterval() time.Duration { |
|||
return s.minInterval |
|||
} |
|||
|
|||
// IsEnabled returns whether the scheduler is enabled
|
|||
func (s *BalanceScheduler) IsEnabled() bool { |
|||
return s.enabled |
|||
} |
|||
|
|||
// SetEnabled sets whether the scheduler is enabled
|
|||
func (s *BalanceScheduler) SetEnabled(enabled bool) { |
|||
s.enabled = enabled |
|||
glog.V(1).Infof("🔄 Balance scheduler enabled: %v", enabled) |
|||
} |
|||
|
|||
// SetMaxConcurrent sets the maximum concurrent balance tasks
|
|||
func (s *BalanceScheduler) SetMaxConcurrent(max int) { |
|||
s.maxConcurrent = max |
|||
glog.V(1).Infof("🔄 Balance max concurrent set to: %d", max) |
|||
} |
|||
|
|||
// SetMinInterval sets the minimum interval between balance operations
|
|||
func (s *BalanceScheduler) SetMinInterval(interval time.Duration) { |
|||
s.minInterval = interval |
|||
glog.V(1).Infof("🔄 Balance minimum interval set to: %v", interval) |
|||
} |
|||
|
|||
// GetLastScheduled returns when we last scheduled this task type
|
|||
func (s *BalanceScheduler) GetLastScheduled(taskKey string) time.Time { |
|||
if lastTime, exists := s.lastScheduled[taskKey]; exists { |
|||
return lastTime |
|||
} |
|||
return time.Time{} |
|||
} |
|||
|
|||
// SetLastScheduled updates when we last scheduled this task type
|
|||
func (s *BalanceScheduler) SetLastScheduled(taskKey string, when time.Time) { |
|||
s.lastScheduled[taskKey] = when |
|||
} |
|||
|
|||
// GetMinServerCount returns the minimum server count
|
|||
func (s *BalanceScheduler) GetMinServerCount() int { |
|||
return s.minServerCount |
|||
} |
|||
|
|||
// SetMinServerCount sets the minimum server count
|
|||
func (s *BalanceScheduler) SetMinServerCount(count int) { |
|||
s.minServerCount = count |
|||
glog.V(1).Infof("🔄 Balance minimum server count set to: %d", count) |
|||
} |
|||
|
|||
// GetMoveDuringOffHours returns whether to move only during off-hours
|
|||
func (s *BalanceScheduler) GetMoveDuringOffHours() bool { |
|||
return s.moveDuringOffHours |
|||
} |
|||
|
|||
// SetMoveDuringOffHours sets whether to move only during off-hours
|
|||
func (s *BalanceScheduler) SetMoveDuringOffHours(enabled bool) { |
|||
s.moveDuringOffHours = enabled |
|||
glog.V(1).Infof("🔄 Balance move during off-hours: %v", enabled) |
|||
} |
|||
|
|||
// GetOffHoursStart returns the off-hours start time
|
|||
func (s *BalanceScheduler) GetOffHoursStart() string { |
|||
return s.offHoursStart |
|||
} |
|||
|
|||
// SetOffHoursStart sets the off-hours start time
|
|||
func (s *BalanceScheduler) SetOffHoursStart(start string) { |
|||
s.offHoursStart = start |
|||
glog.V(1).Infof("🔄 Balance off-hours start time set to: %s", start) |
|||
} |
|||
|
|||
// GetOffHoursEnd returns the off-hours end time
|
|||
func (s *BalanceScheduler) GetOffHoursEnd() string { |
|||
return s.offHoursEnd |
|||
} |
|||
|
|||
// SetOffHoursEnd sets the off-hours end time
|
|||
func (s *BalanceScheduler) SetOffHoursEnd(end string) { |
|||
s.offHoursEnd = end |
|||
glog.V(1).Infof("🔄 Balance off-hours end time set to: %s", end) |
|||
} |
|||
|
|||
// GetMinInterval returns the minimum interval
|
|||
func (s *BalanceScheduler) GetMinInterval() time.Duration { |
|||
return s.minInterval |
|||
} |
|||
@ -0,0 +1,361 @@ |
|||
package balance |
|||
|
|||
import ( |
|||
"fmt" |
|||
"html/template" |
|||
"strconv" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// UIProvider provides the UI for balance task configuration
|
|||
type UIProvider struct { |
|||
detector *BalanceDetector |
|||
scheduler *BalanceScheduler |
|||
} |
|||
|
|||
// NewUIProvider creates a new balance UI provider
|
|||
func NewUIProvider(detector *BalanceDetector, scheduler *BalanceScheduler) *UIProvider { |
|||
return &UIProvider{ |
|||
detector: detector, |
|||
scheduler: scheduler, |
|||
} |
|||
} |
|||
|
|||
// GetTaskType returns the task type
|
|||
func (ui *UIProvider) GetTaskType() types.TaskType { |
|||
return types.TaskTypeBalance |
|||
} |
|||
|
|||
// GetDisplayName returns the human-readable name
|
|||
func (ui *UIProvider) GetDisplayName() string { |
|||
return "Volume Balance" |
|||
} |
|||
|
|||
// GetDescription returns a description of what this task does
|
|||
func (ui *UIProvider) GetDescription() string { |
|||
return "Redistributes volumes across volume servers to optimize storage utilization and performance" |
|||
} |
|||
|
|||
// GetIcon returns the icon CSS class for this task type
|
|||
func (ui *UIProvider) GetIcon() string { |
|||
return "fas fa-balance-scale text-secondary" |
|||
} |
|||
|
|||
// BalanceConfig represents the balance configuration
|
|||
type BalanceConfig struct { |
|||
Enabled bool `json:"enabled"` |
|||
ImbalanceThreshold float64 `json:"imbalance_threshold"` |
|||
ScanIntervalSeconds int `json:"scan_interval_seconds"` |
|||
MaxConcurrent int `json:"max_concurrent"` |
|||
MinServerCount int `json:"min_server_count"` |
|||
MoveDuringOffHours bool `json:"move_during_off_hours"` |
|||
OffHoursStart string `json:"off_hours_start"` |
|||
OffHoursEnd string `json:"off_hours_end"` |
|||
MinIntervalSeconds int `json:"min_interval_seconds"` |
|||
} |
|||
|
|||
// Helper functions for duration conversion
|
|||
func secondsToDuration(seconds int) time.Duration { |
|||
return time.Duration(seconds) * time.Second |
|||
} |
|||
|
|||
func durationToSeconds(d time.Duration) int { |
|||
return int(d.Seconds()) |
|||
} |
|||
|
|||
// formatDurationForUser formats seconds as a user-friendly duration string
|
|||
func formatDurationForUser(seconds int) string { |
|||
d := secondsToDuration(seconds) |
|||
if d < time.Minute { |
|||
return fmt.Sprintf("%ds", seconds) |
|||
} |
|||
if d < time.Hour { |
|||
return fmt.Sprintf("%.0fm", d.Minutes()) |
|||
} |
|||
if d < 24*time.Hour { |
|||
return fmt.Sprintf("%.1fh", d.Hours()) |
|||
} |
|||
return fmt.Sprintf("%.1fd", d.Hours()/24) |
|||
} |
|||
|
|||
// RenderConfigForm renders the configuration form HTML
|
|||
func (ui *UIProvider) RenderConfigForm(currentConfig interface{}) (template.HTML, error) { |
|||
config := ui.getCurrentBalanceConfig() |
|||
|
|||
// Build form using the FormBuilder helper
|
|||
form := types.NewFormBuilder() |
|||
|
|||
// Detection Settings
|
|||
form.AddCheckboxField( |
|||
"enabled", |
|||
"Enable Balance Tasks", |
|||
"Whether balance tasks should be automatically created", |
|||
config.Enabled, |
|||
) |
|||
|
|||
form.AddNumberField( |
|||
"imbalance_threshold", |
|||
"Imbalance Threshold (%)", |
|||
"Trigger balance when storage imbalance exceeds this percentage (0.0-1.0)", |
|||
config.ImbalanceThreshold, |
|||
true, |
|||
) |
|||
|
|||
form.AddDurationField("scan_interval", "Scan Interval", "How often to scan for imbalanced volumes", secondsToDuration(config.ScanIntervalSeconds), true) |
|||
|
|||
// Scheduling Settings
|
|||
form.AddNumberField( |
|||
"max_concurrent", |
|||
"Max Concurrent Tasks", |
|||
"Maximum number of balance tasks that can run simultaneously", |
|||
float64(config.MaxConcurrent), |
|||
true, |
|||
) |
|||
|
|||
form.AddNumberField( |
|||
"min_server_count", |
|||
"Minimum Server Count", |
|||
"Only balance when at least this many servers are available", |
|||
float64(config.MinServerCount), |
|||
true, |
|||
) |
|||
|
|||
// Timing Settings
|
|||
form.AddCheckboxField( |
|||
"move_during_off_hours", |
|||
"Restrict to Off-Hours", |
|||
"Only perform balance operations during off-peak hours", |
|||
config.MoveDuringOffHours, |
|||
) |
|||
|
|||
form.AddTextField( |
|||
"off_hours_start", |
|||
"Off-Hours Start Time", |
|||
"Start time for off-hours window (e.g., 23:00)", |
|||
config.OffHoursStart, |
|||
false, |
|||
) |
|||
|
|||
form.AddTextField( |
|||
"off_hours_end", |
|||
"Off-Hours End Time", |
|||
"End time for off-hours window (e.g., 06:00)", |
|||
config.OffHoursEnd, |
|||
false, |
|||
) |
|||
|
|||
// Timing constraints
|
|||
form.AddDurationField("min_interval", "Min Interval", "Minimum time between balance operations", secondsToDuration(config.MinIntervalSeconds), true) |
|||
|
|||
// Generate organized form sections using Bootstrap components
|
|||
html := ` |
|||
<div class="row"> |
|||
<div class="col-12"> |
|||
<div class="card mb-4"> |
|||
<div class="card-header"> |
|||
<h5 class="mb-0"> |
|||
<i class="fas fa-balance-scale me-2"></i> |
|||
Balance Configuration |
|||
</h5> |
|||
</div> |
|||
<div class="card-body"> |
|||
` + string(form.Build()) + ` |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<div class="row"> |
|||
<div class="col-12"> |
|||
<div class="card mb-3"> |
|||
<div class="card-header"> |
|||
<h5 class="mb-0"> |
|||
<i class="fas fa-exclamation-triangle me-2"></i> |
|||
Performance Considerations |
|||
</h5> |
|||
</div> |
|||
<div class="card-body"> |
|||
<div class="alert alert-warning" role="alert"> |
|||
<h6 class="alert-heading">Important Considerations:</h6> |
|||
<p class="mb-2"><strong>Performance:</strong> Volume balancing involves data movement and can impact cluster performance.</p> |
|||
<p class="mb-2"><strong>Recommendation:</strong> Enable off-hours restriction to minimize impact on production workloads.</p> |
|||
<p class="mb-0"><strong>Safety:</strong> Requires at least ` + fmt.Sprintf("%d", config.MinServerCount) + ` servers to ensure data safety during moves.</p> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div>` |
|||
|
|||
return template.HTML(html), nil |
|||
} |
|||
|
|||
// ParseConfigForm parses form data into configuration
|
|||
func (ui *UIProvider) ParseConfigForm(formData map[string][]string) (interface{}, error) { |
|||
config := &BalanceConfig{} |
|||
|
|||
// Parse enabled
|
|||
config.Enabled = len(formData["enabled"]) > 0 |
|||
|
|||
// Parse imbalance threshold
|
|||
if values, ok := formData["imbalance_threshold"]; ok && len(values) > 0 { |
|||
threshold, err := strconv.ParseFloat(values[0], 64) |
|||
if err != nil { |
|||
return nil, fmt.Errorf("invalid imbalance threshold: %v", err) |
|||
} |
|||
if threshold < 0 || threshold > 1 { |
|||
return nil, fmt.Errorf("imbalance threshold must be between 0.0 and 1.0") |
|||
} |
|||
config.ImbalanceThreshold = threshold |
|||
} |
|||
|
|||
// Parse scan interval
|
|||
if values, ok := formData["scan_interval"]; ok && len(values) > 0 { |
|||
duration, err := time.ParseDuration(values[0]) |
|||
if err != nil { |
|||
return nil, fmt.Errorf("invalid scan interval: %v", err) |
|||
} |
|||
config.ScanIntervalSeconds = int(duration.Seconds()) |
|||
} |
|||
|
|||
// Parse max concurrent
|
|||
if values, ok := formData["max_concurrent"]; ok && len(values) > 0 { |
|||
maxConcurrent, err := strconv.Atoi(values[0]) |
|||
if err != nil { |
|||
return nil, fmt.Errorf("invalid max concurrent: %v", err) |
|||
} |
|||
if maxConcurrent < 1 { |
|||
return nil, fmt.Errorf("max concurrent must be at least 1") |
|||
} |
|||
config.MaxConcurrent = maxConcurrent |
|||
} |
|||
|
|||
// Parse min server count
|
|||
if values, ok := formData["min_server_count"]; ok && len(values) > 0 { |
|||
minServerCount, err := strconv.Atoi(values[0]) |
|||
if err != nil { |
|||
return nil, fmt.Errorf("invalid min server count: %v", err) |
|||
} |
|||
if minServerCount < 2 { |
|||
return nil, fmt.Errorf("min server count must be at least 2") |
|||
} |
|||
config.MinServerCount = minServerCount |
|||
} |
|||
|
|||
// Parse off-hours settings
|
|||
config.MoveDuringOffHours = len(formData["move_during_off_hours"]) > 0 |
|||
|
|||
if values, ok := formData["off_hours_start"]; ok && len(values) > 0 { |
|||
config.OffHoursStart = values[0] |
|||
} |
|||
|
|||
if values, ok := formData["off_hours_end"]; ok && len(values) > 0 { |
|||
config.OffHoursEnd = values[0] |
|||
} |
|||
|
|||
// Parse min interval
|
|||
if values, ok := formData["min_interval"]; ok && len(values) > 0 { |
|||
duration, err := time.ParseDuration(values[0]) |
|||
if err != nil { |
|||
return nil, fmt.Errorf("invalid min interval: %v", err) |
|||
} |
|||
config.MinIntervalSeconds = int(duration.Seconds()) |
|||
} |
|||
|
|||
return config, nil |
|||
} |
|||
|
|||
// GetCurrentConfig returns the current configuration
|
|||
func (ui *UIProvider) GetCurrentConfig() interface{} { |
|||
return ui.getCurrentBalanceConfig() |
|||
} |
|||
|
|||
// ApplyConfig applies the new configuration
|
|||
func (ui *UIProvider) ApplyConfig(config interface{}) error { |
|||
balanceConfig, ok := config.(*BalanceConfig) |
|||
if !ok { |
|||
return fmt.Errorf("invalid config type, expected *BalanceConfig") |
|||
} |
|||
|
|||
// Apply to detector
|
|||
if ui.detector != nil { |
|||
ui.detector.SetEnabled(balanceConfig.Enabled) |
|||
ui.detector.SetThreshold(balanceConfig.ImbalanceThreshold) |
|||
ui.detector.SetMinCheckInterval(secondsToDuration(balanceConfig.ScanIntervalSeconds)) |
|||
} |
|||
|
|||
// Apply to scheduler
|
|||
if ui.scheduler != nil { |
|||
ui.scheduler.SetEnabled(balanceConfig.Enabled) |
|||
ui.scheduler.SetMaxConcurrent(balanceConfig.MaxConcurrent) |
|||
ui.scheduler.SetMinServerCount(balanceConfig.MinServerCount) |
|||
ui.scheduler.SetMoveDuringOffHours(balanceConfig.MoveDuringOffHours) |
|||
ui.scheduler.SetOffHoursStart(balanceConfig.OffHoursStart) |
|||
ui.scheduler.SetOffHoursEnd(balanceConfig.OffHoursEnd) |
|||
} |
|||
|
|||
glog.V(1).Infof("Applied balance configuration: enabled=%v, threshold=%.1f%%, max_concurrent=%d, min_servers=%d, off_hours=%v", |
|||
balanceConfig.Enabled, balanceConfig.ImbalanceThreshold*100, balanceConfig.MaxConcurrent, |
|||
balanceConfig.MinServerCount, balanceConfig.MoveDuringOffHours) |
|||
|
|||
return nil |
|||
} |
|||
|
|||
// getCurrentBalanceConfig gets the current configuration from detector and scheduler
|
|||
func (ui *UIProvider) getCurrentBalanceConfig() *BalanceConfig { |
|||
config := &BalanceConfig{ |
|||
// Default values (fallback if detectors/schedulers are nil)
|
|||
Enabled: true, |
|||
ImbalanceThreshold: 0.1, // 10% imbalance
|
|||
ScanIntervalSeconds: durationToSeconds(4 * time.Hour), |
|||
MaxConcurrent: 1, |
|||
MinServerCount: 3, |
|||
MoveDuringOffHours: true, |
|||
OffHoursStart: "23:00", |
|||
OffHoursEnd: "06:00", |
|||
MinIntervalSeconds: durationToSeconds(1 * time.Hour), |
|||
} |
|||
|
|||
// Get current values from detector
|
|||
if ui.detector != nil { |
|||
config.Enabled = ui.detector.IsEnabled() |
|||
config.ImbalanceThreshold = ui.detector.GetThreshold() |
|||
config.ScanIntervalSeconds = int(ui.detector.ScanInterval().Seconds()) |
|||
} |
|||
|
|||
// Get current values from scheduler
|
|||
if ui.scheduler != nil { |
|||
config.MaxConcurrent = ui.scheduler.GetMaxConcurrent() |
|||
config.MinServerCount = ui.scheduler.GetMinServerCount() |
|||
config.MoveDuringOffHours = ui.scheduler.GetMoveDuringOffHours() |
|||
config.OffHoursStart = ui.scheduler.GetOffHoursStart() |
|||
config.OffHoursEnd = ui.scheduler.GetOffHoursEnd() |
|||
} |
|||
|
|||
return config |
|||
} |
|||
|
|||
// RegisterUI registers the balance UI provider with the UI registry
|
|||
func RegisterUI(uiRegistry *types.UIRegistry, detector *BalanceDetector, scheduler *BalanceScheduler) { |
|||
uiProvider := NewUIProvider(detector, scheduler) |
|||
uiRegistry.RegisterUI(uiProvider) |
|||
|
|||
glog.V(1).Infof("✅ Registered balance task UI provider") |
|||
} |
|||
|
|||
// DefaultBalanceConfig returns default balance configuration
|
|||
func DefaultBalanceConfig() *BalanceConfig { |
|||
return &BalanceConfig{ |
|||
Enabled: false, |
|||
ImbalanceThreshold: 0.3, |
|||
ScanIntervalSeconds: durationToSeconds(4 * time.Hour), |
|||
MaxConcurrent: 1, |
|||
MinServerCount: 3, |
|||
MoveDuringOffHours: false, |
|||
OffHoursStart: "22:00", |
|||
OffHoursEnd: "06:00", |
|||
MinIntervalSeconds: durationToSeconds(1 * time.Hour), |
|||
} |
|||
} |
|||
@ -0,0 +1,369 @@ |
|||
package balance |
|||
|
|||
import ( |
|||
"fmt" |
|||
"strconv" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/admin/view/components" |
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// Helper function to format seconds as duration string
|
|||
func formatDurationFromSeconds(seconds int) string { |
|||
d := time.Duration(seconds) * time.Second |
|||
return d.String() |
|||
} |
|||
|
|||
// Helper functions to convert between seconds and value+unit format
|
|||
func secondsToValueAndUnit(seconds int) (float64, string) { |
|||
if seconds == 0 { |
|||
return 0, "minutes" |
|||
} |
|||
|
|||
// Try days first
|
|||
if seconds%(24*3600) == 0 && seconds >= 24*3600 { |
|||
return float64(seconds / (24 * 3600)), "days" |
|||
} |
|||
|
|||
// Try hours
|
|||
if seconds%3600 == 0 && seconds >= 3600 { |
|||
return float64(seconds / 3600), "hours" |
|||
} |
|||
|
|||
// Default to minutes
|
|||
return float64(seconds / 60), "minutes" |
|||
} |
|||
|
|||
func valueAndUnitToSeconds(value float64, unit string) int { |
|||
switch unit { |
|||
case "days": |
|||
return int(value * 24 * 3600) |
|||
case "hours": |
|||
return int(value * 3600) |
|||
case "minutes": |
|||
return int(value * 60) |
|||
default: |
|||
return int(value * 60) // Default to minutes
|
|||
} |
|||
} |
|||
|
|||
// UITemplProvider provides the templ-based UI for balance task configuration
|
|||
type UITemplProvider struct { |
|||
detector *BalanceDetector |
|||
scheduler *BalanceScheduler |
|||
} |
|||
|
|||
// NewUITemplProvider creates a new balance templ UI provider
|
|||
func NewUITemplProvider(detector *BalanceDetector, scheduler *BalanceScheduler) *UITemplProvider { |
|||
return &UITemplProvider{ |
|||
detector: detector, |
|||
scheduler: scheduler, |
|||
} |
|||
} |
|||
|
|||
// GetTaskType returns the task type
|
|||
func (ui *UITemplProvider) GetTaskType() types.TaskType { |
|||
return types.TaskTypeBalance |
|||
} |
|||
|
|||
// GetDisplayName returns the human-readable name
|
|||
func (ui *UITemplProvider) GetDisplayName() string { |
|||
return "Volume Balance" |
|||
} |
|||
|
|||
// GetDescription returns a description of what this task does
|
|||
func (ui *UITemplProvider) GetDescription() string { |
|||
return "Redistributes volumes across volume servers to optimize storage utilization and performance" |
|||
} |
|||
|
|||
// GetIcon returns the icon CSS class for this task type
|
|||
func (ui *UITemplProvider) GetIcon() string { |
|||
return "fas fa-balance-scale text-secondary" |
|||
} |
|||
|
|||
// RenderConfigSections renders the configuration as templ section data
|
|||
func (ui *UITemplProvider) RenderConfigSections(currentConfig interface{}) ([]components.ConfigSectionData, error) { |
|||
config := ui.getCurrentBalanceConfig() |
|||
|
|||
// Detection settings section
|
|||
detectionSection := components.ConfigSectionData{ |
|||
Title: "Detection Settings", |
|||
Icon: "fas fa-search", |
|||
Description: "Configure when balance tasks should be triggered", |
|||
Fields: []interface{}{ |
|||
components.CheckboxFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "enabled", |
|||
Label: "Enable Balance Tasks", |
|||
Description: "Whether balance tasks should be automatically created", |
|||
}, |
|||
Checked: config.Enabled, |
|||
}, |
|||
components.NumberFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "imbalance_threshold", |
|||
Label: "Imbalance Threshold", |
|||
Description: "Trigger balance when storage imbalance exceeds this percentage (0.0-1.0)", |
|||
Required: true, |
|||
}, |
|||
Value: config.ImbalanceThreshold, |
|||
Step: "0.01", |
|||
Min: floatPtr(0.0), |
|||
Max: floatPtr(1.0), |
|||
}, |
|||
components.DurationInputFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "scan_interval", |
|||
Label: "Scan Interval", |
|||
Description: "How often to scan for imbalanced volumes", |
|||
Required: true, |
|||
}, |
|||
Seconds: config.ScanIntervalSeconds, |
|||
}, |
|||
}, |
|||
} |
|||
|
|||
// Scheduling settings section
|
|||
schedulingSection := components.ConfigSectionData{ |
|||
Title: "Scheduling Settings", |
|||
Icon: "fas fa-clock", |
|||
Description: "Configure task scheduling and concurrency", |
|||
Fields: []interface{}{ |
|||
components.NumberFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "max_concurrent", |
|||
Label: "Max Concurrent Tasks", |
|||
Description: "Maximum number of balance tasks that can run simultaneously", |
|||
Required: true, |
|||
}, |
|||
Value: float64(config.MaxConcurrent), |
|||
Step: "1", |
|||
Min: floatPtr(1), |
|||
}, |
|||
components.NumberFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "min_server_count", |
|||
Label: "Minimum Server Count", |
|||
Description: "Only balance when at least this many servers are available", |
|||
Required: true, |
|||
}, |
|||
Value: float64(config.MinServerCount), |
|||
Step: "1", |
|||
Min: floatPtr(1), |
|||
}, |
|||
}, |
|||
} |
|||
|
|||
// Timing constraints section
|
|||
timingSection := components.ConfigSectionData{ |
|||
Title: "Timing Constraints", |
|||
Icon: "fas fa-calendar-clock", |
|||
Description: "Configure when balance operations are allowed", |
|||
Fields: []interface{}{ |
|||
components.CheckboxFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "move_during_off_hours", |
|||
Label: "Restrict to Off-Hours", |
|||
Description: "Only perform balance operations during off-peak hours", |
|||
}, |
|||
Checked: config.MoveDuringOffHours, |
|||
}, |
|||
components.TextFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "off_hours_start", |
|||
Label: "Off-Hours Start Time", |
|||
Description: "Start time for off-hours window (e.g., 23:00)", |
|||
}, |
|||
Value: config.OffHoursStart, |
|||
}, |
|||
components.TextFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "off_hours_end", |
|||
Label: "Off-Hours End Time", |
|||
Description: "End time for off-hours window (e.g., 06:00)", |
|||
}, |
|||
Value: config.OffHoursEnd, |
|||
}, |
|||
}, |
|||
} |
|||
|
|||
// Performance impact info section
|
|||
performanceSection := components.ConfigSectionData{ |
|||
Title: "Performance Considerations", |
|||
Icon: "fas fa-exclamation-triangle", |
|||
Description: "Important information about balance operations", |
|||
Fields: []interface{}{ |
|||
components.TextFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "performance_info", |
|||
Label: "Performance Impact", |
|||
Description: "Volume balancing involves data movement and can impact cluster performance", |
|||
}, |
|||
Value: "Enable off-hours restriction to minimize impact on production workloads", |
|||
}, |
|||
components.TextFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "safety_info", |
|||
Label: "Safety Requirements", |
|||
Description: fmt.Sprintf("Requires at least %d servers to ensure data safety during moves", config.MinServerCount), |
|||
}, |
|||
Value: "Maintains data safety during volume moves between servers", |
|||
}, |
|||
}, |
|||
} |
|||
|
|||
return []components.ConfigSectionData{detectionSection, schedulingSection, timingSection, performanceSection}, nil |
|||
} |
|||
|
|||
// ParseConfigForm parses form data into configuration
|
|||
func (ui *UITemplProvider) ParseConfigForm(formData map[string][]string) (interface{}, error) { |
|||
config := &BalanceConfig{} |
|||
|
|||
// Parse enabled checkbox
|
|||
config.Enabled = len(formData["enabled"]) > 0 && formData["enabled"][0] == "on" |
|||
|
|||
// Parse imbalance threshold
|
|||
if thresholdStr := formData["imbalance_threshold"]; len(thresholdStr) > 0 { |
|||
if threshold, err := strconv.ParseFloat(thresholdStr[0], 64); err != nil { |
|||
return nil, fmt.Errorf("invalid imbalance threshold: %v", err) |
|||
} else if threshold < 0 || threshold > 1 { |
|||
return nil, fmt.Errorf("imbalance threshold must be between 0.0 and 1.0") |
|||
} else { |
|||
config.ImbalanceThreshold = threshold |
|||
} |
|||
} |
|||
|
|||
// Parse scan interval
|
|||
if valueStr := formData["scan_interval"]; len(valueStr) > 0 { |
|||
if value, err := strconv.ParseFloat(valueStr[0], 64); err != nil { |
|||
return nil, fmt.Errorf("invalid scan interval value: %v", err) |
|||
} else { |
|||
unit := "minutes" // default
|
|||
if unitStr := formData["scan_interval_unit"]; len(unitStr) > 0 { |
|||
unit = unitStr[0] |
|||
} |
|||
config.ScanIntervalSeconds = valueAndUnitToSeconds(value, unit) |
|||
} |
|||
} |
|||
|
|||
// Parse max concurrent
|
|||
if concurrentStr := formData["max_concurrent"]; len(concurrentStr) > 0 { |
|||
if concurrent, err := strconv.Atoi(concurrentStr[0]); err != nil { |
|||
return nil, fmt.Errorf("invalid max concurrent: %v", err) |
|||
} else if concurrent < 1 { |
|||
return nil, fmt.Errorf("max concurrent must be at least 1") |
|||
} else { |
|||
config.MaxConcurrent = concurrent |
|||
} |
|||
} |
|||
|
|||
// Parse min server count
|
|||
if serverCountStr := formData["min_server_count"]; len(serverCountStr) > 0 { |
|||
if serverCount, err := strconv.Atoi(serverCountStr[0]); err != nil { |
|||
return nil, fmt.Errorf("invalid min server count: %v", err) |
|||
} else if serverCount < 1 { |
|||
return nil, fmt.Errorf("min server count must be at least 1") |
|||
} else { |
|||
config.MinServerCount = serverCount |
|||
} |
|||
} |
|||
|
|||
// Parse move during off hours
|
|||
config.MoveDuringOffHours = len(formData["move_during_off_hours"]) > 0 && formData["move_during_off_hours"][0] == "on" |
|||
|
|||
// Parse off hours start time
|
|||
if startStr := formData["off_hours_start"]; len(startStr) > 0 { |
|||
config.OffHoursStart = startStr[0] |
|||
} |
|||
|
|||
// Parse off hours end time
|
|||
if endStr := formData["off_hours_end"]; len(endStr) > 0 { |
|||
config.OffHoursEnd = endStr[0] |
|||
} |
|||
|
|||
return config, nil |
|||
} |
|||
|
|||
// GetCurrentConfig returns the current configuration
|
|||
func (ui *UITemplProvider) GetCurrentConfig() interface{} { |
|||
return ui.getCurrentBalanceConfig() |
|||
} |
|||
|
|||
// ApplyConfig applies the new configuration
|
|||
func (ui *UITemplProvider) ApplyConfig(config interface{}) error { |
|||
balanceConfig, ok := config.(*BalanceConfig) |
|||
if !ok { |
|||
return fmt.Errorf("invalid config type, expected *BalanceConfig") |
|||
} |
|||
|
|||
// Apply to detector
|
|||
if ui.detector != nil { |
|||
ui.detector.SetEnabled(balanceConfig.Enabled) |
|||
ui.detector.SetThreshold(balanceConfig.ImbalanceThreshold) |
|||
ui.detector.SetMinCheckInterval(time.Duration(balanceConfig.ScanIntervalSeconds) * time.Second) |
|||
} |
|||
|
|||
// Apply to scheduler
|
|||
if ui.scheduler != nil { |
|||
ui.scheduler.SetEnabled(balanceConfig.Enabled) |
|||
ui.scheduler.SetMaxConcurrent(balanceConfig.MaxConcurrent) |
|||
ui.scheduler.SetMinServerCount(balanceConfig.MinServerCount) |
|||
ui.scheduler.SetMoveDuringOffHours(balanceConfig.MoveDuringOffHours) |
|||
ui.scheduler.SetOffHoursStart(balanceConfig.OffHoursStart) |
|||
ui.scheduler.SetOffHoursEnd(balanceConfig.OffHoursEnd) |
|||
} |
|||
|
|||
glog.V(1).Infof("Applied balance configuration: enabled=%v, threshold=%.1f%%, max_concurrent=%d, min_servers=%d, off_hours=%v", |
|||
balanceConfig.Enabled, balanceConfig.ImbalanceThreshold*100, balanceConfig.MaxConcurrent, |
|||
balanceConfig.MinServerCount, balanceConfig.MoveDuringOffHours) |
|||
|
|||
return nil |
|||
} |
|||
|
|||
// getCurrentBalanceConfig gets the current configuration from detector and scheduler
|
|||
func (ui *UITemplProvider) getCurrentBalanceConfig() *BalanceConfig { |
|||
config := &BalanceConfig{ |
|||
// Default values (fallback if detectors/schedulers are nil)
|
|||
Enabled: true, |
|||
ImbalanceThreshold: 0.1, // 10% imbalance
|
|||
ScanIntervalSeconds: int((4 * time.Hour).Seconds()), |
|||
MaxConcurrent: 1, |
|||
MinServerCount: 3, |
|||
MoveDuringOffHours: true, |
|||
OffHoursStart: "23:00", |
|||
OffHoursEnd: "06:00", |
|||
} |
|||
|
|||
// Get current values from detector
|
|||
if ui.detector != nil { |
|||
config.Enabled = ui.detector.IsEnabled() |
|||
config.ImbalanceThreshold = ui.detector.GetThreshold() |
|||
config.ScanIntervalSeconds = int(ui.detector.ScanInterval().Seconds()) |
|||
} |
|||
|
|||
// Get current values from scheduler
|
|||
if ui.scheduler != nil { |
|||
config.MaxConcurrent = ui.scheduler.GetMaxConcurrent() |
|||
config.MinServerCount = ui.scheduler.GetMinServerCount() |
|||
config.MoveDuringOffHours = ui.scheduler.GetMoveDuringOffHours() |
|||
config.OffHoursStart = ui.scheduler.GetOffHoursStart() |
|||
config.OffHoursEnd = ui.scheduler.GetOffHoursEnd() |
|||
} |
|||
|
|||
return config |
|||
} |
|||
|
|||
// floatPtr is a helper function to create float64 pointers
|
|||
func floatPtr(f float64) *float64 { |
|||
return &f |
|||
} |
|||
|
|||
// RegisterUITempl registers the balance templ UI provider with the UI registry
|
|||
func RegisterUITempl(uiRegistry *types.UITemplRegistry, detector *BalanceDetector, scheduler *BalanceScheduler) { |
|||
uiProvider := NewUITemplProvider(detector, scheduler) |
|||
uiRegistry.RegisterUI(uiProvider) |
|||
|
|||
glog.V(1).Infof("✅ Registered balance task templ UI provider") |
|||
} |
|||
@ -0,0 +1,79 @@ |
|||
package erasure_coding |
|||
|
|||
import ( |
|||
"fmt" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/tasks" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// Task implements erasure coding operation to convert volumes to EC format
|
|||
type Task struct { |
|||
*tasks.BaseTask |
|||
server string |
|||
volumeID uint32 |
|||
} |
|||
|
|||
// NewTask creates a new erasure coding task instance
|
|||
func NewTask(server string, volumeID uint32) *Task { |
|||
task := &Task{ |
|||
BaseTask: tasks.NewBaseTask(types.TaskTypeErasureCoding), |
|||
server: server, |
|||
volumeID: volumeID, |
|||
} |
|||
return task |
|||
} |
|||
|
|||
// Execute executes the erasure coding task
|
|||
func (t *Task) Execute(params types.TaskParams) error { |
|||
glog.Infof("Starting erasure coding task for volume %d on server %s", t.volumeID, t.server) |
|||
|
|||
// Simulate erasure coding operation with progress updates
|
|||
steps := []struct { |
|||
name string |
|||
duration time.Duration |
|||
progress float64 |
|||
}{ |
|||
{"Analyzing volume", 2 * time.Second, 15}, |
|||
{"Creating EC shards", 5 * time.Second, 50}, |
|||
{"Verifying shards", 2 * time.Second, 75}, |
|||
{"Finalizing EC volume", 1 * time.Second, 100}, |
|||
} |
|||
|
|||
for _, step := range steps { |
|||
if t.IsCancelled() { |
|||
return fmt.Errorf("erasure coding task cancelled") |
|||
} |
|||
|
|||
glog.V(1).Infof("Erasure coding task step: %s", step.name) |
|||
t.SetProgress(step.progress) |
|||
|
|||
// Simulate work
|
|||
time.Sleep(step.duration) |
|||
} |
|||
|
|||
glog.Infof("Erasure coding task completed for volume %d on server %s", t.volumeID, t.server) |
|||
return nil |
|||
} |
|||
|
|||
// Validate validates the task parameters
|
|||
func (t *Task) Validate(params types.TaskParams) error { |
|||
if params.VolumeID == 0 { |
|||
return fmt.Errorf("volume_id is required") |
|||
} |
|||
if params.Server == "" { |
|||
return fmt.Errorf("server is required") |
|||
} |
|||
return nil |
|||
} |
|||
|
|||
// EstimateTime estimates the time needed for the task
|
|||
func (t *Task) EstimateTime(params types.TaskParams) time.Duration { |
|||
// Base time for erasure coding operation
|
|||
baseTime := 30 * time.Second |
|||
|
|||
// Could adjust based on volume size or other factors
|
|||
return baseTime |
|||
} |
|||
@ -0,0 +1,139 @@ |
|||
package erasure_coding |
|||
|
|||
import ( |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// EcDetector implements erasure coding task detection
|
|||
type EcDetector struct { |
|||
enabled bool |
|||
volumeAgeHours int |
|||
fullnessRatio float64 |
|||
scanInterval time.Duration |
|||
} |
|||
|
|||
// Compile-time interface assertions
|
|||
var ( |
|||
_ types.TaskDetector = (*EcDetector)(nil) |
|||
) |
|||
|
|||
// NewEcDetector creates a new erasure coding detector
|
|||
func NewEcDetector() *EcDetector { |
|||
return &EcDetector{ |
|||
enabled: false, // Conservative default
|
|||
volumeAgeHours: 24 * 7, // 1 week
|
|||
fullnessRatio: 0.9, // 90% full
|
|||
scanInterval: 2 * time.Hour, |
|||
} |
|||
} |
|||
|
|||
// GetTaskType returns the task type
|
|||
func (d *EcDetector) GetTaskType() types.TaskType { |
|||
return types.TaskTypeErasureCoding |
|||
} |
|||
|
|||
// ScanForTasks scans for volumes that should be converted to erasure coding
|
|||
func (d *EcDetector) ScanForTasks(volumeMetrics []*types.VolumeHealthMetrics, clusterInfo *types.ClusterInfo) ([]*types.TaskDetectionResult, error) { |
|||
if !d.enabled { |
|||
return nil, nil |
|||
} |
|||
|
|||
var results []*types.TaskDetectionResult |
|||
now := time.Now() |
|||
ageThreshold := time.Duration(d.volumeAgeHours) * time.Hour |
|||
|
|||
for _, metric := range volumeMetrics { |
|||
// Skip if already EC volume
|
|||
if metric.IsECVolume { |
|||
continue |
|||
} |
|||
|
|||
// Check age and fullness criteria
|
|||
if metric.Age >= ageThreshold && metric.FullnessRatio >= d.fullnessRatio { |
|||
// Check if volume is read-only (safe for EC conversion)
|
|||
if !metric.IsReadOnly { |
|||
continue |
|||
} |
|||
|
|||
result := &types.TaskDetectionResult{ |
|||
TaskType: types.TaskTypeErasureCoding, |
|||
VolumeID: metric.VolumeID, |
|||
Server: metric.Server, |
|||
Collection: metric.Collection, |
|||
Priority: types.TaskPriorityLow, // EC is not urgent
|
|||
Reason: "Volume is old and full enough for EC conversion", |
|||
Parameters: map[string]interface{}{ |
|||
"age_hours": int(metric.Age.Hours()), |
|||
"fullness_ratio": metric.FullnessRatio, |
|||
}, |
|||
ScheduleAt: now, |
|||
} |
|||
results = append(results, result) |
|||
} |
|||
} |
|||
|
|||
glog.V(2).Infof("EC detector found %d tasks to schedule", len(results)) |
|||
return results, nil |
|||
} |
|||
|
|||
// ScanInterval returns how often this task type should be scanned
|
|||
func (d *EcDetector) ScanInterval() time.Duration { |
|||
return d.scanInterval |
|||
} |
|||
|
|||
// IsEnabled returns whether this task type is enabled
|
|||
func (d *EcDetector) IsEnabled() bool { |
|||
return d.enabled |
|||
} |
|||
|
|||
// Configuration setters
|
|||
|
|||
func (d *EcDetector) SetEnabled(enabled bool) { |
|||
d.enabled = enabled |
|||
} |
|||
|
|||
func (d *EcDetector) SetVolumeAgeHours(hours int) { |
|||
d.volumeAgeHours = hours |
|||
} |
|||
|
|||
func (d *EcDetector) SetFullnessRatio(ratio float64) { |
|||
d.fullnessRatio = ratio |
|||
} |
|||
|
|||
func (d *EcDetector) SetScanInterval(interval time.Duration) { |
|||
d.scanInterval = interval |
|||
} |
|||
|
|||
// GetVolumeAgeHours returns the current volume age threshold in hours
|
|||
func (d *EcDetector) GetVolumeAgeHours() int { |
|||
return d.volumeAgeHours |
|||
} |
|||
|
|||
// GetFullnessRatio returns the current fullness ratio threshold
|
|||
func (d *EcDetector) GetFullnessRatio() float64 { |
|||
return d.fullnessRatio |
|||
} |
|||
|
|||
// GetScanInterval returns the scan interval
|
|||
func (d *EcDetector) GetScanInterval() time.Duration { |
|||
return d.scanInterval |
|||
} |
|||
|
|||
// ConfigureFromPolicy configures the detector based on the maintenance policy
|
|||
func (d *EcDetector) ConfigureFromPolicy(policy interface{}) { |
|||
// Type assert to the maintenance policy type we expect
|
|||
if maintenancePolicy, ok := policy.(interface { |
|||
GetECEnabled() bool |
|||
GetECVolumeAgeHours() int |
|||
GetECFullnessRatio() float64 |
|||
}); ok { |
|||
d.SetEnabled(maintenancePolicy.GetECEnabled()) |
|||
d.SetVolumeAgeHours(maintenancePolicy.GetECVolumeAgeHours()) |
|||
d.SetFullnessRatio(maintenancePolicy.GetECFullnessRatio()) |
|||
} else { |
|||
glog.V(1).Infof("Could not configure EC detector from policy: unsupported policy type") |
|||
} |
|||
} |
|||
@ -0,0 +1,81 @@ |
|||
package erasure_coding |
|||
|
|||
import ( |
|||
"fmt" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/worker/tasks" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// Factory creates erasure coding task instances
|
|||
type Factory struct { |
|||
*tasks.BaseTaskFactory |
|||
} |
|||
|
|||
// NewFactory creates a new erasure coding task factory
|
|||
func NewFactory() *Factory { |
|||
return &Factory{ |
|||
BaseTaskFactory: tasks.NewBaseTaskFactory( |
|||
types.TaskTypeErasureCoding, |
|||
[]string{"erasure_coding", "storage", "durability"}, |
|||
"Convert volumes to erasure coded format for improved durability", |
|||
), |
|||
} |
|||
} |
|||
|
|||
// Create creates a new erasure coding task instance
|
|||
func (f *Factory) Create(params types.TaskParams) (types.TaskInterface, error) { |
|||
// Validate parameters
|
|||
if params.VolumeID == 0 { |
|||
return nil, fmt.Errorf("volume_id is required") |
|||
} |
|||
if params.Server == "" { |
|||
return nil, fmt.Errorf("server is required") |
|||
} |
|||
|
|||
task := NewTask(params.Server, params.VolumeID) |
|||
task.SetEstimatedDuration(task.EstimateTime(params)) |
|||
|
|||
return task, nil |
|||
} |
|||
|
|||
// Shared detector and scheduler instances
|
|||
var ( |
|||
sharedDetector *EcDetector |
|||
sharedScheduler *Scheduler |
|||
) |
|||
|
|||
// getSharedInstances returns the shared detector and scheduler instances
|
|||
func getSharedInstances() (*EcDetector, *Scheduler) { |
|||
if sharedDetector == nil { |
|||
sharedDetector = NewEcDetector() |
|||
} |
|||
if sharedScheduler == nil { |
|||
sharedScheduler = NewScheduler() |
|||
} |
|||
return sharedDetector, sharedScheduler |
|||
} |
|||
|
|||
// GetSharedInstances returns the shared detector and scheduler instances (public access)
|
|||
func GetSharedInstances() (*EcDetector, *Scheduler) { |
|||
return getSharedInstances() |
|||
} |
|||
|
|||
// Auto-register this task when the package is imported
|
|||
func init() { |
|||
factory := NewFactory() |
|||
tasks.AutoRegister(types.TaskTypeErasureCoding, factory) |
|||
|
|||
// Get shared instances for all registrations
|
|||
detector, scheduler := getSharedInstances() |
|||
|
|||
// Register with types registry
|
|||
tasks.AutoRegisterTypes(func(registry *types.TaskRegistry) { |
|||
registry.RegisterTask(detector, scheduler) |
|||
}) |
|||
|
|||
// Register with UI registry using the same instances
|
|||
tasks.AutoRegisterUI(func(uiRegistry *types.UIRegistry) { |
|||
RegisterUI(uiRegistry, detector, scheduler) |
|||
}) |
|||
} |
|||
@ -0,0 +1,114 @@ |
|||
package erasure_coding |
|||
|
|||
import ( |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// Scheduler implements erasure coding task scheduling
|
|||
type Scheduler struct { |
|||
maxConcurrent int |
|||
enabled bool |
|||
} |
|||
|
|||
// NewScheduler creates a new erasure coding scheduler
|
|||
func NewScheduler() *Scheduler { |
|||
return &Scheduler{ |
|||
maxConcurrent: 1, // Conservative default
|
|||
enabled: false, // Conservative default
|
|||
} |
|||
} |
|||
|
|||
// GetTaskType returns the task type
|
|||
func (s *Scheduler) GetTaskType() types.TaskType { |
|||
return types.TaskTypeErasureCoding |
|||
} |
|||
|
|||
// CanScheduleNow determines if an erasure coding task can be scheduled now
|
|||
func (s *Scheduler) CanScheduleNow(task *types.Task, runningTasks []*types.Task, availableWorkers []*types.Worker) bool { |
|||
if !s.enabled { |
|||
return false |
|||
} |
|||
|
|||
// Check if we have available workers
|
|||
if len(availableWorkers) == 0 { |
|||
return false |
|||
} |
|||
|
|||
// Count running EC tasks
|
|||
runningCount := 0 |
|||
for _, runningTask := range runningTasks { |
|||
if runningTask.Type == types.TaskTypeErasureCoding { |
|||
runningCount++ |
|||
} |
|||
} |
|||
|
|||
// Check concurrency limit
|
|||
if runningCount >= s.maxConcurrent { |
|||
glog.V(3).Infof("EC scheduler: at concurrency limit (%d/%d)", runningCount, s.maxConcurrent) |
|||
return false |
|||
} |
|||
|
|||
// Check if any worker can handle EC tasks
|
|||
for _, worker := range availableWorkers { |
|||
for _, capability := range worker.Capabilities { |
|||
if capability == types.TaskTypeErasureCoding { |
|||
glog.V(3).Infof("EC scheduler: can schedule task for volume %d", task.VolumeID) |
|||
return true |
|||
} |
|||
} |
|||
} |
|||
|
|||
return false |
|||
} |
|||
|
|||
// GetMaxConcurrent returns the maximum number of concurrent tasks
|
|||
func (s *Scheduler) GetMaxConcurrent() int { |
|||
return s.maxConcurrent |
|||
} |
|||
|
|||
// GetDefaultRepeatInterval returns the default interval to wait before repeating EC tasks
|
|||
func (s *Scheduler) GetDefaultRepeatInterval() time.Duration { |
|||
return 24 * time.Hour // Don't repeat EC for 24 hours
|
|||
} |
|||
|
|||
// GetPriority returns the priority for this task
|
|||
func (s *Scheduler) GetPriority(task *types.Task) types.TaskPriority { |
|||
return types.TaskPriorityLow // EC is not urgent
|
|||
} |
|||
|
|||
// WasTaskRecentlyCompleted checks if a similar task was recently completed
|
|||
func (s *Scheduler) WasTaskRecentlyCompleted(task *types.Task, completedTasks []*types.Task, now time.Time) bool { |
|||
// Don't repeat EC for 24 hours
|
|||
interval := 24 * time.Hour |
|||
cutoff := now.Add(-interval) |
|||
|
|||
for _, completedTask := range completedTasks { |
|||
if completedTask.Type == types.TaskTypeErasureCoding && |
|||
completedTask.VolumeID == task.VolumeID && |
|||
completedTask.Server == task.Server && |
|||
completedTask.Status == types.TaskStatusCompleted && |
|||
completedTask.CompletedAt != nil && |
|||
completedTask.CompletedAt.After(cutoff) { |
|||
return true |
|||
} |
|||
} |
|||
return false |
|||
} |
|||
|
|||
// IsEnabled returns whether this task type is enabled
|
|||
func (s *Scheduler) IsEnabled() bool { |
|||
return s.enabled |
|||
} |
|||
|
|||
// Configuration setters
|
|||
|
|||
func (s *Scheduler) SetEnabled(enabled bool) { |
|||
s.enabled = enabled |
|||
} |
|||
|
|||
func (s *Scheduler) SetMaxConcurrent(max int) { |
|||
s.maxConcurrent = max |
|||
} |
|||
@ -0,0 +1,309 @@ |
|||
package erasure_coding |
|||
|
|||
import ( |
|||
"fmt" |
|||
"html/template" |
|||
"strconv" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// UIProvider provides the UI for erasure coding task configuration
|
|||
type UIProvider struct { |
|||
detector *EcDetector |
|||
scheduler *Scheduler |
|||
} |
|||
|
|||
// NewUIProvider creates a new erasure coding UI provider
|
|||
func NewUIProvider(detector *EcDetector, scheduler *Scheduler) *UIProvider { |
|||
return &UIProvider{ |
|||
detector: detector, |
|||
scheduler: scheduler, |
|||
} |
|||
} |
|||
|
|||
// GetTaskType returns the task type
|
|||
func (ui *UIProvider) GetTaskType() types.TaskType { |
|||
return types.TaskTypeErasureCoding |
|||
} |
|||
|
|||
// GetDisplayName returns the human-readable name
|
|||
func (ui *UIProvider) GetDisplayName() string { |
|||
return "Erasure Coding" |
|||
} |
|||
|
|||
// GetDescription returns a description of what this task does
|
|||
func (ui *UIProvider) GetDescription() string { |
|||
return "Converts volumes to erasure coded format for improved data durability and fault tolerance" |
|||
} |
|||
|
|||
// GetIcon returns the icon CSS class for this task type
|
|||
func (ui *UIProvider) GetIcon() string { |
|||
return "fas fa-shield-alt text-info" |
|||
} |
|||
|
|||
// ErasureCodingConfig represents the erasure coding configuration
|
|||
type ErasureCodingConfig struct { |
|||
Enabled bool `json:"enabled"` |
|||
VolumeAgeHoursSeconds int `json:"volume_age_hours_seconds"` |
|||
FullnessRatio float64 `json:"fullness_ratio"` |
|||
ScanIntervalSeconds int `json:"scan_interval_seconds"` |
|||
MaxConcurrent int `json:"max_concurrent"` |
|||
ShardCount int `json:"shard_count"` |
|||
ParityCount int `json:"parity_count"` |
|||
CollectionFilter string `json:"collection_filter"` |
|||
} |
|||
|
|||
// Helper functions for duration conversion
|
|||
func secondsToDuration(seconds int) time.Duration { |
|||
return time.Duration(seconds) * time.Second |
|||
} |
|||
|
|||
func durationToSeconds(d time.Duration) int { |
|||
return int(d.Seconds()) |
|||
} |
|||
|
|||
// formatDurationForUser formats seconds as a user-friendly duration string
|
|||
func formatDurationForUser(seconds int) string { |
|||
d := secondsToDuration(seconds) |
|||
if d < time.Minute { |
|||
return fmt.Sprintf("%ds", seconds) |
|||
} |
|||
if d < time.Hour { |
|||
return fmt.Sprintf("%.0fm", d.Minutes()) |
|||
} |
|||
if d < 24*time.Hour { |
|||
return fmt.Sprintf("%.1fh", d.Hours()) |
|||
} |
|||
return fmt.Sprintf("%.1fd", d.Hours()/24) |
|||
} |
|||
|
|||
// RenderConfigForm renders the configuration form HTML
|
|||
func (ui *UIProvider) RenderConfigForm(currentConfig interface{}) (template.HTML, error) { |
|||
config := ui.getCurrentECConfig() |
|||
|
|||
// Build form using the FormBuilder helper
|
|||
form := types.NewFormBuilder() |
|||
|
|||
// Detection Settings
|
|||
form.AddCheckboxField( |
|||
"enabled", |
|||
"Enable Erasure Coding Tasks", |
|||
"Whether erasure coding tasks should be automatically created", |
|||
config.Enabled, |
|||
) |
|||
|
|||
form.AddNumberField( |
|||
"volume_age_hours_seconds", |
|||
"Volume Age Threshold", |
|||
"Only apply erasure coding to volumes older than this duration", |
|||
float64(config.VolumeAgeHoursSeconds), |
|||
true, |
|||
) |
|||
|
|||
form.AddNumberField( |
|||
"scan_interval_seconds", |
|||
"Scan Interval", |
|||
"How often to scan for volumes needing erasure coding", |
|||
float64(config.ScanIntervalSeconds), |
|||
true, |
|||
) |
|||
|
|||
// Scheduling Settings
|
|||
form.AddNumberField( |
|||
"max_concurrent", |
|||
"Max Concurrent Tasks", |
|||
"Maximum number of erasure coding tasks that can run simultaneously", |
|||
float64(config.MaxConcurrent), |
|||
true, |
|||
) |
|||
|
|||
// Erasure Coding Parameters
|
|||
form.AddNumberField( |
|||
"shard_count", |
|||
"Data Shards", |
|||
"Number of data shards for erasure coding (recommended: 10)", |
|||
float64(config.ShardCount), |
|||
true, |
|||
) |
|||
|
|||
form.AddNumberField( |
|||
"parity_count", |
|||
"Parity Shards", |
|||
"Number of parity shards for erasure coding (recommended: 4)", |
|||
float64(config.ParityCount), |
|||
true, |
|||
) |
|||
|
|||
// Generate organized form sections using Bootstrap components
|
|||
html := ` |
|||
<div class="row"> |
|||
<div class="col-12"> |
|||
<div class="card mb-4"> |
|||
<div class="card-header"> |
|||
<h5 class="mb-0"> |
|||
<i class="fas fa-shield-alt me-2"></i> |
|||
Erasure Coding Configuration |
|||
</h5> |
|||
</div> |
|||
<div class="card-body"> |
|||
` + string(form.Build()) + ` |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<div class="row"> |
|||
<div class="col-12"> |
|||
<div class="card mb-3"> |
|||
<div class="card-header"> |
|||
<h5 class="mb-0"> |
|||
<i class="fas fa-info-circle me-2"></i> |
|||
Performance Impact |
|||
</h5> |
|||
</div> |
|||
<div class="card-body"> |
|||
<div class="alert alert-info" role="alert"> |
|||
<h6 class="alert-heading">Important Notes:</h6> |
|||
<p class="mb-2"><strong>Performance:</strong> Erasure coding is CPU and I/O intensive. Consider running during off-peak hours.</p> |
|||
<p class="mb-0"><strong>Durability:</strong> With ` + fmt.Sprintf("%d+%d", config.ShardCount, config.ParityCount) + ` configuration, can tolerate up to ` + fmt.Sprintf("%d", config.ParityCount) + ` shard failures.</p> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div>` |
|||
|
|||
return template.HTML(html), nil |
|||
} |
|||
|
|||
// ParseConfigForm parses form data into configuration
|
|||
func (ui *UIProvider) ParseConfigForm(formData map[string][]string) (interface{}, error) { |
|||
config := &ErasureCodingConfig{} |
|||
|
|||
// Parse enabled
|
|||
config.Enabled = len(formData["enabled"]) > 0 |
|||
|
|||
// Parse volume age hours
|
|||
if values, ok := formData["volume_age_hours_seconds"]; ok && len(values) > 0 { |
|||
hours, err := strconv.Atoi(values[0]) |
|||
if err != nil { |
|||
return nil, fmt.Errorf("invalid volume age hours: %v", err) |
|||
} |
|||
config.VolumeAgeHoursSeconds = hours |
|||
} |
|||
|
|||
// Parse scan interval
|
|||
if values, ok := formData["scan_interval_seconds"]; ok && len(values) > 0 { |
|||
interval, err := strconv.Atoi(values[0]) |
|||
if err != nil { |
|||
return nil, fmt.Errorf("invalid scan interval: %v", err) |
|||
} |
|||
config.ScanIntervalSeconds = interval |
|||
} |
|||
|
|||
// Parse max concurrent
|
|||
if values, ok := formData["max_concurrent"]; ok && len(values) > 0 { |
|||
maxConcurrent, err := strconv.Atoi(values[0]) |
|||
if err != nil { |
|||
return nil, fmt.Errorf("invalid max concurrent: %v", err) |
|||
} |
|||
if maxConcurrent < 1 { |
|||
return nil, fmt.Errorf("max concurrent must be at least 1") |
|||
} |
|||
config.MaxConcurrent = maxConcurrent |
|||
} |
|||
|
|||
// Parse shard count
|
|||
if values, ok := formData["shard_count"]; ok && len(values) > 0 { |
|||
shardCount, err := strconv.Atoi(values[0]) |
|||
if err != nil { |
|||
return nil, fmt.Errorf("invalid shard count: %v", err) |
|||
} |
|||
if shardCount < 1 { |
|||
return nil, fmt.Errorf("shard count must be at least 1") |
|||
} |
|||
config.ShardCount = shardCount |
|||
} |
|||
|
|||
// Parse parity count
|
|||
if values, ok := formData["parity_count"]; ok && len(values) > 0 { |
|||
parityCount, err := strconv.Atoi(values[0]) |
|||
if err != nil { |
|||
return nil, fmt.Errorf("invalid parity count: %v", err) |
|||
} |
|||
if parityCount < 1 { |
|||
return nil, fmt.Errorf("parity count must be at least 1") |
|||
} |
|||
config.ParityCount = parityCount |
|||
} |
|||
|
|||
return config, nil |
|||
} |
|||
|
|||
// GetCurrentConfig returns the current configuration
|
|||
func (ui *UIProvider) GetCurrentConfig() interface{} { |
|||
return ui.getCurrentECConfig() |
|||
} |
|||
|
|||
// ApplyConfig applies the new configuration
|
|||
func (ui *UIProvider) ApplyConfig(config interface{}) error { |
|||
ecConfig, ok := config.(ErasureCodingConfig) |
|||
if !ok { |
|||
return fmt.Errorf("invalid config type, expected ErasureCodingConfig") |
|||
} |
|||
|
|||
// Apply to detector
|
|||
if ui.detector != nil { |
|||
ui.detector.SetEnabled(ecConfig.Enabled) |
|||
ui.detector.SetVolumeAgeHours(ecConfig.VolumeAgeHoursSeconds) |
|||
ui.detector.SetScanInterval(secondsToDuration(ecConfig.ScanIntervalSeconds)) |
|||
} |
|||
|
|||
// Apply to scheduler
|
|||
if ui.scheduler != nil { |
|||
ui.scheduler.SetEnabled(ecConfig.Enabled) |
|||
ui.scheduler.SetMaxConcurrent(ecConfig.MaxConcurrent) |
|||
} |
|||
|
|||
glog.V(1).Infof("Applied erasure coding configuration: enabled=%v, age_threshold=%v, max_concurrent=%d, shards=%d+%d", |
|||
ecConfig.Enabled, ecConfig.VolumeAgeHoursSeconds, ecConfig.MaxConcurrent, ecConfig.ShardCount, ecConfig.ParityCount) |
|||
|
|||
return nil |
|||
} |
|||
|
|||
// getCurrentECConfig gets the current configuration from detector and scheduler
|
|||
func (ui *UIProvider) getCurrentECConfig() ErasureCodingConfig { |
|||
config := ErasureCodingConfig{ |
|||
// Default values (fallback if detectors/schedulers are nil)
|
|||
Enabled: true, |
|||
VolumeAgeHoursSeconds: 24 * 3600, // 24 hours in seconds
|
|||
ScanIntervalSeconds: 2 * 3600, // 2 hours in seconds
|
|||
MaxConcurrent: 1, |
|||
ShardCount: 10, |
|||
ParityCount: 4, |
|||
} |
|||
|
|||
// Get current values from detector
|
|||
if ui.detector != nil { |
|||
config.Enabled = ui.detector.IsEnabled() |
|||
config.VolumeAgeHoursSeconds = ui.detector.GetVolumeAgeHours() |
|||
config.ScanIntervalSeconds = durationToSeconds(ui.detector.ScanInterval()) |
|||
} |
|||
|
|||
// Get current values from scheduler
|
|||
if ui.scheduler != nil { |
|||
config.MaxConcurrent = ui.scheduler.GetMaxConcurrent() |
|||
} |
|||
|
|||
return config |
|||
} |
|||
|
|||
// RegisterUI registers the erasure coding UI provider with the UI registry
|
|||
func RegisterUI(uiRegistry *types.UIRegistry, detector *EcDetector, scheduler *Scheduler) { |
|||
uiProvider := NewUIProvider(detector, scheduler) |
|||
uiRegistry.RegisterUI(uiProvider) |
|||
|
|||
glog.V(1).Infof("✅ Registered erasure coding task UI provider") |
|||
} |
|||
@ -0,0 +1,319 @@ |
|||
package erasure_coding |
|||
|
|||
import ( |
|||
"fmt" |
|||
"strconv" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/admin/view/components" |
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// Helper function to format seconds as duration string
|
|||
func formatDurationFromSeconds(seconds int) string { |
|||
d := time.Duration(seconds) * time.Second |
|||
return d.String() |
|||
} |
|||
|
|||
// Helper function to convert value and unit to seconds
|
|||
func valueAndUnitToSeconds(value float64, unit string) int { |
|||
switch unit { |
|||
case "days": |
|||
return int(value * 24 * 60 * 60) |
|||
case "hours": |
|||
return int(value * 60 * 60) |
|||
case "minutes": |
|||
return int(value * 60) |
|||
default: |
|||
return int(value * 60) // Default to minutes
|
|||
} |
|||
} |
|||
|
|||
// UITemplProvider provides the templ-based UI for erasure coding task configuration
|
|||
type UITemplProvider struct { |
|||
detector *EcDetector |
|||
scheduler *Scheduler |
|||
} |
|||
|
|||
// NewUITemplProvider creates a new erasure coding templ UI provider
|
|||
func NewUITemplProvider(detector *EcDetector, scheduler *Scheduler) *UITemplProvider { |
|||
return &UITemplProvider{ |
|||
detector: detector, |
|||
scheduler: scheduler, |
|||
} |
|||
} |
|||
|
|||
// ErasureCodingConfig is defined in ui.go - we reuse it
|
|||
|
|||
// GetTaskType returns the task type
|
|||
func (ui *UITemplProvider) GetTaskType() types.TaskType { |
|||
return types.TaskTypeErasureCoding |
|||
} |
|||
|
|||
// GetDisplayName returns the human-readable name
|
|||
func (ui *UITemplProvider) GetDisplayName() string { |
|||
return "Erasure Coding" |
|||
} |
|||
|
|||
// GetDescription returns a description of what this task does
|
|||
func (ui *UITemplProvider) GetDescription() string { |
|||
return "Converts replicated volumes to erasure-coded format for efficient storage" |
|||
} |
|||
|
|||
// GetIcon returns the icon CSS class for this task type
|
|||
func (ui *UITemplProvider) GetIcon() string { |
|||
return "fas fa-shield-alt text-info" |
|||
} |
|||
|
|||
// RenderConfigSections renders the configuration as templ section data
|
|||
func (ui *UITemplProvider) RenderConfigSections(currentConfig interface{}) ([]components.ConfigSectionData, error) { |
|||
config := ui.getCurrentECConfig() |
|||
|
|||
// Detection settings section
|
|||
detectionSection := components.ConfigSectionData{ |
|||
Title: "Detection Settings", |
|||
Icon: "fas fa-search", |
|||
Description: "Configure when erasure coding tasks should be triggered", |
|||
Fields: []interface{}{ |
|||
components.CheckboxFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "enabled", |
|||
Label: "Enable Erasure Coding Tasks", |
|||
Description: "Whether erasure coding tasks should be automatically created", |
|||
}, |
|||
Checked: config.Enabled, |
|||
}, |
|||
components.DurationInputFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "scan_interval", |
|||
Label: "Scan Interval", |
|||
Description: "How often to scan for volumes needing erasure coding", |
|||
Required: true, |
|||
}, |
|||
Seconds: config.ScanIntervalSeconds, |
|||
}, |
|||
components.DurationInputFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "volume_age_threshold", |
|||
Label: "Volume Age Threshold", |
|||
Description: "Only apply erasure coding to volumes older than this age", |
|||
Required: true, |
|||
}, |
|||
Seconds: config.VolumeAgeHoursSeconds, |
|||
}, |
|||
}, |
|||
} |
|||
|
|||
// Erasure coding parameters section
|
|||
paramsSection := components.ConfigSectionData{ |
|||
Title: "Erasure Coding Parameters", |
|||
Icon: "fas fa-cogs", |
|||
Description: "Configure erasure coding scheme and performance", |
|||
Fields: []interface{}{ |
|||
components.NumberFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "data_shards", |
|||
Label: "Data Shards", |
|||
Description: "Number of data shards in the erasure coding scheme", |
|||
Required: true, |
|||
}, |
|||
Value: float64(config.ShardCount), |
|||
Step: "1", |
|||
Min: floatPtr(1), |
|||
Max: floatPtr(16), |
|||
}, |
|||
components.NumberFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "parity_shards", |
|||
Label: "Parity Shards", |
|||
Description: "Number of parity shards (determines fault tolerance)", |
|||
Required: true, |
|||
}, |
|||
Value: float64(config.ParityCount), |
|||
Step: "1", |
|||
Min: floatPtr(1), |
|||
Max: floatPtr(16), |
|||
}, |
|||
components.NumberFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "max_concurrent", |
|||
Label: "Max Concurrent Tasks", |
|||
Description: "Maximum number of erasure coding tasks that can run simultaneously", |
|||
Required: true, |
|||
}, |
|||
Value: float64(config.MaxConcurrent), |
|||
Step: "1", |
|||
Min: floatPtr(1), |
|||
}, |
|||
}, |
|||
} |
|||
|
|||
// Performance impact info section
|
|||
infoSection := components.ConfigSectionData{ |
|||
Title: "Performance Impact", |
|||
Icon: "fas fa-info-circle", |
|||
Description: "Important information about erasure coding operations", |
|||
Fields: []interface{}{ |
|||
components.TextFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "durability_info", |
|||
Label: "Durability", |
|||
Description: fmt.Sprintf("With %d+%d configuration, can tolerate up to %d shard failures", |
|||
config.ShardCount, config.ParityCount, config.ParityCount), |
|||
}, |
|||
Value: "High durability with space efficiency", |
|||
}, |
|||
components.TextFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "performance_info", |
|||
Label: "Performance Note", |
|||
Description: "Erasure coding is CPU and I/O intensive. Consider running during off-peak hours", |
|||
}, |
|||
Value: "Schedule during low-traffic periods", |
|||
}, |
|||
}, |
|||
} |
|||
|
|||
return []components.ConfigSectionData{detectionSection, paramsSection, infoSection}, nil |
|||
} |
|||
|
|||
// ParseConfigForm parses form data into configuration
|
|||
func (ui *UITemplProvider) ParseConfigForm(formData map[string][]string) (interface{}, error) { |
|||
config := &ErasureCodingConfig{} |
|||
|
|||
// Parse enabled checkbox
|
|||
config.Enabled = len(formData["enabled"]) > 0 && formData["enabled"][0] == "on" |
|||
|
|||
// Parse volume age threshold
|
|||
if valueStr := formData["volume_age_threshold"]; len(valueStr) > 0 { |
|||
if value, err := strconv.ParseFloat(valueStr[0], 64); err != nil { |
|||
return nil, fmt.Errorf("invalid volume age threshold value: %v", err) |
|||
} else { |
|||
unit := "hours" // default
|
|||
if unitStr := formData["volume_age_threshold_unit"]; len(unitStr) > 0 { |
|||
unit = unitStr[0] |
|||
} |
|||
config.VolumeAgeHoursSeconds = valueAndUnitToSeconds(value, unit) |
|||
} |
|||
} |
|||
|
|||
// Parse scan interval
|
|||
if valueStr := formData["scan_interval"]; len(valueStr) > 0 { |
|||
if value, err := strconv.ParseFloat(valueStr[0], 64); err != nil { |
|||
return nil, fmt.Errorf("invalid scan interval value: %v", err) |
|||
} else { |
|||
unit := "hours" // default
|
|||
if unitStr := formData["scan_interval_unit"]; len(unitStr) > 0 { |
|||
unit = unitStr[0] |
|||
} |
|||
config.ScanIntervalSeconds = valueAndUnitToSeconds(value, unit) |
|||
} |
|||
} |
|||
|
|||
// Parse data shards
|
|||
if shardsStr := formData["data_shards"]; len(shardsStr) > 0 { |
|||
if shards, err := strconv.Atoi(shardsStr[0]); err != nil { |
|||
return nil, fmt.Errorf("invalid data shards: %v", err) |
|||
} else if shards < 1 || shards > 16 { |
|||
return nil, fmt.Errorf("data shards must be between 1 and 16") |
|||
} else { |
|||
config.ShardCount = shards |
|||
} |
|||
} |
|||
|
|||
// Parse parity shards
|
|||
if shardsStr := formData["parity_shards"]; len(shardsStr) > 0 { |
|||
if shards, err := strconv.Atoi(shardsStr[0]); err != nil { |
|||
return nil, fmt.Errorf("invalid parity shards: %v", err) |
|||
} else if shards < 1 || shards > 16 { |
|||
return nil, fmt.Errorf("parity shards must be between 1 and 16") |
|||
} else { |
|||
config.ParityCount = shards |
|||
} |
|||
} |
|||
|
|||
// Parse max concurrent
|
|||
if concurrentStr := formData["max_concurrent"]; len(concurrentStr) > 0 { |
|||
if concurrent, err := strconv.Atoi(concurrentStr[0]); err != nil { |
|||
return nil, fmt.Errorf("invalid max concurrent: %v", err) |
|||
} else if concurrent < 1 { |
|||
return nil, fmt.Errorf("max concurrent must be at least 1") |
|||
} else { |
|||
config.MaxConcurrent = concurrent |
|||
} |
|||
} |
|||
|
|||
return config, nil |
|||
} |
|||
|
|||
// GetCurrentConfig returns the current configuration
|
|||
func (ui *UITemplProvider) GetCurrentConfig() interface{} { |
|||
return ui.getCurrentECConfig() |
|||
} |
|||
|
|||
// ApplyConfig applies the new configuration
|
|||
func (ui *UITemplProvider) ApplyConfig(config interface{}) error { |
|||
ecConfig, ok := config.(*ErasureCodingConfig) |
|||
if !ok { |
|||
return fmt.Errorf("invalid config type, expected *ErasureCodingConfig") |
|||
} |
|||
|
|||
// Apply to detector
|
|||
if ui.detector != nil { |
|||
ui.detector.SetEnabled(ecConfig.Enabled) |
|||
ui.detector.SetVolumeAgeHours(ecConfig.VolumeAgeHoursSeconds) |
|||
ui.detector.SetScanInterval(time.Duration(ecConfig.ScanIntervalSeconds) * time.Second) |
|||
} |
|||
|
|||
// Apply to scheduler
|
|||
if ui.scheduler != nil { |
|||
ui.scheduler.SetMaxConcurrent(ecConfig.MaxConcurrent) |
|||
ui.scheduler.SetEnabled(ecConfig.Enabled) |
|||
} |
|||
|
|||
glog.V(1).Infof("Applied erasure coding configuration: enabled=%v, age_threshold=%ds, max_concurrent=%d", |
|||
ecConfig.Enabled, ecConfig.VolumeAgeHoursSeconds, ecConfig.MaxConcurrent) |
|||
|
|||
return nil |
|||
} |
|||
|
|||
// getCurrentECConfig gets the current configuration from detector and scheduler
|
|||
func (ui *UITemplProvider) getCurrentECConfig() *ErasureCodingConfig { |
|||
config := &ErasureCodingConfig{ |
|||
// Default values (fallback if detectors/schedulers are nil)
|
|||
Enabled: true, |
|||
VolumeAgeHoursSeconds: int((24 * time.Hour).Seconds()), |
|||
ScanIntervalSeconds: int((2 * time.Hour).Seconds()), |
|||
MaxConcurrent: 1, |
|||
ShardCount: 10, |
|||
ParityCount: 4, |
|||
} |
|||
|
|||
// Get current values from detector
|
|||
if ui.detector != nil { |
|||
config.Enabled = ui.detector.IsEnabled() |
|||
config.VolumeAgeHoursSeconds = ui.detector.GetVolumeAgeHours() |
|||
config.ScanIntervalSeconds = int(ui.detector.ScanInterval().Seconds()) |
|||
} |
|||
|
|||
// Get current values from scheduler
|
|||
if ui.scheduler != nil { |
|||
config.MaxConcurrent = ui.scheduler.GetMaxConcurrent() |
|||
} |
|||
|
|||
return config |
|||
} |
|||
|
|||
// floatPtr is a helper function to create float64 pointers
|
|||
func floatPtr(f float64) *float64 { |
|||
return &f |
|||
} |
|||
|
|||
// RegisterUITempl registers the erasure coding templ UI provider with the UI registry
|
|||
func RegisterUITempl(uiRegistry *types.UITemplRegistry, detector *EcDetector, scheduler *Scheduler) { |
|||
uiProvider := NewUITemplProvider(detector, scheduler) |
|||
uiRegistry.RegisterUI(uiProvider) |
|||
|
|||
glog.V(1).Infof("✅ Registered erasure coding task templ UI provider") |
|||
} |
|||
@ -0,0 +1,110 @@ |
|||
package tasks |
|||
|
|||
import ( |
|||
"sync" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
var ( |
|||
globalRegistry *TaskRegistry |
|||
globalTypesRegistry *types.TaskRegistry |
|||
globalUIRegistry *types.UIRegistry |
|||
registryOnce sync.Once |
|||
typesRegistryOnce sync.Once |
|||
uiRegistryOnce sync.Once |
|||
) |
|||
|
|||
// GetGlobalRegistry returns the global task registry (singleton)
|
|||
func GetGlobalRegistry() *TaskRegistry { |
|||
registryOnce.Do(func() { |
|||
globalRegistry = NewTaskRegistry() |
|||
glog.V(1).Infof("Created global task registry") |
|||
}) |
|||
return globalRegistry |
|||
} |
|||
|
|||
// GetGlobalTypesRegistry returns the global types registry (singleton)
|
|||
func GetGlobalTypesRegistry() *types.TaskRegistry { |
|||
typesRegistryOnce.Do(func() { |
|||
globalTypesRegistry = types.NewTaskRegistry() |
|||
glog.V(1).Infof("Created global types registry") |
|||
}) |
|||
return globalTypesRegistry |
|||
} |
|||
|
|||
// GetGlobalUIRegistry returns the global UI registry (singleton)
|
|||
func GetGlobalUIRegistry() *types.UIRegistry { |
|||
uiRegistryOnce.Do(func() { |
|||
globalUIRegistry = types.NewUIRegistry() |
|||
glog.V(1).Infof("Created global UI registry") |
|||
}) |
|||
return globalUIRegistry |
|||
} |
|||
|
|||
// AutoRegister registers a task directly with the global registry
|
|||
func AutoRegister(taskType types.TaskType, factory types.TaskFactory) { |
|||
registry := GetGlobalRegistry() |
|||
registry.Register(taskType, factory) |
|||
glog.V(1).Infof("Auto-registered task type: %s", taskType) |
|||
} |
|||
|
|||
// AutoRegisterTypes registers a task with the global types registry
|
|||
func AutoRegisterTypes(registerFunc func(*types.TaskRegistry)) { |
|||
registry := GetGlobalTypesRegistry() |
|||
registerFunc(registry) |
|||
glog.V(1).Infof("Auto-registered task with types registry") |
|||
} |
|||
|
|||
// AutoRegisterUI registers a UI provider with the global UI registry
|
|||
func AutoRegisterUI(registerFunc func(*types.UIRegistry)) { |
|||
registry := GetGlobalUIRegistry() |
|||
registerFunc(registry) |
|||
glog.V(1).Infof("Auto-registered task UI provider") |
|||
} |
|||
|
|||
// SetDefaultCapabilitiesFromRegistry sets the default worker capabilities
|
|||
// based on all registered task types
|
|||
func SetDefaultCapabilitiesFromRegistry() { |
|||
typesRegistry := GetGlobalTypesRegistry() |
|||
|
|||
var capabilities []types.TaskType |
|||
for taskType := range typesRegistry.GetAllDetectors() { |
|||
capabilities = append(capabilities, taskType) |
|||
} |
|||
|
|||
// Set the default capabilities in the types package
|
|||
types.SetDefaultCapabilities(capabilities) |
|||
|
|||
glog.V(1).Infof("Set default worker capabilities from registry: %v", capabilities) |
|||
} |
|||
|
|||
// BuildMaintenancePolicyFromTasks creates a maintenance policy with default configurations
|
|||
// from all registered tasks using their UI providers
|
|||
func BuildMaintenancePolicyFromTasks() *types.MaintenancePolicy { |
|||
policy := types.NewMaintenancePolicy() |
|||
|
|||
// Get all registered task types from the UI registry
|
|||
uiRegistry := GetGlobalUIRegistry() |
|||
|
|||
for taskType, provider := range uiRegistry.GetAllProviders() { |
|||
// Get the default configuration from the UI provider
|
|||
defaultConfig := provider.GetCurrentConfig() |
|||
|
|||
// Set the configuration in the policy
|
|||
policy.SetTaskConfig(taskType, defaultConfig) |
|||
|
|||
glog.V(3).Infof("Added default config for task type %s to policy", taskType) |
|||
} |
|||
|
|||
glog.V(2).Infof("Built maintenance policy with %d task configurations", len(policy.TaskConfigs)) |
|||
return policy |
|||
} |
|||
|
|||
// SetMaintenancePolicyFromTasks sets the default maintenance policy from registered tasks
|
|||
func SetMaintenancePolicyFromTasks() { |
|||
// This function can be called to initialize the policy from registered tasks
|
|||
// For now, we'll just log that this should be called by the integration layer
|
|||
glog.V(1).Infof("SetMaintenancePolicyFromTasks called - policy should be built by the integration layer") |
|||
} |
|||
@ -0,0 +1,252 @@ |
|||
package tasks |
|||
|
|||
import ( |
|||
"context" |
|||
"sync" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// BaseTask provides common functionality for all tasks
|
|||
type BaseTask struct { |
|||
taskType types.TaskType |
|||
progress float64 |
|||
cancelled bool |
|||
mutex sync.RWMutex |
|||
startTime time.Time |
|||
estimatedDuration time.Duration |
|||
} |
|||
|
|||
// NewBaseTask creates a new base task
|
|||
func NewBaseTask(taskType types.TaskType) *BaseTask { |
|||
return &BaseTask{ |
|||
taskType: taskType, |
|||
progress: 0.0, |
|||
cancelled: false, |
|||
} |
|||
} |
|||
|
|||
// Type returns the task type
|
|||
func (t *BaseTask) Type() types.TaskType { |
|||
return t.taskType |
|||
} |
|||
|
|||
// GetProgress returns the current progress (0.0 to 100.0)
|
|||
func (t *BaseTask) GetProgress() float64 { |
|||
t.mutex.RLock() |
|||
defer t.mutex.RUnlock() |
|||
return t.progress |
|||
} |
|||
|
|||
// SetProgress sets the current progress
|
|||
func (t *BaseTask) SetProgress(progress float64) { |
|||
t.mutex.Lock() |
|||
defer t.mutex.Unlock() |
|||
if progress < 0 { |
|||
progress = 0 |
|||
} |
|||
if progress > 100 { |
|||
progress = 100 |
|||
} |
|||
t.progress = progress |
|||
} |
|||
|
|||
// Cancel cancels the task
|
|||
func (t *BaseTask) Cancel() error { |
|||
t.mutex.Lock() |
|||
defer t.mutex.Unlock() |
|||
t.cancelled = true |
|||
return nil |
|||
} |
|||
|
|||
// IsCancelled returns whether the task is cancelled
|
|||
func (t *BaseTask) IsCancelled() bool { |
|||
t.mutex.RLock() |
|||
defer t.mutex.RUnlock() |
|||
return t.cancelled |
|||
} |
|||
|
|||
// SetStartTime sets the task start time
|
|||
func (t *BaseTask) SetStartTime(startTime time.Time) { |
|||
t.mutex.Lock() |
|||
defer t.mutex.Unlock() |
|||
t.startTime = startTime |
|||
} |
|||
|
|||
// GetStartTime returns the task start time
|
|||
func (t *BaseTask) GetStartTime() time.Time { |
|||
t.mutex.RLock() |
|||
defer t.mutex.RUnlock() |
|||
return t.startTime |
|||
} |
|||
|
|||
// SetEstimatedDuration sets the estimated duration
|
|||
func (t *BaseTask) SetEstimatedDuration(duration time.Duration) { |
|||
t.mutex.Lock() |
|||
defer t.mutex.Unlock() |
|||
t.estimatedDuration = duration |
|||
} |
|||
|
|||
// GetEstimatedDuration returns the estimated duration
|
|||
func (t *BaseTask) GetEstimatedDuration() time.Duration { |
|||
t.mutex.RLock() |
|||
defer t.mutex.RUnlock() |
|||
return t.estimatedDuration |
|||
} |
|||
|
|||
// ExecuteTask is a wrapper that handles common task execution logic
|
|||
func (t *BaseTask) ExecuteTask(ctx context.Context, params types.TaskParams, executor func(context.Context, types.TaskParams) error) error { |
|||
t.SetStartTime(time.Now()) |
|||
t.SetProgress(0) |
|||
|
|||
// Create a context that can be cancelled
|
|||
ctx, cancel := context.WithCancel(ctx) |
|||
defer cancel() |
|||
|
|||
// Monitor for cancellation
|
|||
go func() { |
|||
for !t.IsCancelled() { |
|||
select { |
|||
case <-ctx.Done(): |
|||
return |
|||
case <-time.After(time.Second): |
|||
// Check cancellation every second
|
|||
} |
|||
} |
|||
cancel() |
|||
}() |
|||
|
|||
// Execute the actual task
|
|||
err := executor(ctx, params) |
|||
|
|||
if err != nil { |
|||
return err |
|||
} |
|||
|
|||
if t.IsCancelled() { |
|||
return context.Canceled |
|||
} |
|||
|
|||
t.SetProgress(100) |
|||
return nil |
|||
} |
|||
|
|||
// TaskRegistry manages task factories
|
|||
type TaskRegistry struct { |
|||
factories map[types.TaskType]types.TaskFactory |
|||
mutex sync.RWMutex |
|||
} |
|||
|
|||
// NewTaskRegistry creates a new task registry
|
|||
func NewTaskRegistry() *TaskRegistry { |
|||
return &TaskRegistry{ |
|||
factories: make(map[types.TaskType]types.TaskFactory), |
|||
} |
|||
} |
|||
|
|||
// Register registers a task factory
|
|||
func (r *TaskRegistry) Register(taskType types.TaskType, factory types.TaskFactory) { |
|||
r.mutex.Lock() |
|||
defer r.mutex.Unlock() |
|||
r.factories[taskType] = factory |
|||
} |
|||
|
|||
// CreateTask creates a task instance
|
|||
func (r *TaskRegistry) CreateTask(taskType types.TaskType, params types.TaskParams) (types.TaskInterface, error) { |
|||
r.mutex.RLock() |
|||
factory, exists := r.factories[taskType] |
|||
r.mutex.RUnlock() |
|||
|
|||
if !exists { |
|||
return nil, &UnsupportedTaskTypeError{TaskType: taskType} |
|||
} |
|||
|
|||
return factory.Create(params) |
|||
} |
|||
|
|||
// GetSupportedTypes returns all supported task types
|
|||
func (r *TaskRegistry) GetSupportedTypes() []types.TaskType { |
|||
r.mutex.RLock() |
|||
defer r.mutex.RUnlock() |
|||
|
|||
types := make([]types.TaskType, 0, len(r.factories)) |
|||
for taskType := range r.factories { |
|||
types = append(types, taskType) |
|||
} |
|||
return types |
|||
} |
|||
|
|||
// GetFactory returns the factory for a task type
|
|||
func (r *TaskRegistry) GetFactory(taskType types.TaskType) (types.TaskFactory, bool) { |
|||
r.mutex.RLock() |
|||
defer r.mutex.RUnlock() |
|||
factory, exists := r.factories[taskType] |
|||
return factory, exists |
|||
} |
|||
|
|||
// UnsupportedTaskTypeError represents an error for unsupported task types
|
|||
type UnsupportedTaskTypeError struct { |
|||
TaskType types.TaskType |
|||
} |
|||
|
|||
func (e *UnsupportedTaskTypeError) Error() string { |
|||
return "unsupported task type: " + string(e.TaskType) |
|||
} |
|||
|
|||
// BaseTaskFactory provides common functionality for task factories
|
|||
type BaseTaskFactory struct { |
|||
taskType types.TaskType |
|||
capabilities []string |
|||
description string |
|||
} |
|||
|
|||
// NewBaseTaskFactory creates a new base task factory
|
|||
func NewBaseTaskFactory(taskType types.TaskType, capabilities []string, description string) *BaseTaskFactory { |
|||
return &BaseTaskFactory{ |
|||
taskType: taskType, |
|||
capabilities: capabilities, |
|||
description: description, |
|||
} |
|||
} |
|||
|
|||
// Capabilities returns the capabilities required for this task type
|
|||
func (f *BaseTaskFactory) Capabilities() []string { |
|||
return f.capabilities |
|||
} |
|||
|
|||
// Description returns the description of this task type
|
|||
func (f *BaseTaskFactory) Description() string { |
|||
return f.description |
|||
} |
|||
|
|||
// ValidateParams validates task parameters
|
|||
func ValidateParams(params types.TaskParams, requiredFields ...string) error { |
|||
for _, field := range requiredFields { |
|||
switch field { |
|||
case "volume_id": |
|||
if params.VolumeID == 0 { |
|||
return &ValidationError{Field: field, Message: "volume_id is required"} |
|||
} |
|||
case "server": |
|||
if params.Server == "" { |
|||
return &ValidationError{Field: field, Message: "server is required"} |
|||
} |
|||
case "collection": |
|||
if params.Collection == "" { |
|||
return &ValidationError{Field: field, Message: "collection is required"} |
|||
} |
|||
} |
|||
} |
|||
return nil |
|||
} |
|||
|
|||
// ValidationError represents a parameter validation error
|
|||
type ValidationError struct { |
|||
Field string |
|||
Message string |
|||
} |
|||
|
|||
func (e *ValidationError) Error() string { |
|||
return e.Field + ": " + e.Message |
|||
} |
|||
@ -0,0 +1,314 @@ |
|||
package vacuum |
|||
|
|||
import ( |
|||
"fmt" |
|||
"html/template" |
|||
"strconv" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// UIProvider provides the UI for vacuum task configuration
|
|||
type UIProvider struct { |
|||
detector *VacuumDetector |
|||
scheduler *VacuumScheduler |
|||
} |
|||
|
|||
// NewUIProvider creates a new vacuum UI provider
|
|||
func NewUIProvider(detector *VacuumDetector, scheduler *VacuumScheduler) *UIProvider { |
|||
return &UIProvider{ |
|||
detector: detector, |
|||
scheduler: scheduler, |
|||
} |
|||
} |
|||
|
|||
// GetTaskType returns the task type
|
|||
func (ui *UIProvider) GetTaskType() types.TaskType { |
|||
return types.TaskTypeVacuum |
|||
} |
|||
|
|||
// GetDisplayName returns the human-readable name
|
|||
func (ui *UIProvider) GetDisplayName() string { |
|||
return "Volume Vacuum" |
|||
} |
|||
|
|||
// GetDescription returns a description of what this task does
|
|||
func (ui *UIProvider) GetDescription() string { |
|||
return "Reclaims disk space by removing deleted files from volumes" |
|||
} |
|||
|
|||
// GetIcon returns the icon CSS class for this task type
|
|||
func (ui *UIProvider) GetIcon() string { |
|||
return "fas fa-broom text-primary" |
|||
} |
|||
|
|||
// VacuumConfig represents the vacuum configuration
|
|||
type VacuumConfig struct { |
|||
Enabled bool `json:"enabled"` |
|||
GarbageThreshold float64 `json:"garbage_threshold"` |
|||
ScanIntervalSeconds int `json:"scan_interval_seconds"` |
|||
MaxConcurrent int `json:"max_concurrent"` |
|||
MinVolumeAgeSeconds int `json:"min_volume_age_seconds"` |
|||
MinIntervalSeconds int `json:"min_interval_seconds"` |
|||
} |
|||
|
|||
// Helper functions for duration conversion
|
|||
func secondsToDuration(seconds int) time.Duration { |
|||
return time.Duration(seconds) * time.Second |
|||
} |
|||
|
|||
func durationToSeconds(d time.Duration) int { |
|||
return int(d.Seconds()) |
|||
} |
|||
|
|||
// formatDurationForUser formats seconds as a user-friendly duration string
|
|||
func formatDurationForUser(seconds int) string { |
|||
d := secondsToDuration(seconds) |
|||
if d < time.Minute { |
|||
return fmt.Sprintf("%ds", seconds) |
|||
} |
|||
if d < time.Hour { |
|||
return fmt.Sprintf("%.0fm", d.Minutes()) |
|||
} |
|||
if d < 24*time.Hour { |
|||
return fmt.Sprintf("%.1fh", d.Hours()) |
|||
} |
|||
return fmt.Sprintf("%.1fd", d.Hours()/24) |
|||
} |
|||
|
|||
// RenderConfigForm renders the configuration form HTML
|
|||
func (ui *UIProvider) RenderConfigForm(currentConfig interface{}) (template.HTML, error) { |
|||
config := ui.getCurrentVacuumConfig() |
|||
|
|||
// Build form using the FormBuilder helper
|
|||
form := types.NewFormBuilder() |
|||
|
|||
// Detection Settings
|
|||
form.AddCheckboxField( |
|||
"enabled", |
|||
"Enable Vacuum Tasks", |
|||
"Whether vacuum tasks should be automatically created", |
|||
config.Enabled, |
|||
) |
|||
|
|||
form.AddNumberField( |
|||
"garbage_threshold", |
|||
"Garbage Threshold (%)", |
|||
"Trigger vacuum when garbage ratio exceeds this percentage (0.0-1.0)", |
|||
config.GarbageThreshold, |
|||
true, |
|||
) |
|||
|
|||
form.AddDurationField( |
|||
"scan_interval", |
|||
"Scan Interval", |
|||
"How often to scan for volumes needing vacuum", |
|||
secondsToDuration(config.ScanIntervalSeconds), |
|||
true, |
|||
) |
|||
|
|||
form.AddDurationField( |
|||
"min_volume_age", |
|||
"Minimum Volume Age", |
|||
"Only vacuum volumes older than this duration", |
|||
secondsToDuration(config.MinVolumeAgeSeconds), |
|||
true, |
|||
) |
|||
|
|||
// Scheduling Settings
|
|||
form.AddNumberField( |
|||
"max_concurrent", |
|||
"Max Concurrent Tasks", |
|||
"Maximum number of vacuum tasks that can run simultaneously", |
|||
float64(config.MaxConcurrent), |
|||
true, |
|||
) |
|||
|
|||
form.AddDurationField( |
|||
"min_interval", |
|||
"Minimum Interval", |
|||
"Minimum time between vacuum operations on the same volume", |
|||
secondsToDuration(config.MinIntervalSeconds), |
|||
true, |
|||
) |
|||
|
|||
// Generate organized form sections using Bootstrap components
|
|||
html := ` |
|||
<div class="row"> |
|||
<div class="col-12"> |
|||
<div class="card mb-4"> |
|||
<div class="card-header"> |
|||
<h5 class="mb-0"> |
|||
<i class="fas fa-search me-2"></i> |
|||
Detection Settings |
|||
</h5> |
|||
</div> |
|||
<div class="card-body"> |
|||
` + string(form.Build()) + ` |
|||
</div> |
|||
</div> |
|||
</div> |
|||
</div> |
|||
|
|||
<script> |
|||
function resetForm() { |
|||
if (confirm('Reset all vacuum settings to defaults?')) { |
|||
// Reset to default values
|
|||
document.querySelector('input[name="enabled"]').checked = true; |
|||
document.querySelector('input[name="garbage_threshold"]').value = '0.3'; |
|||
document.querySelector('input[name="scan_interval"]').value = '30m'; |
|||
document.querySelector('input[name="min_volume_age"]').value = '1h'; |
|||
document.querySelector('input[name="max_concurrent"]').value = '2'; |
|||
document.querySelector('input[name="min_interval"]').value = '6h'; |
|||
} |
|||
} |
|||
</script> |
|||
` |
|||
|
|||
return template.HTML(html), nil |
|||
} |
|||
|
|||
// ParseConfigForm parses form data into configuration
|
|||
func (ui *UIProvider) ParseConfigForm(formData map[string][]string) (interface{}, error) { |
|||
config := &VacuumConfig{} |
|||
|
|||
// Parse enabled checkbox
|
|||
config.Enabled = len(formData["enabled"]) > 0 && formData["enabled"][0] == "on" |
|||
|
|||
// Parse garbage threshold
|
|||
if thresholdStr := formData["garbage_threshold"]; len(thresholdStr) > 0 { |
|||
if threshold, err := strconv.ParseFloat(thresholdStr[0], 64); err != nil { |
|||
return nil, fmt.Errorf("invalid garbage threshold: %v", err) |
|||
} else if threshold < 0 || threshold > 1 { |
|||
return nil, fmt.Errorf("garbage threshold must be between 0.0 and 1.0") |
|||
} else { |
|||
config.GarbageThreshold = threshold |
|||
} |
|||
} |
|||
|
|||
// Parse scan interval
|
|||
if intervalStr := formData["scan_interval"]; len(intervalStr) > 0 { |
|||
if interval, err := time.ParseDuration(intervalStr[0]); err != nil { |
|||
return nil, fmt.Errorf("invalid scan interval: %v", err) |
|||
} else { |
|||
config.ScanIntervalSeconds = durationToSeconds(interval) |
|||
} |
|||
} |
|||
|
|||
// Parse min volume age
|
|||
if ageStr := formData["min_volume_age"]; len(ageStr) > 0 { |
|||
if age, err := time.ParseDuration(ageStr[0]); err != nil { |
|||
return nil, fmt.Errorf("invalid min volume age: %v", err) |
|||
} else { |
|||
config.MinVolumeAgeSeconds = durationToSeconds(age) |
|||
} |
|||
} |
|||
|
|||
// Parse max concurrent
|
|||
if concurrentStr := formData["max_concurrent"]; len(concurrentStr) > 0 { |
|||
if concurrent, err := strconv.Atoi(concurrentStr[0]); err != nil { |
|||
return nil, fmt.Errorf("invalid max concurrent: %v", err) |
|||
} else if concurrent < 1 { |
|||
return nil, fmt.Errorf("max concurrent must be at least 1") |
|||
} else { |
|||
config.MaxConcurrent = concurrent |
|||
} |
|||
} |
|||
|
|||
// Parse min interval
|
|||
if intervalStr := formData["min_interval"]; len(intervalStr) > 0 { |
|||
if interval, err := time.ParseDuration(intervalStr[0]); err != nil { |
|||
return nil, fmt.Errorf("invalid min interval: %v", err) |
|||
} else { |
|||
config.MinIntervalSeconds = durationToSeconds(interval) |
|||
} |
|||
} |
|||
|
|||
return config, nil |
|||
} |
|||
|
|||
// GetCurrentConfig returns the current configuration
|
|||
func (ui *UIProvider) GetCurrentConfig() interface{} { |
|||
return ui.getCurrentVacuumConfig() |
|||
} |
|||
|
|||
// ApplyConfig applies the new configuration
|
|||
func (ui *UIProvider) ApplyConfig(config interface{}) error { |
|||
vacuumConfig, ok := config.(*VacuumConfig) |
|||
if !ok { |
|||
return fmt.Errorf("invalid config type, expected *VacuumConfig") |
|||
} |
|||
|
|||
// Apply to detector
|
|||
if ui.detector != nil { |
|||
ui.detector.SetEnabled(vacuumConfig.Enabled) |
|||
ui.detector.SetGarbageThreshold(vacuumConfig.GarbageThreshold) |
|||
ui.detector.SetScanInterval(secondsToDuration(vacuumConfig.ScanIntervalSeconds)) |
|||
ui.detector.SetMinVolumeAge(secondsToDuration(vacuumConfig.MinVolumeAgeSeconds)) |
|||
} |
|||
|
|||
// Apply to scheduler
|
|||
if ui.scheduler != nil { |
|||
ui.scheduler.SetEnabled(vacuumConfig.Enabled) |
|||
ui.scheduler.SetMaxConcurrent(vacuumConfig.MaxConcurrent) |
|||
ui.scheduler.SetMinInterval(secondsToDuration(vacuumConfig.MinIntervalSeconds)) |
|||
} |
|||
|
|||
glog.V(1).Infof("Applied vacuum configuration: enabled=%v, threshold=%.1f%%, scan_interval=%s, max_concurrent=%d", |
|||
vacuumConfig.Enabled, vacuumConfig.GarbageThreshold*100, formatDurationForUser(vacuumConfig.ScanIntervalSeconds), vacuumConfig.MaxConcurrent) |
|||
|
|||
return nil |
|||
} |
|||
|
|||
// getCurrentVacuumConfig gets the current configuration from detector and scheduler
|
|||
func (ui *UIProvider) getCurrentVacuumConfig() *VacuumConfig { |
|||
config := &VacuumConfig{ |
|||
// Default values (fallback if detectors/schedulers are nil)
|
|||
Enabled: true, |
|||
GarbageThreshold: 0.3, |
|||
ScanIntervalSeconds: 30 * 60, |
|||
MinVolumeAgeSeconds: 1 * 60 * 60, |
|||
MaxConcurrent: 2, |
|||
MinIntervalSeconds: 6 * 60 * 60, |
|||
} |
|||
|
|||
// Get current values from detector
|
|||
if ui.detector != nil { |
|||
config.Enabled = ui.detector.IsEnabled() |
|||
config.GarbageThreshold = ui.detector.GetGarbageThreshold() |
|||
config.ScanIntervalSeconds = durationToSeconds(ui.detector.ScanInterval()) |
|||
config.MinVolumeAgeSeconds = durationToSeconds(ui.detector.GetMinVolumeAge()) |
|||
} |
|||
|
|||
// Get current values from scheduler
|
|||
if ui.scheduler != nil { |
|||
config.MaxConcurrent = ui.scheduler.GetMaxConcurrent() |
|||
config.MinIntervalSeconds = durationToSeconds(ui.scheduler.GetMinInterval()) |
|||
} |
|||
|
|||
return config |
|||
} |
|||
|
|||
// RegisterUI registers the vacuum UI provider with the UI registry
|
|||
func RegisterUI(uiRegistry *types.UIRegistry, detector *VacuumDetector, scheduler *VacuumScheduler) { |
|||
uiProvider := NewUIProvider(detector, scheduler) |
|||
uiRegistry.RegisterUI(uiProvider) |
|||
|
|||
glog.V(1).Infof("✅ Registered vacuum task UI provider") |
|||
} |
|||
|
|||
// Example: How to get the UI provider for external use
|
|||
func GetUIProvider(uiRegistry *types.UIRegistry) *UIProvider { |
|||
provider := uiRegistry.GetProvider(types.TaskTypeVacuum) |
|||
if provider == nil { |
|||
return nil |
|||
} |
|||
|
|||
if vacuumProvider, ok := provider.(*UIProvider); ok { |
|||
return vacuumProvider |
|||
} |
|||
|
|||
return nil |
|||
} |
|||
@ -0,0 +1,330 @@ |
|||
package vacuum |
|||
|
|||
import ( |
|||
"fmt" |
|||
"strconv" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/admin/view/components" |
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// Helper function to format seconds as duration string
|
|||
func formatDurationFromSeconds(seconds int) string { |
|||
d := time.Duration(seconds) * time.Second |
|||
return d.String() |
|||
} |
|||
|
|||
// Helper functions to convert between seconds and value+unit format
|
|||
func secondsToValueAndUnit(seconds int) (float64, string) { |
|||
if seconds == 0 { |
|||
return 0, "minutes" |
|||
} |
|||
|
|||
// Try days first
|
|||
if seconds%(24*3600) == 0 && seconds >= 24*3600 { |
|||
return float64(seconds / (24 * 3600)), "days" |
|||
} |
|||
|
|||
// Try hours
|
|||
if seconds%3600 == 0 && seconds >= 3600 { |
|||
return float64(seconds / 3600), "hours" |
|||
} |
|||
|
|||
// Default to minutes
|
|||
return float64(seconds / 60), "minutes" |
|||
} |
|||
|
|||
func valueAndUnitToSeconds(value float64, unit string) int { |
|||
switch unit { |
|||
case "days": |
|||
return int(value * 24 * 3600) |
|||
case "hours": |
|||
return int(value * 3600) |
|||
case "minutes": |
|||
return int(value * 60) |
|||
default: |
|||
return int(value * 60) // Default to minutes
|
|||
} |
|||
} |
|||
|
|||
// UITemplProvider provides the templ-based UI for vacuum task configuration
|
|||
type UITemplProvider struct { |
|||
detector *VacuumDetector |
|||
scheduler *VacuumScheduler |
|||
} |
|||
|
|||
// NewUITemplProvider creates a new vacuum templ UI provider
|
|||
func NewUITemplProvider(detector *VacuumDetector, scheduler *VacuumScheduler) *UITemplProvider { |
|||
return &UITemplProvider{ |
|||
detector: detector, |
|||
scheduler: scheduler, |
|||
} |
|||
} |
|||
|
|||
// GetTaskType returns the task type
|
|||
func (ui *UITemplProvider) GetTaskType() types.TaskType { |
|||
return types.TaskTypeVacuum |
|||
} |
|||
|
|||
// GetDisplayName returns the human-readable name
|
|||
func (ui *UITemplProvider) GetDisplayName() string { |
|||
return "Volume Vacuum" |
|||
} |
|||
|
|||
// GetDescription returns a description of what this task does
|
|||
func (ui *UITemplProvider) GetDescription() string { |
|||
return "Reclaims disk space by removing deleted files from volumes" |
|||
} |
|||
|
|||
// GetIcon returns the icon CSS class for this task type
|
|||
func (ui *UITemplProvider) GetIcon() string { |
|||
return "fas fa-broom text-primary" |
|||
} |
|||
|
|||
// RenderConfigSections renders the configuration as templ section data
|
|||
func (ui *UITemplProvider) RenderConfigSections(currentConfig interface{}) ([]components.ConfigSectionData, error) { |
|||
config := ui.getCurrentVacuumConfig() |
|||
|
|||
// Detection settings section
|
|||
detectionSection := components.ConfigSectionData{ |
|||
Title: "Detection Settings", |
|||
Icon: "fas fa-search", |
|||
Description: "Configure when vacuum tasks should be triggered", |
|||
Fields: []interface{}{ |
|||
components.CheckboxFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "enabled", |
|||
Label: "Enable Vacuum Tasks", |
|||
Description: "Whether vacuum tasks should be automatically created", |
|||
}, |
|||
Checked: config.Enabled, |
|||
}, |
|||
components.NumberFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "garbage_threshold", |
|||
Label: "Garbage Threshold", |
|||
Description: "Trigger vacuum when garbage ratio exceeds this percentage (0.0-1.0)", |
|||
Required: true, |
|||
}, |
|||
Value: config.GarbageThreshold, |
|||
Step: "0.01", |
|||
Min: floatPtr(0.0), |
|||
Max: floatPtr(1.0), |
|||
}, |
|||
components.DurationInputFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "scan_interval", |
|||
Label: "Scan Interval", |
|||
Description: "How often to scan for volumes needing vacuum", |
|||
Required: true, |
|||
}, |
|||
Seconds: config.ScanIntervalSeconds, |
|||
}, |
|||
components.DurationInputFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "min_volume_age", |
|||
Label: "Minimum Volume Age", |
|||
Description: "Only vacuum volumes older than this duration", |
|||
Required: true, |
|||
}, |
|||
Seconds: config.MinVolumeAgeSeconds, |
|||
}, |
|||
}, |
|||
} |
|||
|
|||
// Scheduling settings section
|
|||
schedulingSection := components.ConfigSectionData{ |
|||
Title: "Scheduling Settings", |
|||
Icon: "fas fa-clock", |
|||
Description: "Configure task scheduling and concurrency", |
|||
Fields: []interface{}{ |
|||
components.NumberFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "max_concurrent", |
|||
Label: "Max Concurrent Tasks", |
|||
Description: "Maximum number of vacuum tasks that can run simultaneously", |
|||
Required: true, |
|||
}, |
|||
Value: float64(config.MaxConcurrent), |
|||
Step: "1", |
|||
Min: floatPtr(1), |
|||
}, |
|||
components.DurationInputFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "min_interval", |
|||
Label: "Minimum Interval", |
|||
Description: "Minimum time between vacuum operations on the same volume", |
|||
Required: true, |
|||
}, |
|||
Seconds: config.MinIntervalSeconds, |
|||
}, |
|||
}, |
|||
} |
|||
|
|||
// Performance impact info section
|
|||
performanceSection := components.ConfigSectionData{ |
|||
Title: "Performance Impact", |
|||
Icon: "fas fa-exclamation-triangle", |
|||
Description: "Important information about vacuum operations", |
|||
Fields: []interface{}{ |
|||
components.TextFieldData{ |
|||
FormFieldData: components.FormFieldData{ |
|||
Name: "info_impact", |
|||
Label: "Impact", |
|||
Description: "Volume vacuum operations are I/O intensive and should be scheduled appropriately", |
|||
}, |
|||
Value: "Configure thresholds and intervals based on your storage usage patterns", |
|||
}, |
|||
}, |
|||
} |
|||
|
|||
return []components.ConfigSectionData{detectionSection, schedulingSection, performanceSection}, nil |
|||
} |
|||
|
|||
// ParseConfigForm parses form data into configuration
|
|||
func (ui *UITemplProvider) ParseConfigForm(formData map[string][]string) (interface{}, error) { |
|||
config := &VacuumConfig{} |
|||
|
|||
// Parse enabled checkbox
|
|||
config.Enabled = len(formData["enabled"]) > 0 && formData["enabled"][0] == "on" |
|||
|
|||
// Parse garbage threshold
|
|||
if thresholdStr := formData["garbage_threshold"]; len(thresholdStr) > 0 { |
|||
if threshold, err := strconv.ParseFloat(thresholdStr[0], 64); err != nil { |
|||
return nil, fmt.Errorf("invalid garbage threshold: %v", err) |
|||
} else if threshold < 0 || threshold > 1 { |
|||
return nil, fmt.Errorf("garbage threshold must be between 0.0 and 1.0") |
|||
} else { |
|||
config.GarbageThreshold = threshold |
|||
} |
|||
} |
|||
|
|||
// Parse scan interval
|
|||
if valueStr := formData["scan_interval"]; len(valueStr) > 0 { |
|||
if value, err := strconv.ParseFloat(valueStr[0], 64); err != nil { |
|||
return nil, fmt.Errorf("invalid scan interval value: %v", err) |
|||
} else { |
|||
unit := "minutes" // default
|
|||
if unitStr := formData["scan_interval_unit"]; len(unitStr) > 0 { |
|||
unit = unitStr[0] |
|||
} |
|||
config.ScanIntervalSeconds = valueAndUnitToSeconds(value, unit) |
|||
} |
|||
} |
|||
|
|||
// Parse min volume age
|
|||
if valueStr := formData["min_volume_age"]; len(valueStr) > 0 { |
|||
if value, err := strconv.ParseFloat(valueStr[0], 64); err != nil { |
|||
return nil, fmt.Errorf("invalid min volume age value: %v", err) |
|||
} else { |
|||
unit := "minutes" // default
|
|||
if unitStr := formData["min_volume_age_unit"]; len(unitStr) > 0 { |
|||
unit = unitStr[0] |
|||
} |
|||
config.MinVolumeAgeSeconds = valueAndUnitToSeconds(value, unit) |
|||
} |
|||
} |
|||
|
|||
// Parse max concurrent
|
|||
if concurrentStr := formData["max_concurrent"]; len(concurrentStr) > 0 { |
|||
if concurrent, err := strconv.Atoi(concurrentStr[0]); err != nil { |
|||
return nil, fmt.Errorf("invalid max concurrent: %v", err) |
|||
} else if concurrent < 1 { |
|||
return nil, fmt.Errorf("max concurrent must be at least 1") |
|||
} else { |
|||
config.MaxConcurrent = concurrent |
|||
} |
|||
} |
|||
|
|||
// Parse min interval
|
|||
if valueStr := formData["min_interval"]; len(valueStr) > 0 { |
|||
if value, err := strconv.ParseFloat(valueStr[0], 64); err != nil { |
|||
return nil, fmt.Errorf("invalid min interval value: %v", err) |
|||
} else { |
|||
unit := "minutes" // default
|
|||
if unitStr := formData["min_interval_unit"]; len(unitStr) > 0 { |
|||
unit = unitStr[0] |
|||
} |
|||
config.MinIntervalSeconds = valueAndUnitToSeconds(value, unit) |
|||
} |
|||
} |
|||
|
|||
return config, nil |
|||
} |
|||
|
|||
// GetCurrentConfig returns the current configuration
|
|||
func (ui *UITemplProvider) GetCurrentConfig() interface{} { |
|||
return ui.getCurrentVacuumConfig() |
|||
} |
|||
|
|||
// ApplyConfig applies the new configuration
|
|||
func (ui *UITemplProvider) ApplyConfig(config interface{}) error { |
|||
vacuumConfig, ok := config.(*VacuumConfig) |
|||
if !ok { |
|||
return fmt.Errorf("invalid config type, expected *VacuumConfig") |
|||
} |
|||
|
|||
// Apply to detector
|
|||
if ui.detector != nil { |
|||
ui.detector.SetEnabled(vacuumConfig.Enabled) |
|||
ui.detector.SetGarbageThreshold(vacuumConfig.GarbageThreshold) |
|||
ui.detector.SetScanInterval(time.Duration(vacuumConfig.ScanIntervalSeconds) * time.Second) |
|||
ui.detector.SetMinVolumeAge(time.Duration(vacuumConfig.MinVolumeAgeSeconds) * time.Second) |
|||
} |
|||
|
|||
// Apply to scheduler
|
|||
if ui.scheduler != nil { |
|||
ui.scheduler.SetEnabled(vacuumConfig.Enabled) |
|||
ui.scheduler.SetMaxConcurrent(vacuumConfig.MaxConcurrent) |
|||
ui.scheduler.SetMinInterval(time.Duration(vacuumConfig.MinIntervalSeconds) * time.Second) |
|||
} |
|||
|
|||
glog.V(1).Infof("Applied vacuum configuration: enabled=%v, threshold=%.1f%%, scan_interval=%s, max_concurrent=%d", |
|||
vacuumConfig.Enabled, vacuumConfig.GarbageThreshold*100, formatDurationFromSeconds(vacuumConfig.ScanIntervalSeconds), vacuumConfig.MaxConcurrent) |
|||
|
|||
return nil |
|||
} |
|||
|
|||
// getCurrentVacuumConfig gets the current configuration from detector and scheduler
|
|||
func (ui *UITemplProvider) getCurrentVacuumConfig() *VacuumConfig { |
|||
config := &VacuumConfig{ |
|||
// Default values (fallback if detectors/schedulers are nil)
|
|||
Enabled: true, |
|||
GarbageThreshold: 0.3, |
|||
ScanIntervalSeconds: int((30 * time.Minute).Seconds()), |
|||
MinVolumeAgeSeconds: int((1 * time.Hour).Seconds()), |
|||
MaxConcurrent: 2, |
|||
MinIntervalSeconds: int((6 * time.Hour).Seconds()), |
|||
} |
|||
|
|||
// Get current values from detector
|
|||
if ui.detector != nil { |
|||
config.Enabled = ui.detector.IsEnabled() |
|||
config.GarbageThreshold = ui.detector.GetGarbageThreshold() |
|||
config.ScanIntervalSeconds = int(ui.detector.ScanInterval().Seconds()) |
|||
config.MinVolumeAgeSeconds = int(ui.detector.GetMinVolumeAge().Seconds()) |
|||
} |
|||
|
|||
// Get current values from scheduler
|
|||
if ui.scheduler != nil { |
|||
config.MaxConcurrent = ui.scheduler.GetMaxConcurrent() |
|||
config.MinIntervalSeconds = int(ui.scheduler.GetMinInterval().Seconds()) |
|||
} |
|||
|
|||
return config |
|||
} |
|||
|
|||
// floatPtr is a helper function to create float64 pointers
|
|||
func floatPtr(f float64) *float64 { |
|||
return &f |
|||
} |
|||
|
|||
// RegisterUITempl registers the vacuum templ UI provider with the UI registry
|
|||
func RegisterUITempl(uiRegistry *types.UITemplRegistry, detector *VacuumDetector, scheduler *VacuumScheduler) { |
|||
uiProvider := NewUITemplProvider(detector, scheduler) |
|||
uiRegistry.RegisterUI(uiProvider) |
|||
|
|||
glog.V(1).Infof("✅ Registered vacuum task templ UI provider") |
|||
} |
|||
@ -0,0 +1,79 @@ |
|||
package vacuum |
|||
|
|||
import ( |
|||
"fmt" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/tasks" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// Task implements vacuum operation to reclaim disk space
|
|||
type Task struct { |
|||
*tasks.BaseTask |
|||
server string |
|||
volumeID uint32 |
|||
} |
|||
|
|||
// NewTask creates a new vacuum task instance
|
|||
func NewTask(server string, volumeID uint32) *Task { |
|||
task := &Task{ |
|||
BaseTask: tasks.NewBaseTask(types.TaskTypeVacuum), |
|||
server: server, |
|||
volumeID: volumeID, |
|||
} |
|||
return task |
|||
} |
|||
|
|||
// Execute executes the vacuum task
|
|||
func (t *Task) Execute(params types.TaskParams) error { |
|||
glog.Infof("Starting vacuum task for volume %d on server %s", t.volumeID, t.server) |
|||
|
|||
// Simulate vacuum operation with progress updates
|
|||
steps := []struct { |
|||
name string |
|||
duration time.Duration |
|||
progress float64 |
|||
}{ |
|||
{"Scanning volume", 1 * time.Second, 20}, |
|||
{"Identifying deleted files", 2 * time.Second, 50}, |
|||
{"Compacting data", 3 * time.Second, 80}, |
|||
{"Finalizing vacuum", 1 * time.Second, 100}, |
|||
} |
|||
|
|||
for _, step := range steps { |
|||
if t.IsCancelled() { |
|||
return fmt.Errorf("vacuum task cancelled") |
|||
} |
|||
|
|||
glog.V(1).Infof("Vacuum task step: %s", step.name) |
|||
t.SetProgress(step.progress) |
|||
|
|||
// Simulate work
|
|||
time.Sleep(step.duration) |
|||
} |
|||
|
|||
glog.Infof("Vacuum task completed for volume %d on server %s", t.volumeID, t.server) |
|||
return nil |
|||
} |
|||
|
|||
// Validate validates the task parameters
|
|||
func (t *Task) Validate(params types.TaskParams) error { |
|||
if params.VolumeID == 0 { |
|||
return fmt.Errorf("volume_id is required") |
|||
} |
|||
if params.Server == "" { |
|||
return fmt.Errorf("server is required") |
|||
} |
|||
return nil |
|||
} |
|||
|
|||
// EstimateTime estimates the time needed for the task
|
|||
func (t *Task) EstimateTime(params types.TaskParams) time.Duration { |
|||
// Base time for vacuum operation
|
|||
baseTime := 25 * time.Second |
|||
|
|||
// Could adjust based on volume size or usage patterns
|
|||
return baseTime |
|||
} |
|||
@ -0,0 +1,132 @@ |
|||
package vacuum |
|||
|
|||
import ( |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// VacuumDetector implements vacuum task detection using code instead of schemas
|
|||
type VacuumDetector struct { |
|||
enabled bool |
|||
garbageThreshold float64 |
|||
minVolumeAge time.Duration |
|||
scanInterval time.Duration |
|||
} |
|||
|
|||
// Compile-time interface assertions
|
|||
var ( |
|||
_ types.TaskDetector = (*VacuumDetector)(nil) |
|||
_ types.PolicyConfigurableDetector = (*VacuumDetector)(nil) |
|||
) |
|||
|
|||
// NewVacuumDetector creates a new simple vacuum detector
|
|||
func NewVacuumDetector() *VacuumDetector { |
|||
return &VacuumDetector{ |
|||
enabled: true, |
|||
garbageThreshold: 0.3, |
|||
minVolumeAge: 24 * time.Hour, |
|||
scanInterval: 30 * time.Minute, |
|||
} |
|||
} |
|||
|
|||
// GetTaskType returns the task type
|
|||
func (d *VacuumDetector) GetTaskType() types.TaskType { |
|||
return types.TaskTypeVacuum |
|||
} |
|||
|
|||
// ScanForTasks scans for volumes that need vacuum operations
|
|||
func (d *VacuumDetector) ScanForTasks(volumeMetrics []*types.VolumeHealthMetrics, clusterInfo *types.ClusterInfo) ([]*types.TaskDetectionResult, error) { |
|||
if !d.enabled { |
|||
return nil, nil |
|||
} |
|||
|
|||
var results []*types.TaskDetectionResult |
|||
|
|||
for _, metric := range volumeMetrics { |
|||
// Check if volume needs vacuum
|
|||
if metric.GarbageRatio >= d.garbageThreshold && metric.Age >= d.minVolumeAge { |
|||
// Higher priority for volumes with more garbage
|
|||
priority := types.TaskPriorityNormal |
|||
if metric.GarbageRatio > 0.6 { |
|||
priority = types.TaskPriorityHigh |
|||
} |
|||
|
|||
result := &types.TaskDetectionResult{ |
|||
TaskType: types.TaskTypeVacuum, |
|||
VolumeID: metric.VolumeID, |
|||
Server: metric.Server, |
|||
Collection: metric.Collection, |
|||
Priority: priority, |
|||
Reason: "Volume has excessive garbage requiring vacuum", |
|||
Parameters: map[string]interface{}{ |
|||
"garbage_ratio": metric.GarbageRatio, |
|||
"volume_age": metric.Age.String(), |
|||
}, |
|||
ScheduleAt: time.Now(), |
|||
} |
|||
results = append(results, result) |
|||
} |
|||
} |
|||
|
|||
glog.V(2).Infof("Vacuum detector found %d volumes needing vacuum", len(results)) |
|||
return results, nil |
|||
} |
|||
|
|||
// ScanInterval returns how often this detector should scan
|
|||
func (d *VacuumDetector) ScanInterval() time.Duration { |
|||
return d.scanInterval |
|||
} |
|||
|
|||
// IsEnabled returns whether this detector is enabled
|
|||
func (d *VacuumDetector) IsEnabled() bool { |
|||
return d.enabled |
|||
} |
|||
|
|||
// Configuration setters
|
|||
|
|||
func (d *VacuumDetector) SetEnabled(enabled bool) { |
|||
d.enabled = enabled |
|||
} |
|||
|
|||
func (d *VacuumDetector) SetGarbageThreshold(threshold float64) { |
|||
d.garbageThreshold = threshold |
|||
} |
|||
|
|||
func (d *VacuumDetector) SetScanInterval(interval time.Duration) { |
|||
d.scanInterval = interval |
|||
} |
|||
|
|||
func (d *VacuumDetector) SetMinVolumeAge(age time.Duration) { |
|||
d.minVolumeAge = age |
|||
} |
|||
|
|||
// GetGarbageThreshold returns the current garbage threshold
|
|||
func (d *VacuumDetector) GetGarbageThreshold() float64 { |
|||
return d.garbageThreshold |
|||
} |
|||
|
|||
// GetMinVolumeAge returns the minimum volume age
|
|||
func (d *VacuumDetector) GetMinVolumeAge() time.Duration { |
|||
return d.minVolumeAge |
|||
} |
|||
|
|||
// GetScanInterval returns the scan interval
|
|||
func (d *VacuumDetector) GetScanInterval() time.Duration { |
|||
return d.scanInterval |
|||
} |
|||
|
|||
// ConfigureFromPolicy configures the detector based on the maintenance policy
|
|||
func (d *VacuumDetector) ConfigureFromPolicy(policy interface{}) { |
|||
// Type assert to the maintenance policy type we expect
|
|||
if maintenancePolicy, ok := policy.(interface { |
|||
GetVacuumEnabled() bool |
|||
GetVacuumGarbageRatio() float64 |
|||
}); ok { |
|||
d.SetEnabled(maintenancePolicy.GetVacuumEnabled()) |
|||
d.SetGarbageThreshold(maintenancePolicy.GetVacuumGarbageRatio()) |
|||
} else { |
|||
glog.V(1).Infof("Could not configure vacuum detector from policy: unsupported policy type") |
|||
} |
|||
} |
|||
@ -0,0 +1,81 @@ |
|||
package vacuum |
|||
|
|||
import ( |
|||
"fmt" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/worker/tasks" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// Factory creates vacuum task instances
|
|||
type Factory struct { |
|||
*tasks.BaseTaskFactory |
|||
} |
|||
|
|||
// NewFactory creates a new vacuum task factory
|
|||
func NewFactory() *Factory { |
|||
return &Factory{ |
|||
BaseTaskFactory: tasks.NewBaseTaskFactory( |
|||
types.TaskTypeVacuum, |
|||
[]string{"vacuum", "storage"}, |
|||
"Vacuum operation to reclaim disk space by removing deleted files", |
|||
), |
|||
} |
|||
} |
|||
|
|||
// Create creates a new vacuum task instance
|
|||
func (f *Factory) Create(params types.TaskParams) (types.TaskInterface, error) { |
|||
// Validate parameters
|
|||
if params.VolumeID == 0 { |
|||
return nil, fmt.Errorf("volume_id is required") |
|||
} |
|||
if params.Server == "" { |
|||
return nil, fmt.Errorf("server is required") |
|||
} |
|||
|
|||
task := NewTask(params.Server, params.VolumeID) |
|||
task.SetEstimatedDuration(task.EstimateTime(params)) |
|||
|
|||
return task, nil |
|||
} |
|||
|
|||
// Shared detector and scheduler instances
|
|||
var ( |
|||
sharedDetector *VacuumDetector |
|||
sharedScheduler *VacuumScheduler |
|||
) |
|||
|
|||
// getSharedInstances returns the shared detector and scheduler instances
|
|||
func getSharedInstances() (*VacuumDetector, *VacuumScheduler) { |
|||
if sharedDetector == nil { |
|||
sharedDetector = NewVacuumDetector() |
|||
} |
|||
if sharedScheduler == nil { |
|||
sharedScheduler = NewVacuumScheduler() |
|||
} |
|||
return sharedDetector, sharedScheduler |
|||
} |
|||
|
|||
// GetSharedInstances returns the shared detector and scheduler instances (public access)
|
|||
func GetSharedInstances() (*VacuumDetector, *VacuumScheduler) { |
|||
return getSharedInstances() |
|||
} |
|||
|
|||
// Auto-register this task when the package is imported
|
|||
func init() { |
|||
factory := NewFactory() |
|||
tasks.AutoRegister(types.TaskTypeVacuum, factory) |
|||
|
|||
// Get shared instances for all registrations
|
|||
detector, scheduler := getSharedInstances() |
|||
|
|||
// Register with types registry
|
|||
tasks.AutoRegisterTypes(func(registry *types.TaskRegistry) { |
|||
registry.RegisterTask(detector, scheduler) |
|||
}) |
|||
|
|||
// Register with UI registry using the same instances
|
|||
tasks.AutoRegisterUI(func(uiRegistry *types.UIRegistry) { |
|||
RegisterUI(uiRegistry, detector, scheduler) |
|||
}) |
|||
} |
|||
@ -0,0 +1,111 @@ |
|||
package vacuum |
|||
|
|||
import ( |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
) |
|||
|
|||
// VacuumScheduler implements vacuum task scheduling using code instead of schemas
|
|||
type VacuumScheduler struct { |
|||
enabled bool |
|||
maxConcurrent int |
|||
minInterval time.Duration |
|||
} |
|||
|
|||
// Compile-time interface assertions
|
|||
var ( |
|||
_ types.TaskScheduler = (*VacuumScheduler)(nil) |
|||
) |
|||
|
|||
// NewVacuumScheduler creates a new simple vacuum scheduler
|
|||
func NewVacuumScheduler() *VacuumScheduler { |
|||
return &VacuumScheduler{ |
|||
enabled: true, |
|||
maxConcurrent: 2, |
|||
minInterval: 6 * time.Hour, |
|||
} |
|||
} |
|||
|
|||
// GetTaskType returns the task type
|
|||
func (s *VacuumScheduler) GetTaskType() types.TaskType { |
|||
return types.TaskTypeVacuum |
|||
} |
|||
|
|||
// CanScheduleNow determines if a vacuum task can be scheduled right now
|
|||
func (s *VacuumScheduler) CanScheduleNow(task *types.Task, runningTasks []*types.Task, availableWorkers []*types.Worker) bool { |
|||
// Check if scheduler is enabled
|
|||
if !s.enabled { |
|||
return false |
|||
} |
|||
|
|||
// Check concurrent limit
|
|||
runningVacuumCount := 0 |
|||
for _, runningTask := range runningTasks { |
|||
if runningTask.Type == types.TaskTypeVacuum { |
|||
runningVacuumCount++ |
|||
} |
|||
} |
|||
|
|||
if runningVacuumCount >= s.maxConcurrent { |
|||
return false |
|||
} |
|||
|
|||
// Check if there's an available worker with vacuum capability
|
|||
for _, worker := range availableWorkers { |
|||
if worker.CurrentLoad < worker.MaxConcurrent { |
|||
for _, capability := range worker.Capabilities { |
|||
if capability == types.TaskTypeVacuum { |
|||
return true |
|||
} |
|||
} |
|||
} |
|||
} |
|||
|
|||
return false |
|||
} |
|||
|
|||
// GetPriority returns the priority for this task
|
|||
func (s *VacuumScheduler) GetPriority(task *types.Task) types.TaskPriority { |
|||
// Could adjust priority based on task parameters
|
|||
if params, ok := task.Parameters["garbage_ratio"].(float64); ok { |
|||
if params > 0.8 { |
|||
return types.TaskPriorityHigh |
|||
} |
|||
} |
|||
return task.Priority |
|||
} |
|||
|
|||
// GetMaxConcurrent returns max concurrent tasks of this type
|
|||
func (s *VacuumScheduler) GetMaxConcurrent() int { |
|||
return s.maxConcurrent |
|||
} |
|||
|
|||
// GetDefaultRepeatInterval returns the default interval to wait before repeating vacuum tasks
|
|||
func (s *VacuumScheduler) GetDefaultRepeatInterval() time.Duration { |
|||
return s.minInterval |
|||
} |
|||
|
|||
// IsEnabled returns whether this scheduler is enabled
|
|||
func (s *VacuumScheduler) IsEnabled() bool { |
|||
return s.enabled |
|||
} |
|||
|
|||
// Configuration setters
|
|||
|
|||
func (s *VacuumScheduler) SetEnabled(enabled bool) { |
|||
s.enabled = enabled |
|||
} |
|||
|
|||
func (s *VacuumScheduler) SetMaxConcurrent(max int) { |
|||
s.maxConcurrent = max |
|||
} |
|||
|
|||
func (s *VacuumScheduler) SetMinInterval(interval time.Duration) { |
|||
s.minInterval = interval |
|||
} |
|||
|
|||
// GetMinInterval returns the minimum interval
|
|||
func (s *VacuumScheduler) GetMinInterval() time.Duration { |
|||
return s.minInterval |
|||
} |
|||
@ -0,0 +1,268 @@ |
|||
package types |
|||
|
|||
import ( |
|||
"sync" |
|||
"time" |
|||
) |
|||
|
|||
// WorkerConfig represents the configuration for a worker
|
|||
type WorkerConfig struct { |
|||
AdminServer string `json:"admin_server"` |
|||
Capabilities []TaskType `json:"capabilities"` |
|||
MaxConcurrent int `json:"max_concurrent"` |
|||
HeartbeatInterval time.Duration `json:"heartbeat_interval"` |
|||
TaskRequestInterval time.Duration `json:"task_request_interval"` |
|||
CustomParameters map[string]interface{} `json:"custom_parameters,omitempty"` |
|||
} |
|||
|
|||
// MaintenanceConfig represents the configuration for the maintenance system
|
|||
type MaintenanceConfig struct { |
|||
Enabled bool `json:"enabled"` |
|||
ScanInterval time.Duration `json:"scan_interval"` |
|||
CleanInterval time.Duration `json:"clean_interval"` |
|||
TaskRetention time.Duration `json:"task_retention"` |
|||
WorkerTimeout time.Duration `json:"worker_timeout"` |
|||
Policy *MaintenancePolicy `json:"policy"` |
|||
} |
|||
|
|||
// MaintenancePolicy represents policies for maintenance operations
|
|||
// This is now dynamic - task configurations are stored by task type
|
|||
type MaintenancePolicy struct { |
|||
// Task-specific configurations indexed by task type
|
|||
TaskConfigs map[TaskType]interface{} `json:"task_configs"` |
|||
|
|||
// Global maintenance settings
|
|||
GlobalSettings *GlobalMaintenanceSettings `json:"global_settings"` |
|||
} |
|||
|
|||
// GlobalMaintenanceSettings contains settings that apply to all tasks
|
|||
type GlobalMaintenanceSettings struct { |
|||
DefaultMaxConcurrent int `json:"default_max_concurrent"` |
|||
MaintenanceEnabled bool `json:"maintenance_enabled"` |
|||
|
|||
// Global timing settings
|
|||
DefaultScanInterval time.Duration `json:"default_scan_interval"` |
|||
DefaultTaskTimeout time.Duration `json:"default_task_timeout"` |
|||
DefaultRetryCount int `json:"default_retry_count"` |
|||
DefaultRetryInterval time.Duration `json:"default_retry_interval"` |
|||
|
|||
// Global thresholds
|
|||
DefaultPriorityBoostAge time.Duration `json:"default_priority_boost_age"` |
|||
GlobalConcurrentLimit int `json:"global_concurrent_limit"` |
|||
} |
|||
|
|||
// MaintenanceStats represents statistics for the maintenance system
|
|||
type MaintenanceStats struct { |
|||
TotalTasks int `json:"total_tasks"` |
|||
CompletedToday int `json:"completed_today"` |
|||
FailedToday int `json:"failed_today"` |
|||
ActiveWorkers int `json:"active_workers"` |
|||
AverageTaskTime time.Duration `json:"average_task_time"` |
|||
TasksByStatus map[TaskStatus]int `json:"tasks_by_status"` |
|||
TasksByType map[TaskType]int `json:"tasks_by_type"` |
|||
LastScanTime time.Time `json:"last_scan_time"` |
|||
NextScanTime time.Time `json:"next_scan_time"` |
|||
} |
|||
|
|||
// QueueStats represents statistics for the task queue
|
|||
type QueueStats struct { |
|||
PendingTasks int `json:"pending_tasks"` |
|||
AssignedTasks int `json:"assigned_tasks"` |
|||
InProgressTasks int `json:"in_progress_tasks"` |
|||
CompletedTasks int `json:"completed_tasks"` |
|||
FailedTasks int `json:"failed_tasks"` |
|||
CancelledTasks int `json:"cancelled_tasks"` |
|||
ActiveWorkers int `json:"active_workers"` |
|||
} |
|||
|
|||
// MaintenanceConfigData represents the complete maintenance configuration data
|
|||
type MaintenanceConfigData struct { |
|||
Config *MaintenanceConfig `json:"config"` |
|||
IsEnabled bool `json:"is_enabled"` |
|||
LastScanTime time.Time `json:"last_scan_time"` |
|||
NextScanTime time.Time `json:"next_scan_time"` |
|||
SystemStats *MaintenanceStats `json:"system_stats"` |
|||
} |
|||
|
|||
// MaintenanceQueueData represents data for the maintenance queue UI
|
|||
type MaintenanceQueueData struct { |
|||
Tasks []*Task `json:"tasks"` |
|||
Workers []*Worker `json:"workers"` |
|||
Stats *QueueStats `json:"stats"` |
|||
LastUpdated time.Time `json:"last_updated"` |
|||
} |
|||
|
|||
// MaintenanceWorkersData represents data for the maintenance workers UI
|
|||
type MaintenanceWorkersData struct { |
|||
Workers []*WorkerDetailsData `json:"workers"` |
|||
ActiveWorkers int `json:"active_workers"` |
|||
BusyWorkers int `json:"busy_workers"` |
|||
TotalLoad int `json:"total_load"` |
|||
LastUpdated time.Time `json:"last_updated"` |
|||
} |
|||
|
|||
// defaultCapabilities holds the default capabilities for workers
|
|||
var defaultCapabilities []TaskType |
|||
var defaultCapabilitiesMutex sync.RWMutex |
|||
|
|||
// SetDefaultCapabilities sets the default capabilities for workers
|
|||
// This should be called after task registration is complete
|
|||
func SetDefaultCapabilities(capabilities []TaskType) { |
|||
defaultCapabilitiesMutex.Lock() |
|||
defer defaultCapabilitiesMutex.Unlock() |
|||
defaultCapabilities = make([]TaskType, len(capabilities)) |
|||
copy(defaultCapabilities, capabilities) |
|||
} |
|||
|
|||
// GetDefaultCapabilities returns the default capabilities for workers
|
|||
func GetDefaultCapabilities() []TaskType { |
|||
defaultCapabilitiesMutex.RLock() |
|||
defer defaultCapabilitiesMutex.RUnlock() |
|||
|
|||
// Return a copy to prevent modification
|
|||
result := make([]TaskType, len(defaultCapabilities)) |
|||
copy(result, defaultCapabilities) |
|||
return result |
|||
} |
|||
|
|||
// DefaultMaintenanceConfig returns default maintenance configuration
|
|||
func DefaultMaintenanceConfig() *MaintenanceConfig { |
|||
return &MaintenanceConfig{ |
|||
Enabled: true, |
|||
ScanInterval: 30 * time.Minute, |
|||
CleanInterval: 6 * time.Hour, |
|||
TaskRetention: 7 * 24 * time.Hour, // 7 days
|
|||
WorkerTimeout: 5 * time.Minute, |
|||
Policy: NewMaintenancePolicy(), |
|||
} |
|||
} |
|||
|
|||
// DefaultWorkerConfig returns default worker configuration
|
|||
func DefaultWorkerConfig() *WorkerConfig { |
|||
// Get dynamic capabilities from registered task types
|
|||
capabilities := GetDefaultCapabilities() |
|||
|
|||
return &WorkerConfig{ |
|||
AdminServer: "localhost:9333", |
|||
MaxConcurrent: 2, |
|||
HeartbeatInterval: 30 * time.Second, |
|||
TaskRequestInterval: 5 * time.Second, |
|||
Capabilities: capabilities, |
|||
} |
|||
} |
|||
|
|||
// NewMaintenancePolicy creates a new dynamic maintenance policy
|
|||
func NewMaintenancePolicy() *MaintenancePolicy { |
|||
return &MaintenancePolicy{ |
|||
TaskConfigs: make(map[TaskType]interface{}), |
|||
GlobalSettings: &GlobalMaintenanceSettings{ |
|||
DefaultMaxConcurrent: 2, |
|||
MaintenanceEnabled: true, |
|||
DefaultScanInterval: 30 * time.Minute, |
|||
DefaultTaskTimeout: 5 * time.Minute, |
|||
DefaultRetryCount: 3, |
|||
DefaultRetryInterval: 5 * time.Minute, |
|||
DefaultPriorityBoostAge: 24 * time.Hour, |
|||
GlobalConcurrentLimit: 5, |
|||
}, |
|||
} |
|||
} |
|||
|
|||
// SetTaskConfig sets the configuration for a specific task type
|
|||
func (p *MaintenancePolicy) SetTaskConfig(taskType TaskType, config interface{}) { |
|||
if p.TaskConfigs == nil { |
|||
p.TaskConfigs = make(map[TaskType]interface{}) |
|||
} |
|||
p.TaskConfigs[taskType] = config |
|||
} |
|||
|
|||
// GetTaskConfig returns the configuration for a specific task type
|
|||
func (p *MaintenancePolicy) GetTaskConfig(taskType TaskType) interface{} { |
|||
if p.TaskConfigs == nil { |
|||
return nil |
|||
} |
|||
return p.TaskConfigs[taskType] |
|||
} |
|||
|
|||
// IsTaskEnabled returns whether a task type is enabled (generic helper)
|
|||
func (p *MaintenancePolicy) IsTaskEnabled(taskType TaskType) bool { |
|||
if !p.GlobalSettings.MaintenanceEnabled { |
|||
return false |
|||
} |
|||
|
|||
config := p.GetTaskConfig(taskType) |
|||
if config == nil { |
|||
return false |
|||
} |
|||
|
|||
// Try to get enabled field from config using type assertion
|
|||
if configMap, ok := config.(map[string]interface{}); ok { |
|||
if enabled, exists := configMap["enabled"]; exists { |
|||
if enabledBool, ok := enabled.(bool); ok { |
|||
return enabledBool |
|||
} |
|||
} |
|||
} |
|||
|
|||
// If we can't determine from config, default to global setting
|
|||
return p.GlobalSettings.MaintenanceEnabled |
|||
} |
|||
|
|||
// GetMaxConcurrent returns the max concurrent setting for a task type
|
|||
func (p *MaintenancePolicy) GetMaxConcurrent(taskType TaskType) int { |
|||
config := p.GetTaskConfig(taskType) |
|||
if config == nil { |
|||
return p.GlobalSettings.DefaultMaxConcurrent |
|||
} |
|||
|
|||
// Try to get max_concurrent field from config
|
|||
if configMap, ok := config.(map[string]interface{}); ok { |
|||
if maxConcurrent, exists := configMap["max_concurrent"]; exists { |
|||
if maxConcurrentInt, ok := maxConcurrent.(int); ok { |
|||
return maxConcurrentInt |
|||
} |
|||
if maxConcurrentFloat, ok := maxConcurrent.(float64); ok { |
|||
return int(maxConcurrentFloat) |
|||
} |
|||
} |
|||
} |
|||
|
|||
return p.GlobalSettings.DefaultMaxConcurrent |
|||
} |
|||
|
|||
// GetScanInterval returns the scan interval for a task type
|
|||
func (p *MaintenancePolicy) GetScanInterval(taskType TaskType) time.Duration { |
|||
config := p.GetTaskConfig(taskType) |
|||
if config == nil { |
|||
return p.GlobalSettings.DefaultScanInterval |
|||
} |
|||
|
|||
// Try to get scan_interval field from config
|
|||
if configMap, ok := config.(map[string]interface{}); ok { |
|||
if scanInterval, exists := configMap["scan_interval"]; exists { |
|||
if scanIntervalDuration, ok := scanInterval.(time.Duration); ok { |
|||
return scanIntervalDuration |
|||
} |
|||
if scanIntervalString, ok := scanInterval.(string); ok { |
|||
if duration, err := time.ParseDuration(scanIntervalString); err == nil { |
|||
return duration |
|||
} |
|||
} |
|||
} |
|||
} |
|||
|
|||
return p.GlobalSettings.DefaultScanInterval |
|||
} |
|||
|
|||
// GetAllTaskTypes returns all configured task types
|
|||
func (p *MaintenancePolicy) GetAllTaskTypes() []TaskType { |
|||
if p.TaskConfigs == nil { |
|||
return []TaskType{} |
|||
} |
|||
|
|||
taskTypes := make([]TaskType, 0, len(p.TaskConfigs)) |
|||
for taskType := range p.TaskConfigs { |
|||
taskTypes = append(taskTypes, taskType) |
|||
} |
|||
return taskTypes |
|||
} |
|||
@ -0,0 +1,40 @@ |
|||
package types |
|||
|
|||
import ( |
|||
"time" |
|||
) |
|||
|
|||
// ClusterInfo contains cluster information for task detection
|
|||
type ClusterInfo struct { |
|||
Servers []*VolumeServerInfo |
|||
TotalVolumes int |
|||
TotalServers int |
|||
LastUpdated time.Time |
|||
} |
|||
|
|||
// VolumeHealthMetrics contains health information about a volume (simplified)
|
|||
type VolumeHealthMetrics struct { |
|||
VolumeID uint32 |
|||
Server string |
|||
Collection string |
|||
Size uint64 |
|||
DeletedBytes uint64 |
|||
GarbageRatio float64 |
|||
LastModified time.Time |
|||
Age time.Duration |
|||
ReplicaCount int |
|||
ExpectedReplicas int |
|||
IsReadOnly bool |
|||
HasRemoteCopy bool |
|||
IsECVolume bool |
|||
FullnessRatio float64 |
|||
} |
|||
|
|||
// VolumeServerInfo contains information about a volume server (simplified)
|
|||
type VolumeServerInfo struct { |
|||
Address string |
|||
Volumes int |
|||
UsedSpace uint64 |
|||
FreeSpace uint64 |
|||
IsActive bool |
|||
} |
|||
@ -0,0 +1,28 @@ |
|||
package types |
|||
|
|||
import ( |
|||
"time" |
|||
) |
|||
|
|||
// TaskDetector defines the interface for task detection
|
|||
type TaskDetector interface { |
|||
// GetTaskType returns the task type this detector handles
|
|||
GetTaskType() TaskType |
|||
|
|||
// ScanForTasks scans for tasks that need to be executed
|
|||
ScanForTasks(volumeMetrics []*VolumeHealthMetrics, clusterInfo *ClusterInfo) ([]*TaskDetectionResult, error) |
|||
|
|||
// ScanInterval returns how often this detector should scan
|
|||
ScanInterval() time.Duration |
|||
|
|||
// IsEnabled returns whether this detector is enabled
|
|||
IsEnabled() bool |
|||
} |
|||
|
|||
// PolicyConfigurableDetector defines the interface for detectors that can be configured from policy
|
|||
type PolicyConfigurableDetector interface { |
|||
TaskDetector |
|||
|
|||
// ConfigureFromPolicy configures the detector based on the maintenance policy
|
|||
ConfigureFromPolicy(policy interface{}) |
|||
} |
|||
@ -0,0 +1,54 @@ |
|||
package types |
|||
|
|||
// TaskRegistry manages task detectors and schedulers
|
|||
type TaskRegistry struct { |
|||
detectors map[TaskType]TaskDetector |
|||
schedulers map[TaskType]TaskScheduler |
|||
} |
|||
|
|||
// NewTaskRegistry creates a new simple task registry
|
|||
func NewTaskRegistry() *TaskRegistry { |
|||
return &TaskRegistry{ |
|||
detectors: make(map[TaskType]TaskDetector), |
|||
schedulers: make(map[TaskType]TaskScheduler), |
|||
} |
|||
} |
|||
|
|||
// RegisterTask registers both detector and scheduler for a task type
|
|||
func (r *TaskRegistry) RegisterTask(detector TaskDetector, scheduler TaskScheduler) { |
|||
taskType := detector.GetTaskType() |
|||
if taskType != scheduler.GetTaskType() { |
|||
panic("detector and scheduler task types must match") |
|||
} |
|||
|
|||
r.detectors[taskType] = detector |
|||
r.schedulers[taskType] = scheduler |
|||
} |
|||
|
|||
// GetDetector returns the detector for a task type
|
|||
func (r *TaskRegistry) GetDetector(taskType TaskType) TaskDetector { |
|||
return r.detectors[taskType] |
|||
} |
|||
|
|||
// GetScheduler returns the scheduler for a task type
|
|||
func (r *TaskRegistry) GetScheduler(taskType TaskType) TaskScheduler { |
|||
return r.schedulers[taskType] |
|||
} |
|||
|
|||
// GetAllDetectors returns all registered detectors
|
|||
func (r *TaskRegistry) GetAllDetectors() map[TaskType]TaskDetector { |
|||
result := make(map[TaskType]TaskDetector) |
|||
for k, v := range r.detectors { |
|||
result[k] = v |
|||
} |
|||
return result |
|||
} |
|||
|
|||
// GetAllSchedulers returns all registered schedulers
|
|||
func (r *TaskRegistry) GetAllSchedulers() map[TaskType]TaskScheduler { |
|||
result := make(map[TaskType]TaskScheduler) |
|||
for k, v := range r.schedulers { |
|||
result[k] = v |
|||
} |
|||
return result |
|||
} |
|||
@ -0,0 +1,32 @@ |
|||
package types |
|||
|
|||
import "time" |
|||
|
|||
// TaskScheduler defines the interface for task scheduling
|
|||
type TaskScheduler interface { |
|||
// GetTaskType returns the task type this scheduler handles
|
|||
GetTaskType() TaskType |
|||
|
|||
// CanScheduleNow determines if a task can be scheduled now
|
|||
CanScheduleNow(task *Task, runningTasks []*Task, availableWorkers []*Worker) bool |
|||
|
|||
// GetPriority returns the priority for tasks of this type
|
|||
GetPriority(task *Task) TaskPriority |
|||
|
|||
// GetMaxConcurrent returns the maximum concurrent tasks of this type
|
|||
GetMaxConcurrent() int |
|||
|
|||
// GetDefaultRepeatInterval returns the default interval to wait before repeating tasks of this type
|
|||
GetDefaultRepeatInterval() time.Duration |
|||
|
|||
// IsEnabled returns whether this scheduler is enabled
|
|||
IsEnabled() bool |
|||
} |
|||
|
|||
// PolicyConfigurableScheduler defines the interface for schedulers that can be configured from policy
|
|||
type PolicyConfigurableScheduler interface { |
|||
TaskScheduler |
|||
|
|||
// ConfigureFromPolicy configures the scheduler based on the maintenance policy
|
|||
ConfigureFromPolicy(policy interface{}) |
|||
} |
|||
@ -0,0 +1,89 @@ |
|||
package types |
|||
|
|||
import ( |
|||
"time" |
|||
) |
|||
|
|||
// TaskType represents the type of maintenance task
|
|||
type TaskType string |
|||
|
|||
const ( |
|||
TaskTypeVacuum TaskType = "vacuum" |
|||
TaskTypeErasureCoding TaskType = "erasure_coding" |
|||
TaskTypeBalance TaskType = "balance" |
|||
) |
|||
|
|||
// TaskStatus represents the status of a maintenance task
|
|||
type TaskStatus string |
|||
|
|||
const ( |
|||
TaskStatusPending TaskStatus = "pending" |
|||
TaskStatusAssigned TaskStatus = "assigned" |
|||
TaskStatusInProgress TaskStatus = "in_progress" |
|||
TaskStatusCompleted TaskStatus = "completed" |
|||
TaskStatusFailed TaskStatus = "failed" |
|||
TaskStatusCancelled TaskStatus = "cancelled" |
|||
) |
|||
|
|||
// TaskPriority represents the priority of a maintenance task
|
|||
type TaskPriority int |
|||
|
|||
const ( |
|||
TaskPriorityLow TaskPriority = 1 |
|||
TaskPriorityNormal TaskPriority = 5 |
|||
TaskPriorityHigh TaskPriority = 10 |
|||
) |
|||
|
|||
// Task represents a maintenance task
|
|||
type Task struct { |
|||
ID string `json:"id"` |
|||
Type TaskType `json:"type"` |
|||
Status TaskStatus `json:"status"` |
|||
Priority TaskPriority `json:"priority"` |
|||
VolumeID uint32 `json:"volume_id,omitempty"` |
|||
Server string `json:"server,omitempty"` |
|||
Collection string `json:"collection,omitempty"` |
|||
WorkerID string `json:"worker_id,omitempty"` |
|||
Progress float64 `json:"progress"` |
|||
Error string `json:"error,omitempty"` |
|||
Parameters map[string]interface{} `json:"parameters,omitempty"` |
|||
CreatedAt time.Time `json:"created_at"` |
|||
ScheduledAt time.Time `json:"scheduled_at"` |
|||
StartedAt *time.Time `json:"started_at,omitempty"` |
|||
CompletedAt *time.Time `json:"completed_at,omitempty"` |
|||
RetryCount int `json:"retry_count"` |
|||
MaxRetries int `json:"max_retries"` |
|||
} |
|||
|
|||
// TaskParams represents parameters for task execution
|
|||
type TaskParams struct { |
|||
VolumeID uint32 `json:"volume_id,omitempty"` |
|||
Server string `json:"server,omitempty"` |
|||
Collection string `json:"collection,omitempty"` |
|||
Parameters map[string]interface{} `json:"parameters,omitempty"` |
|||
} |
|||
|
|||
// TaskDetectionResult represents the result of scanning for maintenance needs
|
|||
type TaskDetectionResult struct { |
|||
TaskType TaskType `json:"task_type"` |
|||
VolumeID uint32 `json:"volume_id,omitempty"` |
|||
Server string `json:"server,omitempty"` |
|||
Collection string `json:"collection,omitempty"` |
|||
Priority TaskPriority `json:"priority"` |
|||
Reason string `json:"reason"` |
|||
Parameters map[string]interface{} `json:"parameters,omitempty"` |
|||
ScheduleAt time.Time `json:"schedule_at"` |
|||
} |
|||
|
|||
// ClusterReplicationTask represents a cluster replication task parameters
|
|||
type ClusterReplicationTask struct { |
|||
SourcePath string `json:"source_path"` |
|||
TargetCluster string `json:"target_cluster"` |
|||
TargetPath string `json:"target_path"` |
|||
ReplicationMode string `json:"replication_mode"` // "sync", "async", "backup"
|
|||
Priority int `json:"priority"` |
|||
Checksum string `json:"checksum,omitempty"` |
|||
FileSize int64 `json:"file_size"` |
|||
CreatedAt time.Time `json:"created_at"` |
|||
Metadata map[string]string `json:"metadata,omitempty"` |
|||
} |
|||
@ -0,0 +1,281 @@ |
|||
package types |
|||
|
|||
import ( |
|||
"fmt" |
|||
"html/template" |
|||
"time" |
|||
) |
|||
|
|||
// TaskUIProvider defines how tasks provide their configuration UI
|
|||
type TaskUIProvider interface { |
|||
// GetTaskType returns the task type
|
|||
GetTaskType() TaskType |
|||
|
|||
// GetDisplayName returns the human-readable name
|
|||
GetDisplayName() string |
|||
|
|||
// GetDescription returns a description of what this task does
|
|||
GetDescription() string |
|||
|
|||
// GetIcon returns the icon CSS class or HTML for this task type
|
|||
GetIcon() string |
|||
|
|||
// RenderConfigForm renders the configuration form HTML
|
|||
RenderConfigForm(currentConfig interface{}) (template.HTML, error) |
|||
|
|||
// ParseConfigForm parses form data into configuration
|
|||
ParseConfigForm(formData map[string][]string) (interface{}, error) |
|||
|
|||
// GetCurrentConfig returns the current configuration
|
|||
GetCurrentConfig() interface{} |
|||
|
|||
// ApplyConfig applies the new configuration
|
|||
ApplyConfig(config interface{}) error |
|||
} |
|||
|
|||
// TaskStats represents runtime statistics for a task type
|
|||
type TaskStats struct { |
|||
TaskType TaskType `json:"task_type"` |
|||
DisplayName string `json:"display_name"` |
|||
Enabled bool `json:"enabled"` |
|||
LastScan time.Time `json:"last_scan"` |
|||
NextScan time.Time `json:"next_scan"` |
|||
PendingTasks int `json:"pending_tasks"` |
|||
RunningTasks int `json:"running_tasks"` |
|||
CompletedToday int `json:"completed_today"` |
|||
FailedToday int `json:"failed_today"` |
|||
MaxConcurrent int `json:"max_concurrent"` |
|||
ScanInterval time.Duration `json:"scan_interval"` |
|||
} |
|||
|
|||
// UIRegistry manages task UI providers
|
|||
type UIRegistry struct { |
|||
providers map[TaskType]TaskUIProvider |
|||
} |
|||
|
|||
// NewUIRegistry creates a new UI registry
|
|||
func NewUIRegistry() *UIRegistry { |
|||
return &UIRegistry{ |
|||
providers: make(map[TaskType]TaskUIProvider), |
|||
} |
|||
} |
|||
|
|||
// RegisterUI registers a task UI provider
|
|||
func (r *UIRegistry) RegisterUI(provider TaskUIProvider) { |
|||
r.providers[provider.GetTaskType()] = provider |
|||
} |
|||
|
|||
// GetProvider returns the UI provider for a task type
|
|||
func (r *UIRegistry) GetProvider(taskType TaskType) TaskUIProvider { |
|||
return r.providers[taskType] |
|||
} |
|||
|
|||
// GetAllProviders returns all registered UI providers
|
|||
func (r *UIRegistry) GetAllProviders() map[TaskType]TaskUIProvider { |
|||
result := make(map[TaskType]TaskUIProvider) |
|||
for k, v := range r.providers { |
|||
result[k] = v |
|||
} |
|||
return result |
|||
} |
|||
|
|||
// Common UI data structures for shared components
|
|||
type TaskListData struct { |
|||
Tasks []*Task `json:"tasks"` |
|||
TaskStats []*TaskStats `json:"task_stats"` |
|||
LastUpdated time.Time `json:"last_updated"` |
|||
} |
|||
|
|||
type TaskDetailsData struct { |
|||
Task *Task `json:"task"` |
|||
TaskType TaskType `json:"task_type"` |
|||
DisplayName string `json:"display_name"` |
|||
Description string `json:"description"` |
|||
Stats *TaskStats `json:"stats"` |
|||
ConfigForm template.HTML `json:"config_form"` |
|||
LastUpdated time.Time `json:"last_updated"` |
|||
} |
|||
|
|||
// Common form field types for simple form building
|
|||
type FormField struct { |
|||
Name string `json:"name"` |
|||
Label string `json:"label"` |
|||
Type string `json:"type"` // text, number, checkbox, select, duration
|
|||
Value interface{} `json:"value"` |
|||
Description string `json:"description"` |
|||
Required bool `json:"required"` |
|||
Options []FormOption `json:"options,omitempty"` // For select fields
|
|||
} |
|||
|
|||
type FormOption struct { |
|||
Value string `json:"value"` |
|||
Label string `json:"label"` |
|||
} |
|||
|
|||
// Helper for building forms in code
|
|||
type FormBuilder struct { |
|||
fields []FormField |
|||
} |
|||
|
|||
// NewFormBuilder creates a new form builder
|
|||
func NewFormBuilder() *FormBuilder { |
|||
return &FormBuilder{ |
|||
fields: make([]FormField, 0), |
|||
} |
|||
} |
|||
|
|||
// AddTextField adds a text input field
|
|||
func (fb *FormBuilder) AddTextField(name, label, description string, value string, required bool) *FormBuilder { |
|||
fb.fields = append(fb.fields, FormField{ |
|||
Name: name, |
|||
Label: label, |
|||
Type: "text", |
|||
Value: value, |
|||
Description: description, |
|||
Required: required, |
|||
}) |
|||
return fb |
|||
} |
|||
|
|||
// AddNumberField adds a number input field
|
|||
func (fb *FormBuilder) AddNumberField(name, label, description string, value float64, required bool) *FormBuilder { |
|||
fb.fields = append(fb.fields, FormField{ |
|||
Name: name, |
|||
Label: label, |
|||
Type: "number", |
|||
Value: value, |
|||
Description: description, |
|||
Required: required, |
|||
}) |
|||
return fb |
|||
} |
|||
|
|||
// AddCheckboxField adds a checkbox field
|
|||
func (fb *FormBuilder) AddCheckboxField(name, label, description string, value bool) *FormBuilder { |
|||
fb.fields = append(fb.fields, FormField{ |
|||
Name: name, |
|||
Label: label, |
|||
Type: "checkbox", |
|||
Value: value, |
|||
Description: description, |
|||
Required: false, |
|||
}) |
|||
return fb |
|||
} |
|||
|
|||
// AddSelectField adds a select dropdown field
|
|||
func (fb *FormBuilder) AddSelectField(name, label, description string, value string, options []FormOption, required bool) *FormBuilder { |
|||
fb.fields = append(fb.fields, FormField{ |
|||
Name: name, |
|||
Label: label, |
|||
Type: "select", |
|||
Value: value, |
|||
Description: description, |
|||
Required: required, |
|||
Options: options, |
|||
}) |
|||
return fb |
|||
} |
|||
|
|||
// AddDurationField adds a duration input field
|
|||
func (fb *FormBuilder) AddDurationField(name, label, description string, value time.Duration, required bool) *FormBuilder { |
|||
fb.fields = append(fb.fields, FormField{ |
|||
Name: name, |
|||
Label: label, |
|||
Type: "duration", |
|||
Value: value.String(), |
|||
Description: description, |
|||
Required: required, |
|||
}) |
|||
return fb |
|||
} |
|||
|
|||
// Build generates the HTML form fields with Bootstrap styling
|
|||
func (fb *FormBuilder) Build() template.HTML { |
|||
html := "" |
|||
|
|||
for _, field := range fb.fields { |
|||
html += fb.renderField(field) |
|||
} |
|||
|
|||
return template.HTML(html) |
|||
} |
|||
|
|||
// renderField renders a single form field with Bootstrap classes
|
|||
func (fb *FormBuilder) renderField(field FormField) string { |
|||
html := "<div class=\"mb-3\">\n" |
|||
|
|||
// Special handling for checkbox fields
|
|||
if field.Type == "checkbox" { |
|||
checked := "" |
|||
if field.Value.(bool) { |
|||
checked = " checked" |
|||
} |
|||
html += " <div class=\"form-check\">\n" |
|||
html += " <input type=\"checkbox\" class=\"form-check-input\" id=\"" + field.Name + "\" name=\"" + field.Name + "\"" + checked + ">\n" |
|||
html += " <label class=\"form-check-label\" for=\"" + field.Name + "\">" + field.Label + "</label>\n" |
|||
html += " </div>\n" |
|||
// Description for checkbox
|
|||
if field.Description != "" { |
|||
html += " <div class=\"form-text text-muted\">" + field.Description + "</div>\n" |
|||
} |
|||
html += "</div>\n" |
|||
return html |
|||
} |
|||
|
|||
// Label for non-checkbox fields
|
|||
required := "" |
|||
if field.Required { |
|||
required = " <span class=\"text-danger\">*</span>" |
|||
} |
|||
html += " <label for=\"" + field.Name + "\" class=\"form-label\">" + field.Label + required + "</label>\n" |
|||
|
|||
// Input based on type
|
|||
switch field.Type { |
|||
case "text": |
|||
html += " <input type=\"text\" class=\"form-control\" id=\"" + field.Name + "\" name=\"" + field.Name + "\" value=\"" + field.Value.(string) + "\"" |
|||
if field.Required { |
|||
html += " required" |
|||
} |
|||
html += ">\n" |
|||
|
|||
case "number": |
|||
html += " <input type=\"number\" class=\"form-control\" id=\"" + field.Name + "\" name=\"" + field.Name + "\" step=\"any\" value=\"" + |
|||
fmt.Sprintf("%v", field.Value) + "\"" |
|||
if field.Required { |
|||
html += " required" |
|||
} |
|||
html += ">\n" |
|||
|
|||
case "select": |
|||
html += " <select class=\"form-select\" id=\"" + field.Name + "\" name=\"" + field.Name + "\"" |
|||
if field.Required { |
|||
html += " required" |
|||
} |
|||
html += ">\n" |
|||
for _, option := range field.Options { |
|||
selected := "" |
|||
if option.Value == field.Value.(string) { |
|||
selected = " selected" |
|||
} |
|||
html += " <option value=\"" + option.Value + "\"" + selected + ">" + option.Label + "</option>\n" |
|||
} |
|||
html += " </select>\n" |
|||
|
|||
case "duration": |
|||
html += " <input type=\"text\" class=\"form-control\" id=\"" + field.Name + "\" name=\"" + field.Name + "\" value=\"" + field.Value.(string) + |
|||
"\" placeholder=\"e.g., 30m, 2h, 24h\"" |
|||
if field.Required { |
|||
html += " required" |
|||
} |
|||
html += ">\n" |
|||
} |
|||
|
|||
// Description for non-checkbox fields
|
|||
if field.Description != "" { |
|||
html += " <div class=\"form-text text-muted\">" + field.Description + "</div>\n" |
|||
} |
|||
|
|||
html += "</div>\n" |
|||
return html |
|||
} |
|||
@ -0,0 +1,63 @@ |
|||
package types |
|||
|
|||
import ( |
|||
"github.com/seaweedfs/seaweedfs/weed/admin/view/components" |
|||
) |
|||
|
|||
// TaskUITemplProvider defines how tasks provide their configuration UI using templ components
|
|||
type TaskUITemplProvider interface { |
|||
// GetTaskType returns the task type
|
|||
GetTaskType() TaskType |
|||
|
|||
// GetDisplayName returns the human-readable name
|
|||
GetDisplayName() string |
|||
|
|||
// GetDescription returns a description of what this task does
|
|||
GetDescription() string |
|||
|
|||
// GetIcon returns the icon CSS class or HTML for this task type
|
|||
GetIcon() string |
|||
|
|||
// RenderConfigSections renders the configuration as templ section data
|
|||
RenderConfigSections(currentConfig interface{}) ([]components.ConfigSectionData, error) |
|||
|
|||
// ParseConfigForm parses form data into configuration
|
|||
ParseConfigForm(formData map[string][]string) (interface{}, error) |
|||
|
|||
// GetCurrentConfig returns the current configuration
|
|||
GetCurrentConfig() interface{} |
|||
|
|||
// ApplyConfig applies the new configuration
|
|||
ApplyConfig(config interface{}) error |
|||
} |
|||
|
|||
// UITemplRegistry manages task UI providers that use templ components
|
|||
type UITemplRegistry struct { |
|||
providers map[TaskType]TaskUITemplProvider |
|||
} |
|||
|
|||
// NewUITemplRegistry creates a new templ-based UI registry
|
|||
func NewUITemplRegistry() *UITemplRegistry { |
|||
return &UITemplRegistry{ |
|||
providers: make(map[TaskType]TaskUITemplProvider), |
|||
} |
|||
} |
|||
|
|||
// RegisterUI registers a task UI provider
|
|||
func (r *UITemplRegistry) RegisterUI(provider TaskUITemplProvider) { |
|||
r.providers[provider.GetTaskType()] = provider |
|||
} |
|||
|
|||
// GetProvider returns the UI provider for a task type
|
|||
func (r *UITemplRegistry) GetProvider(taskType TaskType) TaskUITemplProvider { |
|||
return r.providers[taskType] |
|||
} |
|||
|
|||
// GetAllProviders returns all registered UI providers
|
|||
func (r *UITemplRegistry) GetAllProviders() map[TaskType]TaskUITemplProvider { |
|||
result := make(map[TaskType]TaskUITemplProvider) |
|||
for k, v := range r.providers { |
|||
result[k] = v |
|||
} |
|||
return result |
|||
} |
|||
@ -0,0 +1,111 @@ |
|||
package types |
|||
|
|||
import ( |
|||
"time" |
|||
) |
|||
|
|||
// Worker represents a maintenance worker instance
|
|||
type Worker struct { |
|||
ID string `json:"id"` |
|||
Address string `json:"address"` |
|||
LastHeartbeat time.Time `json:"last_heartbeat"` |
|||
Status string `json:"status"` // active, inactive, busy
|
|||
CurrentTask *Task `json:"current_task,omitempty"` |
|||
Capabilities []TaskType `json:"capabilities"` |
|||
MaxConcurrent int `json:"max_concurrent"` |
|||
CurrentLoad int `json:"current_load"` |
|||
} |
|||
|
|||
// WorkerStatus represents the current status of a worker
|
|||
type WorkerStatus struct { |
|||
WorkerID string `json:"worker_id"` |
|||
Status string `json:"status"` |
|||
Capabilities []TaskType `json:"capabilities"` |
|||
MaxConcurrent int `json:"max_concurrent"` |
|||
CurrentLoad int `json:"current_load"` |
|||
LastHeartbeat time.Time `json:"last_heartbeat"` |
|||
CurrentTasks []Task `json:"current_tasks"` |
|||
Uptime time.Duration `json:"uptime"` |
|||
TasksCompleted int `json:"tasks_completed"` |
|||
TasksFailed int `json:"tasks_failed"` |
|||
} |
|||
|
|||
// WorkerDetailsData represents detailed worker information
|
|||
type WorkerDetailsData struct { |
|||
Worker *Worker `json:"worker"` |
|||
CurrentTasks []*Task `json:"current_tasks"` |
|||
RecentTasks []*Task `json:"recent_tasks"` |
|||
Performance *WorkerPerformance `json:"performance"` |
|||
LastUpdated time.Time `json:"last_updated"` |
|||
} |
|||
|
|||
// WorkerPerformance tracks worker performance metrics
|
|||
type WorkerPerformance struct { |
|||
TasksCompleted int `json:"tasks_completed"` |
|||
TasksFailed int `json:"tasks_failed"` |
|||
AverageTaskTime time.Duration `json:"average_task_time"` |
|||
Uptime time.Duration `json:"uptime"` |
|||
SuccessRate float64 `json:"success_rate"` |
|||
} |
|||
|
|||
// RegistryStats represents statistics for the worker registry
|
|||
type RegistryStats struct { |
|||
TotalWorkers int `json:"total_workers"` |
|||
ActiveWorkers int `json:"active_workers"` |
|||
BusyWorkers int `json:"busy_workers"` |
|||
IdleWorkers int `json:"idle_workers"` |
|||
TotalTasks int `json:"total_tasks"` |
|||
CompletedTasks int `json:"completed_tasks"` |
|||
FailedTasks int `json:"failed_tasks"` |
|||
StartTime time.Time `json:"start_time"` |
|||
Uptime time.Duration `json:"uptime"` |
|||
LastUpdated time.Time `json:"last_updated"` |
|||
} |
|||
|
|||
// WorkerSummary represents a summary of all workers
|
|||
type WorkerSummary struct { |
|||
TotalWorkers int `json:"total_workers"` |
|||
ByStatus map[string]int `json:"by_status"` |
|||
ByCapability map[TaskType]int `json:"by_capability"` |
|||
TotalLoad int `json:"total_load"` |
|||
MaxCapacity int `json:"max_capacity"` |
|||
} |
|||
|
|||
// WorkerFactory creates worker instances
|
|||
type WorkerFactory interface { |
|||
Create(config WorkerConfig) (WorkerInterface, error) |
|||
Type() string |
|||
Description() string |
|||
} |
|||
|
|||
// WorkerInterface defines the interface for all worker implementations
|
|||
type WorkerInterface interface { |
|||
ID() string |
|||
Start() error |
|||
Stop() error |
|||
RegisterTask(taskType TaskType, factory TaskFactory) |
|||
GetCapabilities() []TaskType |
|||
GetStatus() WorkerStatus |
|||
HandleTask(task *Task) error |
|||
SetCapabilities(capabilities []TaskType) |
|||
SetMaxConcurrent(max int) |
|||
SetHeartbeatInterval(interval time.Duration) |
|||
SetTaskRequestInterval(interval time.Duration) |
|||
} |
|||
|
|||
// TaskFactory creates task instances
|
|||
type TaskFactory interface { |
|||
Create(params TaskParams) (TaskInterface, error) |
|||
Capabilities() []string |
|||
Description() string |
|||
} |
|||
|
|||
// TaskInterface defines the interface for all task implementations
|
|||
type TaskInterface interface { |
|||
Type() TaskType |
|||
Execute(params TaskParams) error |
|||
Validate(params TaskParams) error |
|||
EstimateTime(params TaskParams) time.Duration |
|||
GetProgress() float64 |
|||
Cancel() error |
|||
} |
|||
@ -0,0 +1,410 @@ |
|||
package worker |
|||
|
|||
import ( |
|||
"fmt" |
|||
"os" |
|||
"sync" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/tasks" |
|||
"github.com/seaweedfs/seaweedfs/weed/worker/types" |
|||
|
|||
// Import task packages to trigger their auto-registration
|
|||
_ "github.com/seaweedfs/seaweedfs/weed/worker/tasks/balance" |
|||
_ "github.com/seaweedfs/seaweedfs/weed/worker/tasks/erasure_coding" |
|||
_ "github.com/seaweedfs/seaweedfs/weed/worker/tasks/vacuum" |
|||
) |
|||
|
|||
// Worker represents a maintenance worker instance
|
|||
type Worker struct { |
|||
id string |
|||
config *types.WorkerConfig |
|||
registry *tasks.TaskRegistry |
|||
currentTasks map[string]*types.Task |
|||
adminClient AdminClient |
|||
running bool |
|||
stopChan chan struct{} |
|||
mutex sync.RWMutex |
|||
startTime time.Time |
|||
tasksCompleted int |
|||
tasksFailed int |
|||
heartbeatTicker *time.Ticker |
|||
requestTicker *time.Ticker |
|||
} |
|||
|
|||
// AdminClient defines the interface for communicating with the admin server
|
|||
type AdminClient interface { |
|||
Connect() error |
|||
Disconnect() error |
|||
RegisterWorker(worker *types.Worker) error |
|||
SendHeartbeat(workerID string, status *types.WorkerStatus) error |
|||
RequestTask(workerID string, capabilities []types.TaskType) (*types.Task, error) |
|||
CompleteTask(taskID string, success bool, errorMsg string) error |
|||
UpdateTaskProgress(taskID string, progress float64) error |
|||
IsConnected() bool |
|||
} |
|||
|
|||
// NewWorker creates a new worker instance
|
|||
func NewWorker(config *types.WorkerConfig) (*Worker, error) { |
|||
if config == nil { |
|||
config = types.DefaultWorkerConfig() |
|||
} |
|||
|
|||
// Always auto-generate worker ID
|
|||
hostname, _ := os.Hostname() |
|||
workerID := fmt.Sprintf("worker-%s-%d", hostname, time.Now().Unix()) |
|||
|
|||
// Use the global registry that already has all tasks registered
|
|||
registry := tasks.GetGlobalRegistry() |
|||
|
|||
worker := &Worker{ |
|||
id: workerID, |
|||
config: config, |
|||
registry: registry, |
|||
currentTasks: make(map[string]*types.Task), |
|||
stopChan: make(chan struct{}), |
|||
startTime: time.Now(), |
|||
} |
|||
|
|||
glog.V(1).Infof("Worker created with %d registered task types", len(registry.GetSupportedTypes())) |
|||
|
|||
return worker, nil |
|||
} |
|||
|
|||
// ID returns the worker ID
|
|||
func (w *Worker) ID() string { |
|||
return w.id |
|||
} |
|||
|
|||
// Start starts the worker
|
|||
func (w *Worker) Start() error { |
|||
w.mutex.Lock() |
|||
defer w.mutex.Unlock() |
|||
|
|||
if w.running { |
|||
return fmt.Errorf("worker is already running") |
|||
} |
|||
|
|||
if w.adminClient == nil { |
|||
return fmt.Errorf("admin client is not set") |
|||
} |
|||
|
|||
// Connect to admin server
|
|||
if err := w.adminClient.Connect(); err != nil { |
|||
return fmt.Errorf("failed to connect to admin server: %v", err) |
|||
} |
|||
|
|||
w.running = true |
|||
w.startTime = time.Now() |
|||
|
|||
// Register with admin server
|
|||
workerInfo := &types.Worker{ |
|||
ID: w.id, |
|||
Capabilities: w.config.Capabilities, |
|||
MaxConcurrent: w.config.MaxConcurrent, |
|||
Status: "active", |
|||
CurrentLoad: 0, |
|||
LastHeartbeat: time.Now(), |
|||
} |
|||
|
|||
if err := w.adminClient.RegisterWorker(workerInfo); err != nil { |
|||
w.running = false |
|||
w.adminClient.Disconnect() |
|||
return fmt.Errorf("failed to register worker: %v", err) |
|||
} |
|||
|
|||
// Start worker loops
|
|||
go w.heartbeatLoop() |
|||
go w.taskRequestLoop() |
|||
|
|||
glog.Infof("Worker %s started", w.id) |
|||
return nil |
|||
} |
|||
|
|||
// Stop stops the worker
|
|||
func (w *Worker) Stop() error { |
|||
w.mutex.Lock() |
|||
defer w.mutex.Unlock() |
|||
|
|||
if !w.running { |
|||
return nil |
|||
} |
|||
|
|||
w.running = false |
|||
close(w.stopChan) |
|||
|
|||
// Stop tickers
|
|||
if w.heartbeatTicker != nil { |
|||
w.heartbeatTicker.Stop() |
|||
} |
|||
if w.requestTicker != nil { |
|||
w.requestTicker.Stop() |
|||
} |
|||
|
|||
// Wait for current tasks to complete or timeout
|
|||
timeout := time.NewTimer(30 * time.Second) |
|||
defer timeout.Stop() |
|||
|
|||
for len(w.currentTasks) > 0 { |
|||
select { |
|||
case <-timeout.C: |
|||
glog.Warningf("Worker %s stopping with %d tasks still running", w.id, len(w.currentTasks)) |
|||
break |
|||
case <-time.After(time.Second): |
|||
// Check again
|
|||
} |
|||
} |
|||
|
|||
// Disconnect from admin server
|
|||
if w.adminClient != nil { |
|||
if err := w.adminClient.Disconnect(); err != nil { |
|||
glog.Errorf("Error disconnecting from admin server: %v", err) |
|||
} |
|||
} |
|||
|
|||
glog.Infof("Worker %s stopped", w.id) |
|||
return nil |
|||
} |
|||
|
|||
// RegisterTask registers a task factory
|
|||
func (w *Worker) RegisterTask(taskType types.TaskType, factory types.TaskFactory) { |
|||
w.registry.Register(taskType, factory) |
|||
} |
|||
|
|||
// GetCapabilities returns the worker capabilities
|
|||
func (w *Worker) GetCapabilities() []types.TaskType { |
|||
return w.config.Capabilities |
|||
} |
|||
|
|||
// GetStatus returns the current worker status
|
|||
func (w *Worker) GetStatus() types.WorkerStatus { |
|||
w.mutex.RLock() |
|||
defer w.mutex.RUnlock() |
|||
|
|||
var currentTasks []types.Task |
|||
for _, task := range w.currentTasks { |
|||
currentTasks = append(currentTasks, *task) |
|||
} |
|||
|
|||
status := "active" |
|||
if len(w.currentTasks) >= w.config.MaxConcurrent { |
|||
status = "busy" |
|||
} |
|||
|
|||
return types.WorkerStatus{ |
|||
WorkerID: w.id, |
|||
Status: status, |
|||
Capabilities: w.config.Capabilities, |
|||
MaxConcurrent: w.config.MaxConcurrent, |
|||
CurrentLoad: len(w.currentTasks), |
|||
LastHeartbeat: time.Now(), |
|||
CurrentTasks: currentTasks, |
|||
Uptime: time.Since(w.startTime), |
|||
TasksCompleted: w.tasksCompleted, |
|||
TasksFailed: w.tasksFailed, |
|||
} |
|||
} |
|||
|
|||
// HandleTask handles a task execution
|
|||
func (w *Worker) HandleTask(task *types.Task) error { |
|||
w.mutex.Lock() |
|||
if len(w.currentTasks) >= w.config.MaxConcurrent { |
|||
w.mutex.Unlock() |
|||
return fmt.Errorf("worker is at capacity") |
|||
} |
|||
w.currentTasks[task.ID] = task |
|||
w.mutex.Unlock() |
|||
|
|||
// Execute task in goroutine
|
|||
go w.executeTask(task) |
|||
|
|||
return nil |
|||
} |
|||
|
|||
// SetCapabilities sets the worker capabilities
|
|||
func (w *Worker) SetCapabilities(capabilities []types.TaskType) { |
|||
w.config.Capabilities = capabilities |
|||
} |
|||
|
|||
// SetMaxConcurrent sets the maximum concurrent tasks
|
|||
func (w *Worker) SetMaxConcurrent(max int) { |
|||
w.config.MaxConcurrent = max |
|||
} |
|||
|
|||
// SetHeartbeatInterval sets the heartbeat interval
|
|||
func (w *Worker) SetHeartbeatInterval(interval time.Duration) { |
|||
w.config.HeartbeatInterval = interval |
|||
} |
|||
|
|||
// SetTaskRequestInterval sets the task request interval
|
|||
func (w *Worker) SetTaskRequestInterval(interval time.Duration) { |
|||
w.config.TaskRequestInterval = interval |
|||
} |
|||
|
|||
// SetAdminClient sets the admin client
|
|||
func (w *Worker) SetAdminClient(client AdminClient) { |
|||
w.adminClient = client |
|||
} |
|||
|
|||
// executeTask executes a task
|
|||
func (w *Worker) executeTask(task *types.Task) { |
|||
defer func() { |
|||
w.mutex.Lock() |
|||
delete(w.currentTasks, task.ID) |
|||
w.mutex.Unlock() |
|||
}() |
|||
|
|||
glog.Infof("Worker %s executing task %s: %s", w.id, task.ID, task.Type) |
|||
|
|||
// Create task instance
|
|||
taskParams := types.TaskParams{ |
|||
VolumeID: task.VolumeID, |
|||
Server: task.Server, |
|||
Collection: task.Collection, |
|||
Parameters: task.Parameters, |
|||
} |
|||
|
|||
taskInstance, err := w.registry.CreateTask(task.Type, taskParams) |
|||
if err != nil { |
|||
w.completeTask(task.ID, false, fmt.Sprintf("failed to create task: %v", err)) |
|||
return |
|||
} |
|||
|
|||
// Execute task
|
|||
err = taskInstance.Execute(taskParams) |
|||
|
|||
// Report completion
|
|||
if err != nil { |
|||
w.completeTask(task.ID, false, err.Error()) |
|||
w.tasksFailed++ |
|||
glog.Errorf("Worker %s failed to execute task %s: %v", w.id, task.ID, err) |
|||
} else { |
|||
w.completeTask(task.ID, true, "") |
|||
w.tasksCompleted++ |
|||
glog.Infof("Worker %s completed task %s successfully", w.id, task.ID) |
|||
} |
|||
} |
|||
|
|||
// completeTask reports task completion to admin server
|
|||
func (w *Worker) completeTask(taskID string, success bool, errorMsg string) { |
|||
if w.adminClient != nil { |
|||
if err := w.adminClient.CompleteTask(taskID, success, errorMsg); err != nil { |
|||
glog.Errorf("Failed to report task completion: %v", err) |
|||
} |
|||
} |
|||
} |
|||
|
|||
// heartbeatLoop sends periodic heartbeats to the admin server
|
|||
func (w *Worker) heartbeatLoop() { |
|||
w.heartbeatTicker = time.NewTicker(w.config.HeartbeatInterval) |
|||
defer w.heartbeatTicker.Stop() |
|||
|
|||
for { |
|||
select { |
|||
case <-w.stopChan: |
|||
return |
|||
case <-w.heartbeatTicker.C: |
|||
w.sendHeartbeat() |
|||
} |
|||
} |
|||
} |
|||
|
|||
// taskRequestLoop periodically requests new tasks from the admin server
|
|||
func (w *Worker) taskRequestLoop() { |
|||
w.requestTicker = time.NewTicker(w.config.TaskRequestInterval) |
|||
defer w.requestTicker.Stop() |
|||
|
|||
for { |
|||
select { |
|||
case <-w.stopChan: |
|||
return |
|||
case <-w.requestTicker.C: |
|||
w.requestTasks() |
|||
} |
|||
} |
|||
} |
|||
|
|||
// sendHeartbeat sends heartbeat to admin server
|
|||
func (w *Worker) sendHeartbeat() { |
|||
if w.adminClient != nil { |
|||
if err := w.adminClient.SendHeartbeat(w.id, &types.WorkerStatus{ |
|||
WorkerID: w.id, |
|||
Status: "active", |
|||
Capabilities: w.config.Capabilities, |
|||
MaxConcurrent: w.config.MaxConcurrent, |
|||
CurrentLoad: len(w.currentTasks), |
|||
LastHeartbeat: time.Now(), |
|||
}); err != nil { |
|||
glog.Warningf("Failed to send heartbeat: %v", err) |
|||
} |
|||
} |
|||
} |
|||
|
|||
// requestTasks requests new tasks from the admin server
|
|||
func (w *Worker) requestTasks() { |
|||
w.mutex.RLock() |
|||
currentLoad := len(w.currentTasks) |
|||
w.mutex.RUnlock() |
|||
|
|||
if currentLoad >= w.config.MaxConcurrent { |
|||
return // Already at capacity
|
|||
} |
|||
|
|||
if w.adminClient != nil { |
|||
task, err := w.adminClient.RequestTask(w.id, w.config.Capabilities) |
|||
if err != nil { |
|||
glog.V(2).Infof("Failed to request task: %v", err) |
|||
return |
|||
} |
|||
|
|||
if task != nil { |
|||
if err := w.HandleTask(task); err != nil { |
|||
glog.Errorf("Failed to handle task: %v", err) |
|||
} |
|||
} |
|||
} |
|||
} |
|||
|
|||
// GetTaskRegistry returns the task registry
|
|||
func (w *Worker) GetTaskRegistry() *tasks.TaskRegistry { |
|||
return w.registry |
|||
} |
|||
|
|||
// GetCurrentTasks returns the current tasks
|
|||
func (w *Worker) GetCurrentTasks() map[string]*types.Task { |
|||
w.mutex.RLock() |
|||
defer w.mutex.RUnlock() |
|||
|
|||
tasks := make(map[string]*types.Task) |
|||
for id, task := range w.currentTasks { |
|||
tasks[id] = task |
|||
} |
|||
return tasks |
|||
} |
|||
|
|||
// GetConfig returns the worker configuration
|
|||
func (w *Worker) GetConfig() *types.WorkerConfig { |
|||
return w.config |
|||
} |
|||
|
|||
// GetPerformanceMetrics returns performance metrics
|
|||
func (w *Worker) GetPerformanceMetrics() *types.WorkerPerformance { |
|||
w.mutex.RLock() |
|||
defer w.mutex.RUnlock() |
|||
|
|||
uptime := time.Since(w.startTime) |
|||
var successRate float64 |
|||
totalTasks := w.tasksCompleted + w.tasksFailed |
|||
if totalTasks > 0 { |
|||
successRate = float64(w.tasksCompleted) / float64(totalTasks) * 100 |
|||
} |
|||
|
|||
return &types.WorkerPerformance{ |
|||
TasksCompleted: w.tasksCompleted, |
|||
TasksFailed: w.tasksFailed, |
|||
AverageTaskTime: 0, // Would need to track this
|
|||
Uptime: uptime, |
|||
SuccessRate: successRate, |
|||
} |
|||
} |
|||
Write
Preview
Loading…
Cancel
Save
Reference in new issue