Browse Source

Admin UI: Fetch task logs (#7114)

* show task details

* loading tasks

* task UI works

* generic rendering

* rendering the export link

* removing placementConflicts from task parameters

* remove TaskSourceLocation

* remove "Server ID" column

* rendering balance task source

* sources and targets

* fix ec task generation

* move info

* render timeline

* simplified worker id

* simplify

* read task logs from worker

* isValidTaskID

* address comments

* Update weed/worker/tasks/balance/execution.go

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>

* Update weed/worker/tasks/erasure_coding/ec_task.go

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>

* Update weed/worker/tasks/task_log_handler.go

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>

* fix shard ids

* plan distributing shard id

* rendering planned shards in task details

* remove Conflicts

* worker logs correctly

* pass in dc and rack

* task logging

* Update weed/admin/maintenance/maintenance_queue.go

Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>

* display log details

* logs have fields now

* sort field keys

* fix link

* fix collection filtering

* avoid hard coded ec shard counts

---------

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>
pull/7122/head
Chris Lu 2 months ago
committed by GitHub
parent
commit
25bbf4c3d4
No known key found for this signature in database GPG Key ID: B5690EEEBB952194
  1. 265
      weed/admin/dash/admin_server.go
  2. 545
      weed/admin/dash/config_persistence.go
  3. 13
      weed/admin/dash/ec_shard_management.go
  4. 18
      weed/admin/dash/volume_management.go
  5. 164
      weed/admin/dash/worker_grpc_server.go
  6. 6
      weed/admin/handlers/admin_handlers.go
  7. 8
      weed/admin/handlers/cluster_handlers.go
  8. 91
      weed/admin/handlers/maintenance_handlers.go
  9. 236
      weed/admin/maintenance/maintenance_queue.go
  10. 4
      weed/admin/maintenance/maintenance_scanner.go
  11. 67
      weed/admin/maintenance/maintenance_types.go
  12. 19
      weed/admin/topology/structs.go
  13. 9
      weed/admin/topology/task_management.go
  14. 12
      weed/admin/topology/topology_management.go
  15. 2
      weed/admin/view/app/admin.templ
  16. 2
      weed/admin/view/app/admin_templ.go
  17. 13
      weed/admin/view/app/cluster_ec_volumes.templ
  18. 501
      weed/admin/view/app/cluster_ec_volumes_templ.go
  19. 4
      weed/admin/view/app/cluster_volume_servers.templ
  20. 217
      weed/admin/view/app/cluster_volume_servers_templ.go
  21. 10
      weed/admin/view/app/collection_details.templ
  22. 99
      weed/admin/view/app/collection_details_templ.go
  23. 221
      weed/admin/view/app/maintenance_queue.templ
  24. 718
      weed/admin/view/app/maintenance_queue_templ.go
  25. 3
      weed/admin/view/app/task_config_schema.templ
  26. 425
      weed/admin/view/app/task_config_schema_templ.go
  27. 1118
      weed/admin/view/app/task_detail.templ
  28. 1628
      weed/admin/view/app/task_detail_templ.go
  29. 167
      weed/pb/worker.proto
  30. 1332
      weed/pb/worker_pb/worker.pb.go
  31. 10
      weed/worker/client.go
  32. 85
      weed/worker/log_adapter.go
  33. 31
      weed/worker/tasks/balance/balance_task.go
  34. 66
      weed/worker/tasks/balance/detection.go
  35. 72
      weed/worker/tasks/balance/execution.go
  36. 5
      weed/worker/tasks/balance/register.go
  37. 39
      weed/worker/tasks/base/typed_task.go
  38. 194
      weed/worker/tasks/erasure_coding/detection.go
  39. 368
      weed/worker/tasks/erasure_coding/ec_task.go
  40. 5
      weed/worker/tasks/erasure_coding/register.go
  41. 66
      weed/worker/tasks/task.go
  42. 36
      weed/worker/tasks/task_log_handler.go
  43. 4
      weed/worker/tasks/task_logger.go
  44. 22
      weed/worker/tasks/vacuum/detection.go
  45. 5
      weed/worker/tasks/vacuum/register.go
  46. 12
      weed/worker/tasks/vacuum/vacuum_task.go
  47. 25
      weed/worker/types/base/task.go
  48. 2
      weed/worker/types/data_types.go
  49. 27
      weed/worker/types/task.go
  50. 1
      weed/worker/types/task_types.go
  51. 2
      weed/worker/types/typed_task_interface.go
  52. 95
      weed/worker/worker.go

265
weed/admin/dash/admin_server.go

@ -5,6 +5,7 @@ import (
"context"
"fmt"
"net/http"
"strconv"
"time"
"github.com/gin-gonic/gin"
@ -878,6 +879,46 @@ func (as *AdminServer) GetMaintenanceTask(c *gin.Context) {
c.JSON(http.StatusOK, task)
}
// GetMaintenanceTaskDetailAPI returns detailed task information via API
func (as *AdminServer) GetMaintenanceTaskDetailAPI(c *gin.Context) {
taskID := c.Param("id")
taskDetail, err := as.GetMaintenanceTaskDetail(taskID)
if err != nil {
c.JSON(http.StatusNotFound, gin.H{"error": "Task detail not found", "details": err.Error()})
return
}
c.JSON(http.StatusOK, taskDetail)
}
// ShowMaintenanceTaskDetail renders the task detail page
func (as *AdminServer) ShowMaintenanceTaskDetail(c *gin.Context) {
username := c.GetString("username")
if username == "" {
username = "admin" // Default fallback
}
taskID := c.Param("id")
taskDetail, err := as.GetMaintenanceTaskDetail(taskID)
if err != nil {
c.HTML(http.StatusNotFound, "error.html", gin.H{
"error": "Task not found",
"details": err.Error(),
})
return
}
// Prepare data for template
data := gin.H{
"username": username,
"task": taskDetail.Task,
"taskDetail": taskDetail,
"title": fmt.Sprintf("Task Detail - %s", taskID),
}
c.HTML(http.StatusOK, "task_detail.html", data)
}
// CancelMaintenanceTask cancels a pending maintenance task
func (as *AdminServer) CancelMaintenanceTask(c *gin.Context) {
taskID := c.Param("id")
@ -1041,27 +1082,65 @@ func (as *AdminServer) getMaintenanceQueueStats() (*maintenance.QueueStats, erro
// getMaintenanceTasks returns all maintenance tasks
func (as *AdminServer) getMaintenanceTasks() ([]*maintenance.MaintenanceTask, error) {
if as.maintenanceManager == nil {
return []*MaintenanceTask{}, nil
return []*maintenance.MaintenanceTask{}, nil
}
// Collect all tasks from memory across all statuses
allTasks := []*maintenance.MaintenanceTask{}
statuses := []maintenance.MaintenanceTaskStatus{
maintenance.TaskStatusPending,
maintenance.TaskStatusAssigned,
maintenance.TaskStatusInProgress,
maintenance.TaskStatusCompleted,
maintenance.TaskStatusFailed,
maintenance.TaskStatusCancelled,
}
for _, status := range statuses {
tasks := as.maintenanceManager.GetTasks(status, "", 0)
allTasks = append(allTasks, tasks...)
}
// Also load any persisted tasks that might not be in memory
if as.configPersistence != nil {
persistedTasks, err := as.configPersistence.LoadAllTaskStates()
if err == nil {
// Add any persisted tasks not already in memory
for _, persistedTask := range persistedTasks {
found := false
for _, memoryTask := range allTasks {
if memoryTask.ID == persistedTask.ID {
found = true
break
}
}
if !found {
allTasks = append(allTasks, persistedTask)
}
}
}
}
return as.maintenanceManager.GetTasks(maintenance.TaskStatusPending, "", 0), nil
return allTasks, nil
}
// getMaintenanceTask returns a specific maintenance task
func (as *AdminServer) getMaintenanceTask(taskID string) (*MaintenanceTask, error) {
func (as *AdminServer) getMaintenanceTask(taskID string) (*maintenance.MaintenanceTask, error) {
if as.maintenanceManager == nil {
return nil, fmt.Errorf("maintenance manager not initialized")
}
// Search for the task across all statuses since we don't know which status it has
statuses := []MaintenanceTaskStatus{
TaskStatusPending,
TaskStatusAssigned,
TaskStatusInProgress,
TaskStatusCompleted,
TaskStatusFailed,
TaskStatusCancelled,
statuses := []maintenance.MaintenanceTaskStatus{
maintenance.TaskStatusPending,
maintenance.TaskStatusAssigned,
maintenance.TaskStatusInProgress,
maintenance.TaskStatusCompleted,
maintenance.TaskStatusFailed,
maintenance.TaskStatusCancelled,
}
// First, search for the task in memory across all statuses
for _, status := range statuses {
tasks := as.maintenanceManager.GetTasks(status, "", 0) // Get all tasks with this status
for _, task := range tasks {
@ -1071,9 +1150,133 @@ func (as *AdminServer) getMaintenanceTask(taskID string) (*MaintenanceTask, erro
}
}
// If not found in memory, try to load from persistent storage
if as.configPersistence != nil {
task, err := as.configPersistence.LoadTaskState(taskID)
if err == nil {
glog.V(2).Infof("Loaded task %s from persistent storage", taskID)
return task, nil
}
glog.V(2).Infof("Task %s not found in persistent storage: %v", taskID, err)
}
return nil, fmt.Errorf("task %s not found", taskID)
}
// GetMaintenanceTaskDetail returns comprehensive task details including logs and assignment history
func (as *AdminServer) GetMaintenanceTaskDetail(taskID string) (*maintenance.TaskDetailData, error) {
// Get basic task information
task, err := as.getMaintenanceTask(taskID)
if err != nil {
return nil, err
}
// Create task detail structure from the loaded task
taskDetail := &maintenance.TaskDetailData{
Task: task,
AssignmentHistory: task.AssignmentHistory, // Use assignment history from persisted task
ExecutionLogs: []*maintenance.TaskExecutionLog{},
RelatedTasks: []*maintenance.MaintenanceTask{},
LastUpdated: time.Now(),
}
if taskDetail.AssignmentHistory == nil {
taskDetail.AssignmentHistory = []*maintenance.TaskAssignmentRecord{}
}
// Get worker information if task is assigned
if task.WorkerID != "" {
workers := as.maintenanceManager.GetWorkers()
for _, worker := range workers {
if worker.ID == task.WorkerID {
taskDetail.WorkerInfo = worker
break
}
}
}
// Get execution logs from worker if task is active/completed and worker is connected
if task.Status == maintenance.TaskStatusInProgress || task.Status == maintenance.TaskStatusCompleted {
if as.workerGrpcServer != nil && task.WorkerID != "" {
workerLogs, err := as.workerGrpcServer.RequestTaskLogs(task.WorkerID, taskID, 100, "")
if err == nil && len(workerLogs) > 0 {
// Convert worker logs to maintenance logs
for _, workerLog := range workerLogs {
maintenanceLog := &maintenance.TaskExecutionLog{
Timestamp: time.Unix(workerLog.Timestamp, 0),
Level: workerLog.Level,
Message: workerLog.Message,
Source: "worker",
TaskID: taskID,
WorkerID: task.WorkerID,
}
// carry structured fields if present
if len(workerLog.Fields) > 0 {
maintenanceLog.Fields = make(map[string]string, len(workerLog.Fields))
for k, v := range workerLog.Fields {
maintenanceLog.Fields[k] = v
}
}
// carry optional progress/status
if workerLog.Progress != 0 {
p := float64(workerLog.Progress)
maintenanceLog.Progress = &p
}
if workerLog.Status != "" {
maintenanceLog.Status = workerLog.Status
}
taskDetail.ExecutionLogs = append(taskDetail.ExecutionLogs, maintenanceLog)
}
} else if err != nil {
// Add a diagnostic log entry when worker logs cannot be retrieved
diagnosticLog := &maintenance.TaskExecutionLog{
Timestamp: time.Now(),
Level: "WARNING",
Message: fmt.Sprintf("Failed to retrieve worker logs: %v", err),
Source: "admin",
TaskID: taskID,
WorkerID: task.WorkerID,
}
taskDetail.ExecutionLogs = append(taskDetail.ExecutionLogs, diagnosticLog)
glog.V(1).Infof("Failed to get worker logs for task %s from worker %s: %v", taskID, task.WorkerID, err)
}
} else {
// Add diagnostic information when worker is not available
reason := "worker gRPC server not available"
if task.WorkerID == "" {
reason = "no worker assigned to task"
}
diagnosticLog := &maintenance.TaskExecutionLog{
Timestamp: time.Now(),
Level: "INFO",
Message: fmt.Sprintf("Worker logs not available: %s", reason),
Source: "admin",
TaskID: taskID,
WorkerID: task.WorkerID,
}
taskDetail.ExecutionLogs = append(taskDetail.ExecutionLogs, diagnosticLog)
}
}
// Get related tasks (other tasks on same volume/server)
if task.VolumeID != 0 || task.Server != "" {
allTasks := as.maintenanceManager.GetTasks("", "", 50) // Get recent tasks
for _, relatedTask := range allTasks {
if relatedTask.ID != taskID &&
(relatedTask.VolumeID == task.VolumeID || relatedTask.Server == task.Server) {
taskDetail.RelatedTasks = append(taskDetail.RelatedTasks, relatedTask)
}
}
}
// Save updated task detail to disk
if err := as.configPersistence.SaveTaskDetail(taskID, taskDetail); err != nil {
glog.V(1).Infof("Failed to save task detail for %s: %v", taskID, err)
}
return taskDetail, nil
}
// getMaintenanceWorkers returns all maintenance workers
func (as *AdminServer) getMaintenanceWorkers() ([]*maintenance.MaintenanceWorker, error) {
if as.maintenanceManager == nil {
@ -1157,6 +1360,34 @@ func (as *AdminServer) getMaintenanceWorkerDetails(workerID string) (*WorkerDeta
}, nil
}
// GetWorkerLogs fetches logs from a specific worker for a task
func (as *AdminServer) GetWorkerLogs(c *gin.Context) {
workerID := c.Param("id")
taskID := c.Query("taskId")
maxEntriesStr := c.DefaultQuery("maxEntries", "100")
logLevel := c.DefaultQuery("logLevel", "")
maxEntries := int32(100)
if maxEntriesStr != "" {
if parsed, err := strconv.ParseInt(maxEntriesStr, 10, 32); err == nil {
maxEntries = int32(parsed)
}
}
if as.workerGrpcServer == nil {
c.JSON(http.StatusServiceUnavailable, gin.H{"error": "Worker gRPC server not available"})
return
}
logs, err := as.workerGrpcServer.RequestTaskLogs(workerID, taskID, maxEntries, logLevel)
if err != nil {
c.JSON(http.StatusBadGateway, gin.H{"error": fmt.Sprintf("Failed to get logs from worker: %v", err)})
return
}
c.JSON(http.StatusOK, gin.H{"worker_id": workerID, "task_id": taskID, "logs": logs, "count": len(logs)})
}
// getMaintenanceStats returns maintenance statistics
func (as *AdminServer) getMaintenanceStats() (*MaintenanceStats, error) {
if as.maintenanceManager == nil {
@ -1376,6 +1607,20 @@ func (s *AdminServer) GetWorkerGrpcServer() *WorkerGrpcServer {
// InitMaintenanceManager initializes the maintenance manager
func (s *AdminServer) InitMaintenanceManager(config *maintenance.MaintenanceConfig) {
s.maintenanceManager = maintenance.NewMaintenanceManager(s, config)
// Set up task persistence if config persistence is available
if s.configPersistence != nil {
queue := s.maintenanceManager.GetQueue()
if queue != nil {
queue.SetPersistence(s.configPersistence)
// Load tasks from persistence on startup
if err := queue.LoadTasksFromPersistence(); err != nil {
glog.Errorf("Failed to load tasks from persistence: %v", err)
}
}
}
glog.V(1).Infof("Maintenance manager initialized (enabled: %v)", config.Enabled)
}

545
weed/admin/dash/config_persistence.go

@ -1,11 +1,15 @@
package dash
import (
"encoding/json"
"fmt"
"os"
"path/filepath"
"sort"
"strings"
"time"
"github.com/seaweedfs/seaweedfs/weed/admin/maintenance"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb/worker_pb"
"github.com/seaweedfs/seaweedfs/weed/worker/tasks/balance"
@ -33,6 +37,12 @@ const (
BalanceTaskConfigJSONFile = "task_balance.json"
ReplicationTaskConfigJSONFile = "task_replication.json"
// Task persistence subdirectories and settings
TasksSubdir = "tasks"
TaskDetailsSubdir = "task_details"
TaskLogsSubdir = "task_logs"
MaxCompletedTasks = 10 // Only keep last 10 completed tasks
ConfigDirPermissions = 0755
ConfigFilePermissions = 0644
)
@ -45,6 +55,35 @@ type (
ReplicationTaskConfig = worker_pb.ReplicationTaskConfig
)
// isValidTaskID validates that a task ID is safe for use in file paths
// This prevents path traversal attacks by ensuring the task ID doesn't contain
// path separators or parent directory references
func isValidTaskID(taskID string) bool {
if taskID == "" {
return false
}
// Reject task IDs with leading or trailing whitespace
if strings.TrimSpace(taskID) != taskID {
return false
}
// Check for path traversal patterns
if strings.Contains(taskID, "/") ||
strings.Contains(taskID, "\\") ||
strings.Contains(taskID, "..") ||
strings.Contains(taskID, ":") {
return false
}
// Additional safety: ensure it's not just dots or empty after trim
if taskID == "." || taskID == ".." {
return false
}
return true
}
// ConfigPersistence handles saving and loading configuration files
type ConfigPersistence struct {
dataDir string
@ -688,3 +727,509 @@ func buildPolicyFromTaskConfigs() *worker_pb.MaintenancePolicy {
glog.V(1).Infof("Built maintenance policy from separate task configs - %d task policies loaded", len(policy.TaskPolicies))
return policy
}
// SaveTaskDetail saves detailed task information to disk
func (cp *ConfigPersistence) SaveTaskDetail(taskID string, detail *maintenance.TaskDetailData) error {
if cp.dataDir == "" {
return fmt.Errorf("no data directory specified, cannot save task detail")
}
// Validate task ID to prevent path traversal
if !isValidTaskID(taskID) {
return fmt.Errorf("invalid task ID: %q contains illegal path characters", taskID)
}
taskDetailDir := filepath.Join(cp.dataDir, TaskDetailsSubdir)
if err := os.MkdirAll(taskDetailDir, ConfigDirPermissions); err != nil {
return fmt.Errorf("failed to create task details directory: %w", err)
}
// Save task detail as JSON for easy reading and debugging
taskDetailPath := filepath.Join(taskDetailDir, fmt.Sprintf("%s.json", taskID))
jsonData, err := json.MarshalIndent(detail, "", " ")
if err != nil {
return fmt.Errorf("failed to marshal task detail to JSON: %w", err)
}
if err := os.WriteFile(taskDetailPath, jsonData, ConfigFilePermissions); err != nil {
return fmt.Errorf("failed to write task detail file: %w", err)
}
glog.V(2).Infof("Saved task detail for task %s to %s", taskID, taskDetailPath)
return nil
}
// LoadTaskDetail loads detailed task information from disk
func (cp *ConfigPersistence) LoadTaskDetail(taskID string) (*maintenance.TaskDetailData, error) {
if cp.dataDir == "" {
return nil, fmt.Errorf("no data directory specified, cannot load task detail")
}
// Validate task ID to prevent path traversal
if !isValidTaskID(taskID) {
return nil, fmt.Errorf("invalid task ID: %q contains illegal path characters", taskID)
}
taskDetailPath := filepath.Join(cp.dataDir, TaskDetailsSubdir, fmt.Sprintf("%s.json", taskID))
if _, err := os.Stat(taskDetailPath); os.IsNotExist(err) {
return nil, fmt.Errorf("task detail file not found: %s", taskID)
}
jsonData, err := os.ReadFile(taskDetailPath)
if err != nil {
return nil, fmt.Errorf("failed to read task detail file: %w", err)
}
var detail maintenance.TaskDetailData
if err := json.Unmarshal(jsonData, &detail); err != nil {
return nil, fmt.Errorf("failed to unmarshal task detail JSON: %w", err)
}
glog.V(2).Infof("Loaded task detail for task %s from %s", taskID, taskDetailPath)
return &detail, nil
}
// SaveTaskExecutionLogs saves execution logs for a task
func (cp *ConfigPersistence) SaveTaskExecutionLogs(taskID string, logs []*maintenance.TaskExecutionLog) error {
if cp.dataDir == "" {
return fmt.Errorf("no data directory specified, cannot save task logs")
}
// Validate task ID to prevent path traversal
if !isValidTaskID(taskID) {
return fmt.Errorf("invalid task ID: %q contains illegal path characters", taskID)
}
taskLogsDir := filepath.Join(cp.dataDir, TaskLogsSubdir)
if err := os.MkdirAll(taskLogsDir, ConfigDirPermissions); err != nil {
return fmt.Errorf("failed to create task logs directory: %w", err)
}
// Save logs as JSON for easy reading
taskLogsPath := filepath.Join(taskLogsDir, fmt.Sprintf("%s.json", taskID))
logsData := struct {
TaskID string `json:"task_id"`
Logs []*maintenance.TaskExecutionLog `json:"logs"`
}{
TaskID: taskID,
Logs: logs,
}
jsonData, err := json.MarshalIndent(logsData, "", " ")
if err != nil {
return fmt.Errorf("failed to marshal task logs to JSON: %w", err)
}
if err := os.WriteFile(taskLogsPath, jsonData, ConfigFilePermissions); err != nil {
return fmt.Errorf("failed to write task logs file: %w", err)
}
glog.V(2).Infof("Saved %d execution logs for task %s to %s", len(logs), taskID, taskLogsPath)
return nil
}
// LoadTaskExecutionLogs loads execution logs for a task
func (cp *ConfigPersistence) LoadTaskExecutionLogs(taskID string) ([]*maintenance.TaskExecutionLog, error) {
if cp.dataDir == "" {
return nil, fmt.Errorf("no data directory specified, cannot load task logs")
}
// Validate task ID to prevent path traversal
if !isValidTaskID(taskID) {
return nil, fmt.Errorf("invalid task ID: %q contains illegal path characters", taskID)
}
taskLogsPath := filepath.Join(cp.dataDir, TaskLogsSubdir, fmt.Sprintf("%s.json", taskID))
if _, err := os.Stat(taskLogsPath); os.IsNotExist(err) {
// Return empty slice if logs don't exist yet
return []*maintenance.TaskExecutionLog{}, nil
}
jsonData, err := os.ReadFile(taskLogsPath)
if err != nil {
return nil, fmt.Errorf("failed to read task logs file: %w", err)
}
var logsData struct {
TaskID string `json:"task_id"`
Logs []*maintenance.TaskExecutionLog `json:"logs"`
}
if err := json.Unmarshal(jsonData, &logsData); err != nil {
return nil, fmt.Errorf("failed to unmarshal task logs JSON: %w", err)
}
glog.V(2).Infof("Loaded %d execution logs for task %s from %s", len(logsData.Logs), taskID, taskLogsPath)
return logsData.Logs, nil
}
// DeleteTaskDetail removes task detail and logs from disk
func (cp *ConfigPersistence) DeleteTaskDetail(taskID string) error {
if cp.dataDir == "" {
return fmt.Errorf("no data directory specified, cannot delete task detail")
}
// Validate task ID to prevent path traversal
if !isValidTaskID(taskID) {
return fmt.Errorf("invalid task ID: %q contains illegal path characters", taskID)
}
// Delete task detail file
taskDetailPath := filepath.Join(cp.dataDir, TaskDetailsSubdir, fmt.Sprintf("%s.json", taskID))
if err := os.Remove(taskDetailPath); err != nil && !os.IsNotExist(err) {
return fmt.Errorf("failed to delete task detail file: %w", err)
}
// Delete task logs file
taskLogsPath := filepath.Join(cp.dataDir, TaskLogsSubdir, fmt.Sprintf("%s.json", taskID))
if err := os.Remove(taskLogsPath); err != nil && !os.IsNotExist(err) {
return fmt.Errorf("failed to delete task logs file: %w", err)
}
glog.V(2).Infof("Deleted task detail and logs for task %s", taskID)
return nil
}
// ListTaskDetails returns a list of all task IDs that have stored details
func (cp *ConfigPersistence) ListTaskDetails() ([]string, error) {
if cp.dataDir == "" {
return nil, fmt.Errorf("no data directory specified, cannot list task details")
}
taskDetailDir := filepath.Join(cp.dataDir, TaskDetailsSubdir)
if _, err := os.Stat(taskDetailDir); os.IsNotExist(err) {
return []string{}, nil
}
entries, err := os.ReadDir(taskDetailDir)
if err != nil {
return nil, fmt.Errorf("failed to read task details directory: %w", err)
}
var taskIDs []string
for _, entry := range entries {
if !entry.IsDir() && filepath.Ext(entry.Name()) == ".json" {
taskID := entry.Name()[:len(entry.Name())-5] // Remove .json extension
taskIDs = append(taskIDs, taskID)
}
}
return taskIDs, nil
}
// CleanupCompletedTasks removes old completed tasks beyond the retention limit
func (cp *ConfigPersistence) CleanupCompletedTasks() error {
if cp.dataDir == "" {
return fmt.Errorf("no data directory specified, cannot cleanup completed tasks")
}
tasksDir := filepath.Join(cp.dataDir, TasksSubdir)
if _, err := os.Stat(tasksDir); os.IsNotExist(err) {
return nil // No tasks directory, nothing to cleanup
}
// Load all tasks and find completed/failed ones
allTasks, err := cp.LoadAllTaskStates()
if err != nil {
return fmt.Errorf("failed to load tasks for cleanup: %w", err)
}
// Filter completed and failed tasks, sort by completion time
var completedTasks []*maintenance.MaintenanceTask
for _, task := range allTasks {
if (task.Status == maintenance.TaskStatusCompleted || task.Status == maintenance.TaskStatusFailed) && task.CompletedAt != nil {
completedTasks = append(completedTasks, task)
}
}
// Sort by completion time (most recent first)
sort.Slice(completedTasks, func(i, j int) bool {
return completedTasks[i].CompletedAt.After(*completedTasks[j].CompletedAt)
})
// Keep only the most recent MaxCompletedTasks, delete the rest
if len(completedTasks) > MaxCompletedTasks {
tasksToDelete := completedTasks[MaxCompletedTasks:]
for _, task := range tasksToDelete {
if err := cp.DeleteTaskState(task.ID); err != nil {
glog.Warningf("Failed to delete old completed task %s: %v", task.ID, err)
} else {
glog.V(2).Infof("Cleaned up old completed task %s (completed: %v)", task.ID, task.CompletedAt)
}
}
glog.V(1).Infof("Cleaned up %d old completed tasks (keeping %d most recent)", len(tasksToDelete), MaxCompletedTasks)
}
return nil
}
// SaveTaskState saves a task state to protobuf file
func (cp *ConfigPersistence) SaveTaskState(task *maintenance.MaintenanceTask) error {
if cp.dataDir == "" {
return fmt.Errorf("no data directory specified, cannot save task state")
}
// Validate task ID to prevent path traversal
if !isValidTaskID(task.ID) {
return fmt.Errorf("invalid task ID: %q contains illegal path characters", task.ID)
}
tasksDir := filepath.Join(cp.dataDir, TasksSubdir)
if err := os.MkdirAll(tasksDir, ConfigDirPermissions); err != nil {
return fmt.Errorf("failed to create tasks directory: %w", err)
}
taskFilePath := filepath.Join(tasksDir, fmt.Sprintf("%s.pb", task.ID))
// Convert task to protobuf
pbTask := cp.maintenanceTaskToProtobuf(task)
taskStateFile := &worker_pb.TaskStateFile{
Task: pbTask,
LastUpdated: time.Now().Unix(),
AdminVersion: "unknown", // TODO: add version info
}
pbData, err := proto.Marshal(taskStateFile)
if err != nil {
return fmt.Errorf("failed to marshal task state protobuf: %w", err)
}
if err := os.WriteFile(taskFilePath, pbData, ConfigFilePermissions); err != nil {
return fmt.Errorf("failed to write task state file: %w", err)
}
glog.V(2).Infof("Saved task state for task %s to %s", task.ID, taskFilePath)
return nil
}
// LoadTaskState loads a task state from protobuf file
func (cp *ConfigPersistence) LoadTaskState(taskID string) (*maintenance.MaintenanceTask, error) {
if cp.dataDir == "" {
return nil, fmt.Errorf("no data directory specified, cannot load task state")
}
// Validate task ID to prevent path traversal
if !isValidTaskID(taskID) {
return nil, fmt.Errorf("invalid task ID: %q contains illegal path characters", taskID)
}
taskFilePath := filepath.Join(cp.dataDir, TasksSubdir, fmt.Sprintf("%s.pb", taskID))
if _, err := os.Stat(taskFilePath); os.IsNotExist(err) {
return nil, fmt.Errorf("task state file not found: %s", taskID)
}
pbData, err := os.ReadFile(taskFilePath)
if err != nil {
return nil, fmt.Errorf("failed to read task state file: %w", err)
}
var taskStateFile worker_pb.TaskStateFile
if err := proto.Unmarshal(pbData, &taskStateFile); err != nil {
return nil, fmt.Errorf("failed to unmarshal task state protobuf: %w", err)
}
// Convert protobuf to maintenance task
task := cp.protobufToMaintenanceTask(taskStateFile.Task)
glog.V(2).Infof("Loaded task state for task %s from %s", taskID, taskFilePath)
return task, nil
}
// LoadAllTaskStates loads all task states from disk
func (cp *ConfigPersistence) LoadAllTaskStates() ([]*maintenance.MaintenanceTask, error) {
if cp.dataDir == "" {
return []*maintenance.MaintenanceTask{}, nil
}
tasksDir := filepath.Join(cp.dataDir, TasksSubdir)
if _, err := os.Stat(tasksDir); os.IsNotExist(err) {
return []*maintenance.MaintenanceTask{}, nil
}
entries, err := os.ReadDir(tasksDir)
if err != nil {
return nil, fmt.Errorf("failed to read tasks directory: %w", err)
}
var tasks []*maintenance.MaintenanceTask
for _, entry := range entries {
if !entry.IsDir() && filepath.Ext(entry.Name()) == ".pb" {
taskID := entry.Name()[:len(entry.Name())-3] // Remove .pb extension
task, err := cp.LoadTaskState(taskID)
if err != nil {
glog.Warningf("Failed to load task state for %s: %v", taskID, err)
continue
}
tasks = append(tasks, task)
}
}
glog.V(1).Infof("Loaded %d task states from disk", len(tasks))
return tasks, nil
}
// DeleteTaskState removes a task state file from disk
func (cp *ConfigPersistence) DeleteTaskState(taskID string) error {
if cp.dataDir == "" {
return fmt.Errorf("no data directory specified, cannot delete task state")
}
// Validate task ID to prevent path traversal
if !isValidTaskID(taskID) {
return fmt.Errorf("invalid task ID: %q contains illegal path characters", taskID)
}
taskFilePath := filepath.Join(cp.dataDir, TasksSubdir, fmt.Sprintf("%s.pb", taskID))
if err := os.Remove(taskFilePath); err != nil && !os.IsNotExist(err) {
return fmt.Errorf("failed to delete task state file: %w", err)
}
glog.V(2).Infof("Deleted task state for task %s", taskID)
return nil
}
// maintenanceTaskToProtobuf converts a MaintenanceTask to protobuf format
func (cp *ConfigPersistence) maintenanceTaskToProtobuf(task *maintenance.MaintenanceTask) *worker_pb.MaintenanceTaskData {
pbTask := &worker_pb.MaintenanceTaskData{
Id: task.ID,
Type: string(task.Type),
Priority: cp.priorityToString(task.Priority),
Status: string(task.Status),
VolumeId: task.VolumeID,
Server: task.Server,
Collection: task.Collection,
Reason: task.Reason,
CreatedAt: task.CreatedAt.Unix(),
ScheduledAt: task.ScheduledAt.Unix(),
WorkerId: task.WorkerID,
Error: task.Error,
Progress: task.Progress,
RetryCount: int32(task.RetryCount),
MaxRetries: int32(task.MaxRetries),
CreatedBy: task.CreatedBy,
CreationContext: task.CreationContext,
DetailedReason: task.DetailedReason,
Tags: task.Tags,
}
// Handle optional timestamps
if task.StartedAt != nil {
pbTask.StartedAt = task.StartedAt.Unix()
}
if task.CompletedAt != nil {
pbTask.CompletedAt = task.CompletedAt.Unix()
}
// Convert assignment history
if task.AssignmentHistory != nil {
for _, record := range task.AssignmentHistory {
pbRecord := &worker_pb.TaskAssignmentRecord{
WorkerId: record.WorkerID,
WorkerAddress: record.WorkerAddress,
AssignedAt: record.AssignedAt.Unix(),
Reason: record.Reason,
}
if record.UnassignedAt != nil {
pbRecord.UnassignedAt = record.UnassignedAt.Unix()
}
pbTask.AssignmentHistory = append(pbTask.AssignmentHistory, pbRecord)
}
}
// Convert typed parameters if available
if task.TypedParams != nil {
pbTask.TypedParams = task.TypedParams
}
return pbTask
}
// protobufToMaintenanceTask converts protobuf format to MaintenanceTask
func (cp *ConfigPersistence) protobufToMaintenanceTask(pbTask *worker_pb.MaintenanceTaskData) *maintenance.MaintenanceTask {
task := &maintenance.MaintenanceTask{
ID: pbTask.Id,
Type: maintenance.MaintenanceTaskType(pbTask.Type),
Priority: cp.stringToPriority(pbTask.Priority),
Status: maintenance.MaintenanceTaskStatus(pbTask.Status),
VolumeID: pbTask.VolumeId,
Server: pbTask.Server,
Collection: pbTask.Collection,
Reason: pbTask.Reason,
CreatedAt: time.Unix(pbTask.CreatedAt, 0),
ScheduledAt: time.Unix(pbTask.ScheduledAt, 0),
WorkerID: pbTask.WorkerId,
Error: pbTask.Error,
Progress: pbTask.Progress,
RetryCount: int(pbTask.RetryCount),
MaxRetries: int(pbTask.MaxRetries),
CreatedBy: pbTask.CreatedBy,
CreationContext: pbTask.CreationContext,
DetailedReason: pbTask.DetailedReason,
Tags: pbTask.Tags,
}
// Handle optional timestamps
if pbTask.StartedAt > 0 {
startTime := time.Unix(pbTask.StartedAt, 0)
task.StartedAt = &startTime
}
if pbTask.CompletedAt > 0 {
completedTime := time.Unix(pbTask.CompletedAt, 0)
task.CompletedAt = &completedTime
}
// Convert assignment history
if pbTask.AssignmentHistory != nil {
task.AssignmentHistory = make([]*maintenance.TaskAssignmentRecord, 0, len(pbTask.AssignmentHistory))
for _, pbRecord := range pbTask.AssignmentHistory {
record := &maintenance.TaskAssignmentRecord{
WorkerID: pbRecord.WorkerId,
WorkerAddress: pbRecord.WorkerAddress,
AssignedAt: time.Unix(pbRecord.AssignedAt, 0),
Reason: pbRecord.Reason,
}
if pbRecord.UnassignedAt > 0 {
unassignedTime := time.Unix(pbRecord.UnassignedAt, 0)
record.UnassignedAt = &unassignedTime
}
task.AssignmentHistory = append(task.AssignmentHistory, record)
}
}
// Convert typed parameters if available
if pbTask.TypedParams != nil {
task.TypedParams = pbTask.TypedParams
}
return task
}
// priorityToString converts MaintenanceTaskPriority to string for protobuf storage
func (cp *ConfigPersistence) priorityToString(priority maintenance.MaintenanceTaskPriority) string {
switch priority {
case maintenance.PriorityLow:
return "low"
case maintenance.PriorityNormal:
return "normal"
case maintenance.PriorityHigh:
return "high"
case maintenance.PriorityCritical:
return "critical"
default:
return "normal"
}
}
// stringToPriority converts string from protobuf to MaintenanceTaskPriority
func (cp *ConfigPersistence) stringToPriority(priorityStr string) maintenance.MaintenanceTaskPriority {
switch priorityStr {
case "low":
return maintenance.PriorityLow
case "normal":
return maintenance.PriorityNormal
case "high":
return maintenance.PriorityHigh
case "critical":
return maintenance.PriorityCritical
default:
return maintenance.PriorityNormal
}
}

13
weed/admin/dash/ec_shard_management.go

@ -13,6 +13,17 @@ import (
"github.com/seaweedfs/seaweedfs/weed/storage/erasure_coding"
)
// matchesCollection checks if a volume/EC volume collection matches the filter collection.
// Handles the special case where empty collection ("") represents the "default" collection.
func matchesCollection(volumeCollection, filterCollection string) bool {
// Both empty means default collection matches default filter
if volumeCollection == "" && filterCollection == "" {
return true
}
// Direct string match for named collections
return volumeCollection == filterCollection
}
// GetClusterEcShards retrieves cluster EC shards data with pagination, sorting, and filtering
func (s *AdminServer) GetClusterEcShards(page int, pageSize int, sortBy string, sortOrder string, collection string) (*ClusterEcShardsData, error) {
// Set defaults
@ -403,7 +414,7 @@ func (s *AdminServer) GetClusterEcVolumes(page int, pageSize int, sortBy string,
var ecVolumes []EcVolumeWithShards
for _, volume := range volumeData {
// Filter by collection if specified
if collection == "" || volume.Collection == collection {
if collection == "" || matchesCollection(volume.Collection, collection) {
ecVolumes = append(ecVolumes, *volume)
}
}

18
weed/admin/dash/volume_management.go

@ -83,13 +83,7 @@ func (s *AdminServer) GetClusterVolumes(page int, pageSize int, sortBy string, s
var filteredEcTotalSize int64
for _, volume := range volumes {
// Handle "default" collection filtering for empty collections
volumeCollection := volume.Collection
if volumeCollection == "" {
volumeCollection = "default"
}
if volumeCollection == collection {
if matchesCollection(volume.Collection, collection) {
filteredVolumes = append(filteredVolumes, volume)
filteredTotalSize += int64(volume.Size)
}
@ -103,13 +97,7 @@ func (s *AdminServer) GetClusterVolumes(page int, pageSize int, sortBy string, s
for _, node := range rack.DataNodeInfos {
for _, diskInfo := range node.DiskInfos {
for _, ecShardInfo := range diskInfo.EcShardInfos {
// Handle "default" collection filtering for empty collections
ecCollection := ecShardInfo.Collection
if ecCollection == "" {
ecCollection = "default"
}
if ecCollection == collection {
if matchesCollection(ecShardInfo.Collection, collection) {
// Add all shard sizes for this EC volume
for _, shardSize := range ecShardInfo.ShardSizes {
filteredEcTotalSize += shardSize
@ -500,7 +488,7 @@ func (s *AdminServer) GetClusterVolumeServers() (*ClusterVolumeServersData, erro
ecInfo.EcIndexBits |= ecShardInfo.EcIndexBits
// Collect shard sizes from this disk
shardBits := erasure_coding.ShardBits(ecShardInfo.EcIndexBits)
shardBits := erasure_coding.ShardBits(ecShardInfo.EcIndexBits)
shardBits.EachSetIndex(func(shardId erasure_coding.ShardId) {
if size, found := erasure_coding.GetShardSize(ecShardInfo, shardId); found {
allShardSizes[shardId] = size

164
weed/admin/dash/worker_grpc_server.go

@ -26,6 +26,10 @@ type WorkerGrpcServer struct {
connections map[string]*WorkerConnection
connMutex sync.RWMutex
// Log request correlation
pendingLogRequests map[string]*LogRequestContext
logRequestsMutex sync.RWMutex
// gRPC server
grpcServer *grpc.Server
listener net.Listener
@ -33,6 +37,14 @@ type WorkerGrpcServer struct {
stopChan chan struct{}
}
// LogRequestContext tracks pending log requests
type LogRequestContext struct {
TaskID string
WorkerID string
ResponseCh chan *worker_pb.TaskLogResponse
Timeout time.Time
}
// WorkerConnection represents an active worker connection
type WorkerConnection struct {
workerID string
@ -49,9 +61,10 @@ type WorkerConnection struct {
// 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{}),
adminServer: adminServer,
connections: make(map[string]*WorkerConnection),
pendingLogRequests: make(map[string]*LogRequestContext),
stopChan: make(chan struct{}),
}
}
@ -264,6 +277,9 @@ func (s *WorkerGrpcServer) handleWorkerMessage(conn *WorkerConnection, msg *work
case *worker_pb.WorkerMessage_TaskComplete:
s.handleTaskCompletion(conn, m.TaskComplete)
case *worker_pb.WorkerMessage_TaskLogResponse:
s.handleTaskLogResponse(conn, m.TaskLogResponse)
case *worker_pb.WorkerMessage_Shutdown:
glog.Infof("Worker %s shutting down: %s", workerID, m.Shutdown.Reason)
s.unregisterWorker(workerID)
@ -341,8 +357,13 @@ func (s *WorkerGrpcServer) handleTaskRequest(conn *WorkerConnection, request *wo
// Create basic params if none exist
taskParams = &worker_pb.TaskParams{
VolumeId: task.VolumeID,
Server: task.Server,
Collection: task.Collection,
Sources: []*worker_pb.TaskSource{
{
Node: task.Server,
VolumeId: task.VolumeID,
},
},
}
}
@ -396,6 +417,35 @@ func (s *WorkerGrpcServer) handleTaskCompletion(conn *WorkerConnection, completi
}
}
// handleTaskLogResponse processes task log responses from workers
func (s *WorkerGrpcServer) handleTaskLogResponse(conn *WorkerConnection, response *worker_pb.TaskLogResponse) {
requestKey := fmt.Sprintf("%s:%s", response.WorkerId, response.TaskId)
s.logRequestsMutex.RLock()
requestContext, exists := s.pendingLogRequests[requestKey]
s.logRequestsMutex.RUnlock()
if !exists {
glog.Warningf("Received unexpected log response for task %s from worker %s", response.TaskId, response.WorkerId)
return
}
glog.V(1).Infof("Received log response for task %s from worker %s: %d entries", response.TaskId, response.WorkerId, len(response.LogEntries))
// Send response to waiting channel
select {
case requestContext.ResponseCh <- response:
// Response delivered successfully
case <-time.After(time.Second):
glog.Warningf("Failed to deliver log response for task %s from worker %s: timeout", response.TaskId, response.WorkerId)
}
// Clean up the pending request
s.logRequestsMutex.Lock()
delete(s.pendingLogRequests, requestKey)
s.logRequestsMutex.Unlock()
}
// unregisterWorker removes a worker connection
func (s *WorkerGrpcServer) unregisterWorker(workerID string) {
s.connMutex.Lock()
@ -453,6 +503,112 @@ func (s *WorkerGrpcServer) GetConnectedWorkers() []string {
return workers
}
// RequestTaskLogs requests execution logs from a worker for a specific task
func (s *WorkerGrpcServer) RequestTaskLogs(workerID, taskID string, maxEntries int32, logLevel string) ([]*worker_pb.TaskLogEntry, error) {
s.connMutex.RLock()
conn, exists := s.connections[workerID]
s.connMutex.RUnlock()
if !exists {
return nil, fmt.Errorf("worker %s is not connected", workerID)
}
// Create response channel for this request
responseCh := make(chan *worker_pb.TaskLogResponse, 1)
requestKey := fmt.Sprintf("%s:%s", workerID, taskID)
// Register pending request
requestContext := &LogRequestContext{
TaskID: taskID,
WorkerID: workerID,
ResponseCh: responseCh,
Timeout: time.Now().Add(10 * time.Second),
}
s.logRequestsMutex.Lock()
s.pendingLogRequests[requestKey] = requestContext
s.logRequestsMutex.Unlock()
// Create log request message
logRequest := &worker_pb.AdminMessage{
AdminId: "admin-server",
Timestamp: time.Now().Unix(),
Message: &worker_pb.AdminMessage_TaskLogRequest{
TaskLogRequest: &worker_pb.TaskLogRequest{
TaskId: taskID,
WorkerId: workerID,
IncludeMetadata: true,
MaxEntries: maxEntries,
LogLevel: logLevel,
},
},
}
// Send the request through the worker's outgoing channel
select {
case conn.outgoing <- logRequest:
glog.V(1).Infof("Log request sent to worker %s for task %s", workerID, taskID)
case <-time.After(5 * time.Second):
// Clean up pending request on timeout
s.logRequestsMutex.Lock()
delete(s.pendingLogRequests, requestKey)
s.logRequestsMutex.Unlock()
return nil, fmt.Errorf("timeout sending log request to worker %s", workerID)
}
// Wait for response
select {
case response := <-responseCh:
if !response.Success {
return nil, fmt.Errorf("worker log request failed: %s", response.ErrorMessage)
}
glog.V(1).Infof("Received %d log entries for task %s from worker %s", len(response.LogEntries), taskID, workerID)
return response.LogEntries, nil
case <-time.After(10 * time.Second):
// Clean up pending request on timeout
s.logRequestsMutex.Lock()
delete(s.pendingLogRequests, requestKey)
s.logRequestsMutex.Unlock()
return nil, fmt.Errorf("timeout waiting for log response from worker %s", workerID)
}
}
// RequestTaskLogsFromAllWorkers requests logs for a task from all connected workers
func (s *WorkerGrpcServer) RequestTaskLogsFromAllWorkers(taskID string, maxEntries int32, logLevel string) (map[string][]*worker_pb.TaskLogEntry, error) {
s.connMutex.RLock()
workerIDs := make([]string, 0, len(s.connections))
for workerID := range s.connections {
workerIDs = append(workerIDs, workerID)
}
s.connMutex.RUnlock()
results := make(map[string][]*worker_pb.TaskLogEntry)
for _, workerID := range workerIDs {
logs, err := s.RequestTaskLogs(workerID, taskID, maxEntries, logLevel)
if err != nil {
glog.V(1).Infof("Failed to get logs from worker %s for task %s: %v", workerID, taskID, err)
// Store empty result with error information for debugging
results[workerID+"_error"] = []*worker_pb.TaskLogEntry{
{
Timestamp: time.Now().Unix(),
Level: "ERROR",
Message: fmt.Sprintf("Failed to retrieve logs from worker %s: %v", workerID, err),
Fields: map[string]string{"source": "admin"},
},
}
continue
}
if len(logs) > 0 {
results[workerID] = logs
} else {
glog.V(2).Infof("No logs found for task %s on worker %s", taskID, workerID)
}
}
return results, nil
}
// convertTaskParameters converts task parameters to protobuf format
func convertTaskParameters(params map[string]interface{}) map[string]string {
result := make(map[string]string)

6
weed/admin/handlers/admin_handlers.go

@ -94,6 +94,7 @@ func (h *AdminHandlers) SetupRoutes(r *gin.Engine, authRequired bool, username,
protected.POST("/maintenance/config", h.maintenanceHandlers.UpdateMaintenanceConfig)
protected.GET("/maintenance/config/:taskType", h.maintenanceHandlers.ShowTaskConfig)
protected.POST("/maintenance/config/:taskType", h.maintenanceHandlers.UpdateTaskConfig)
protected.GET("/maintenance/tasks/:id", h.maintenanceHandlers.ShowTaskDetail)
// API routes for AJAX calls
api := r.Group("/api")
@ -164,9 +165,11 @@ func (h *AdminHandlers) SetupRoutes(r *gin.Engine, authRequired bool, username,
maintenanceApi.POST("/scan", h.adminServer.TriggerMaintenanceScan)
maintenanceApi.GET("/tasks", h.adminServer.GetMaintenanceTasks)
maintenanceApi.GET("/tasks/:id", h.adminServer.GetMaintenanceTask)
maintenanceApi.GET("/tasks/:id/detail", h.adminServer.GetMaintenanceTaskDetailAPI)
maintenanceApi.POST("/tasks/:id/cancel", h.adminServer.CancelMaintenanceTask)
maintenanceApi.GET("/workers", h.adminServer.GetMaintenanceWorkersAPI)
maintenanceApi.GET("/workers/:id", h.adminServer.GetMaintenanceWorker)
maintenanceApi.GET("/workers/:id/logs", h.adminServer.GetWorkerLogs)
maintenanceApi.GET("/stats", h.adminServer.GetMaintenanceStats)
maintenanceApi.GET("/config", h.adminServer.GetMaintenanceConfigAPI)
maintenanceApi.PUT("/config", h.adminServer.UpdateMaintenanceConfigAPI)
@ -218,6 +221,7 @@ func (h *AdminHandlers) SetupRoutes(r *gin.Engine, authRequired bool, username,
r.POST("/maintenance/config", h.maintenanceHandlers.UpdateMaintenanceConfig)
r.GET("/maintenance/config/:taskType", h.maintenanceHandlers.ShowTaskConfig)
r.POST("/maintenance/config/:taskType", h.maintenanceHandlers.UpdateTaskConfig)
r.GET("/maintenance/tasks/:id", h.maintenanceHandlers.ShowTaskDetail)
// API routes for AJAX calls
api := r.Group("/api")
@ -287,9 +291,11 @@ func (h *AdminHandlers) SetupRoutes(r *gin.Engine, authRequired bool, username,
maintenanceApi.POST("/scan", h.adminServer.TriggerMaintenanceScan)
maintenanceApi.GET("/tasks", h.adminServer.GetMaintenanceTasks)
maintenanceApi.GET("/tasks/:id", h.adminServer.GetMaintenanceTask)
maintenanceApi.GET("/tasks/:id/detail", h.adminServer.GetMaintenanceTaskDetailAPI)
maintenanceApi.POST("/tasks/:id/cancel", h.adminServer.CancelMaintenanceTask)
maintenanceApi.GET("/workers", h.adminServer.GetMaintenanceWorkersAPI)
maintenanceApi.GET("/workers/:id", h.adminServer.GetMaintenanceWorker)
maintenanceApi.GET("/workers/:id/logs", h.adminServer.GetWorkerLogs)
maintenanceApi.GET("/stats", h.adminServer.GetMaintenanceStats)
maintenanceApi.GET("/config", h.adminServer.GetMaintenanceConfigAPI)
maintenanceApi.PUT("/config", h.adminServer.UpdateMaintenanceConfigAPI)

8
weed/admin/handlers/cluster_handlers.go

@ -169,6 +169,12 @@ func (h *ClusterHandlers) ShowCollectionDetails(c *gin.Context) {
return
}
// Map "default" collection to empty string for backend filtering
actualCollectionName := collectionName
if collectionName == "default" {
actualCollectionName = ""
}
// Parse query parameters
page, _ := strconv.Atoi(c.DefaultQuery("page", "1"))
pageSize, _ := strconv.Atoi(c.DefaultQuery("page_size", "25"))
@ -176,7 +182,7 @@ func (h *ClusterHandlers) ShowCollectionDetails(c *gin.Context) {
sortOrder := c.DefaultQuery("sort_order", "asc")
// Get collection details data (volumes and EC volumes)
collectionDetailsData, err := h.adminServer.GetCollectionDetails(collectionName, page, pageSize, sortBy, sortOrder)
collectionDetailsData, err := h.adminServer.GetCollectionDetails(actualCollectionName, page, pageSize, sortBy, sortOrder)
if err != nil {
c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to get collection details: " + err.Error()})
return

91
weed/admin/handlers/maintenance_handlers.go

@ -1,6 +1,7 @@
package handlers
import (
"context"
"fmt"
"net/http"
"reflect"
@ -34,35 +35,82 @@ func NewMaintenanceHandlers(adminServer *dash.AdminServer) *MaintenanceHandlers
}
}
// ShowMaintenanceQueue displays the maintenance queue page
func (h *MaintenanceHandlers) ShowMaintenanceQueue(c *gin.Context) {
data, err := h.getMaintenanceQueueData()
// ShowTaskDetail displays the task detail page
func (h *MaintenanceHandlers) ShowTaskDetail(c *gin.Context) {
taskID := c.Param("id")
glog.Infof("DEBUG ShowTaskDetail: Starting for task ID: %s", taskID)
taskDetail, err := h.adminServer.GetMaintenanceTaskDetail(taskID)
if err != nil {
glog.Infof("DEBUG ShowMaintenanceQueue: error getting data: %v", err)
c.JSON(http.StatusInternalServerError, gin.H{"error": err.Error()})
glog.Errorf("DEBUG ShowTaskDetail: error getting task detail for %s: %v", taskID, err)
c.String(http.StatusNotFound, "Task not found: %s (Error: %v)", taskID, err)
return
}
glog.Infof("DEBUG ShowMaintenanceQueue: got data with %d tasks", len(data.Tasks))
if data.Stats != nil {
glog.Infof("DEBUG ShowMaintenanceQueue: stats = {pending: %d, running: %d, completed: %d}",
data.Stats.PendingTasks, data.Stats.RunningTasks, data.Stats.CompletedToday)
} else {
glog.Infof("DEBUG ShowMaintenanceQueue: stats is nil")
}
glog.Infof("DEBUG ShowTaskDetail: got task detail for %s, task type: %s, status: %s", taskID, taskDetail.Task.Type, taskDetail.Task.Status)
// Render HTML template
c.Header("Content-Type", "text/html")
maintenanceComponent := app.MaintenanceQueue(data)
layoutComponent := layout.Layout(c, maintenanceComponent)
taskDetailComponent := app.TaskDetail(taskDetail)
layoutComponent := layout.Layout(c, taskDetailComponent)
err = layoutComponent.Render(c.Request.Context(), c.Writer)
if err != nil {
glog.Infof("DEBUG ShowMaintenanceQueue: render error: %v", err)
c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to render template: " + err.Error()})
glog.Errorf("DEBUG ShowTaskDetail: render error: %v", err)
c.String(http.StatusInternalServerError, "Failed to render template: %v", err)
return
}
glog.Infof("DEBUG ShowMaintenanceQueue: template rendered successfully")
glog.Infof("DEBUG ShowTaskDetail: template rendered successfully for task %s", taskID)
}
// ShowMaintenanceQueue displays the maintenance queue page
func (h *MaintenanceHandlers) ShowMaintenanceQueue(c *gin.Context) {
// Add timeout to prevent hanging
ctx, cancel := context.WithTimeout(c.Request.Context(), 30*time.Second)
defer cancel()
// Use a channel to handle timeout for data retrieval
type result struct {
data *maintenance.MaintenanceQueueData
err error
}
resultChan := make(chan result, 1)
go func() {
data, err := h.getMaintenanceQueueData()
resultChan <- result{data: data, err: err}
}()
select {
case res := <-resultChan:
if res.err != nil {
glog.V(1).Infof("ShowMaintenanceQueue: error getting data: %v", res.err)
c.JSON(http.StatusInternalServerError, gin.H{"error": res.err.Error()})
return
}
glog.V(2).Infof("ShowMaintenanceQueue: got data with %d tasks", len(res.data.Tasks))
// Render HTML template
c.Header("Content-Type", "text/html")
maintenanceComponent := app.MaintenanceQueue(res.data)
layoutComponent := layout.Layout(c, maintenanceComponent)
err := layoutComponent.Render(ctx, c.Writer)
if err != nil {
glog.V(1).Infof("ShowMaintenanceQueue: render error: %v", err)
c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to render template: " + err.Error()})
return
}
glog.V(3).Infof("ShowMaintenanceQueue: template rendered successfully")
case <-ctx.Done():
glog.Warningf("ShowMaintenanceQueue: timeout waiting for data")
c.JSON(http.StatusRequestTimeout, gin.H{
"error": "Request timeout - maintenance data retrieval took too long. This may indicate a system issue.",
"suggestion": "Try refreshing the page or contact system administrator if the problem persists.",
})
return
}
}
// ShowMaintenanceWorkers displays the maintenance workers page
@ -479,7 +527,7 @@ func (h *MaintenanceHandlers) getMaintenanceQueueStats() (*maintenance.QueueStat
}
func (h *MaintenanceHandlers) getMaintenanceTasks() ([]*maintenance.MaintenanceTask, error) {
// Call the maintenance manager directly to get all tasks
// Call the maintenance manager directly to get recent tasks (limit for performance)
if h.adminServer == nil {
return []*maintenance.MaintenanceTask{}, nil
}
@ -489,8 +537,9 @@ func (h *MaintenanceHandlers) getMaintenanceTasks() ([]*maintenance.MaintenanceT
return []*maintenance.MaintenanceTask{}, nil
}
// Get ALL tasks using empty parameters - this should match what the API returns
allTasks := manager.GetTasks("", "", 0)
// Get recent tasks only (last 100) to prevent slow page loads
// Users can view more tasks via pagination if needed
allTasks := manager.GetTasks("", "", 100)
return allTasks, nil
}

236
weed/admin/maintenance/maintenance_queue.go

@ -7,7 +7,6 @@ import (
"time"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb/worker_pb"
)
// NewMaintenanceQueue creates a new maintenance queue
@ -27,6 +26,102 @@ func (mq *MaintenanceQueue) SetIntegration(integration *MaintenanceIntegration)
glog.V(1).Infof("Maintenance queue configured with integration")
}
// SetPersistence sets the task persistence interface
func (mq *MaintenanceQueue) SetPersistence(persistence TaskPersistence) {
mq.persistence = persistence
glog.V(1).Infof("Maintenance queue configured with task persistence")
}
// LoadTasksFromPersistence loads tasks from persistent storage on startup
func (mq *MaintenanceQueue) LoadTasksFromPersistence() error {
if mq.persistence == nil {
glog.V(1).Infof("No task persistence configured, skipping task loading")
return nil
}
mq.mutex.Lock()
defer mq.mutex.Unlock()
glog.Infof("Loading tasks from persistence...")
tasks, err := mq.persistence.LoadAllTaskStates()
if err != nil {
return fmt.Errorf("failed to load task states: %w", err)
}
glog.Infof("DEBUG LoadTasksFromPersistence: Found %d tasks in persistence", len(tasks))
// Reset task maps
mq.tasks = make(map[string]*MaintenanceTask)
mq.pendingTasks = make([]*MaintenanceTask, 0)
// Load tasks by status
for _, task := range tasks {
glog.Infof("DEBUG LoadTasksFromPersistence: Loading task %s (type: %s, status: %s, scheduled: %v)", task.ID, task.Type, task.Status, task.ScheduledAt)
mq.tasks[task.ID] = task
switch task.Status {
case TaskStatusPending:
glog.Infof("DEBUG LoadTasksFromPersistence: Adding task %s to pending queue", task.ID)
mq.pendingTasks = append(mq.pendingTasks, task)
case TaskStatusAssigned, TaskStatusInProgress:
// For assigned/in-progress tasks, we need to check if the worker is still available
// If not, we should fail them and make them eligible for retry
if task.WorkerID != "" {
if _, exists := mq.workers[task.WorkerID]; !exists {
glog.Warningf("Task %s was assigned to unavailable worker %s, marking as failed", task.ID, task.WorkerID)
task.Status = TaskStatusFailed
task.Error = "Worker unavailable after restart"
completedTime := time.Now()
task.CompletedAt = &completedTime
// Check if it 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(1 * time.Minute) // Retry after restart delay
glog.Infof("DEBUG LoadTasksFromPersistence: Retrying task %s, adding to pending queue", task.ID)
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.Infof("Loaded %d tasks from persistence (%d pending)", len(tasks), len(mq.pendingTasks))
return nil
}
// saveTaskState saves a task to persistent storage
func (mq *MaintenanceQueue) saveTaskState(task *MaintenanceTask) {
if mq.persistence != nil {
if err := mq.persistence.SaveTaskState(task); err != nil {
glog.Errorf("Failed to save task state for %s: %v", task.ID, err)
}
}
}
// cleanupCompletedTasks removes old completed tasks beyond the retention limit
func (mq *MaintenanceQueue) cleanupCompletedTasks() {
if mq.persistence != nil {
if err := mq.persistence.CleanupCompletedTasks(); err != nil {
glog.Errorf("Failed to cleanup completed tasks: %v", err)
}
}
}
// AddTask adds a new maintenance task to the queue with deduplication
func (mq *MaintenanceQueue) AddTask(task *MaintenanceTask) {
mq.mutex.Lock()
@ -44,6 +139,18 @@ func (mq *MaintenanceQueue) AddTask(task *MaintenanceTask) {
task.CreatedAt = time.Now()
task.MaxRetries = 3 // Default retry count
// Initialize assignment history and set creation context
task.AssignmentHistory = make([]*TaskAssignmentRecord, 0)
if task.CreatedBy == "" {
task.CreatedBy = "maintenance-system"
}
if task.CreationContext == "" {
task.CreationContext = "Automatic task creation based on system monitoring"
}
if task.Tags == nil {
task.Tags = make(map[string]string)
}
mq.tasks[task.ID] = task
mq.pendingTasks = append(mq.pendingTasks, task)
@ -55,6 +162,9 @@ func (mq *MaintenanceQueue) AddTask(task *MaintenanceTask) {
return mq.pendingTasks[i].ScheduledAt.Before(mq.pendingTasks[j].ScheduledAt)
})
// Save task state to persistence
mq.saveTaskState(task)
scheduleInfo := ""
if !task.ScheduledAt.IsZero() && time.Until(task.ScheduledAt) > time.Minute {
scheduleInfo = fmt.Sprintf(", scheduled for %v", task.ScheduledAt.Format("15:04:05"))
@ -143,7 +253,11 @@ func (mq *MaintenanceQueue) GetNextTask(workerID string, capabilities []Maintena
// Check if this task type needs a cooldown period
if !mq.canScheduleTaskNow(task) {
glog.V(3).Infof("Task %s (%s) skipped for worker %s: scheduling constraints not met", task.ID, task.Type, workerID)
// Add detailed diagnostic information
runningCount := mq.GetRunningTaskCount(task.Type)
maxConcurrent := mq.getMaxConcurrentForTaskType(task.Type)
glog.V(2).Infof("Task %s (%s) skipped for worker %s: scheduling constraints not met (running: %d, max: %d)",
task.ID, task.Type, workerID, runningCount, maxConcurrent)
continue
}
@ -172,6 +286,26 @@ func (mq *MaintenanceQueue) GetNextTask(workerID string, capabilities []Maintena
return nil
}
// Record assignment history
workerAddress := ""
if worker, exists := mq.workers[workerID]; exists {
workerAddress = worker.Address
}
// Create assignment record
assignmentRecord := &TaskAssignmentRecord{
WorkerID: workerID,
WorkerAddress: workerAddress,
AssignedAt: now,
Reason: "Task assigned to available worker",
}
// Initialize assignment history if nil
if selectedTask.AssignmentHistory == nil {
selectedTask.AssignmentHistory = make([]*TaskAssignmentRecord, 0)
}
selectedTask.AssignmentHistory = append(selectedTask.AssignmentHistory, assignmentRecord)
// Assign the task
selectedTask.Status = TaskStatusAssigned
selectedTask.WorkerID = workerID
@ -188,6 +322,9 @@ func (mq *MaintenanceQueue) GetNextTask(workerID string, capabilities []Maintena
// Track pending operation
mq.trackPendingOperation(selectedTask)
// Save task state after assignment
mq.saveTaskState(selectedTask)
glog.Infof("Task assigned: %s (%s) → worker %s (volume %d, server %s)",
selectedTask.ID, selectedTask.Type, workerID, selectedTask.VolumeID, selectedTask.Server)
@ -220,6 +357,17 @@ func (mq *MaintenanceQueue) CompleteTask(taskID string, error string) {
// Check if task should be retried
if task.RetryCount < task.MaxRetries {
// Record unassignment due to failure/retry
if task.WorkerID != "" && len(task.AssignmentHistory) > 0 {
lastAssignment := task.AssignmentHistory[len(task.AssignmentHistory)-1]
if lastAssignment.UnassignedAt == nil {
unassignedTime := completedTime
lastAssignment.UnassignedAt = &unassignedTime
lastAssignment.Reason = fmt.Sprintf("Task failed, scheduling retry (attempt %d/%d): %s",
task.RetryCount+1, task.MaxRetries, error)
}
}
task.RetryCount++
task.Status = TaskStatusPending
task.WorkerID = ""
@ -229,15 +377,31 @@ func (mq *MaintenanceQueue) CompleteTask(taskID string, error string) {
task.ScheduledAt = time.Now().Add(15 * time.Minute) // Retry delay
mq.pendingTasks = append(mq.pendingTasks, task)
// Save task state after retry setup
mq.saveTaskState(task)
glog.Warningf("Task failed, scheduling retry: %s (%s) attempt %d/%d, worker %s, duration %v, error: %s",
taskID, task.Type, task.RetryCount, task.MaxRetries, task.WorkerID, duration, error)
} else {
// Record unassignment due to permanent failure
if task.WorkerID != "" && len(task.AssignmentHistory) > 0 {
lastAssignment := task.AssignmentHistory[len(task.AssignmentHistory)-1]
if lastAssignment.UnassignedAt == nil {
unassignedTime := completedTime
lastAssignment.UnassignedAt = &unassignedTime
lastAssignment.Reason = fmt.Sprintf("Task failed permanently after %d retries: %s", task.MaxRetries, error)
}
}
// Save task state after permanent failure
mq.saveTaskState(task)
glog.Errorf("Task failed permanently: %s (%s) worker %s, duration %v, after %d retries: %s",
taskID, task.Type, task.WorkerID, duration, task.MaxRetries, error)
}
} else {
task.Status = TaskStatusCompleted
task.Progress = 100
// Save task state after successful completion
mq.saveTaskState(task)
glog.Infof("Task completed: %s (%s) worker %s, duration %v, volume %d",
taskID, task.Type, task.WorkerID, duration, task.VolumeID)
}
@ -257,6 +421,14 @@ func (mq *MaintenanceQueue) CompleteTask(taskID string, error string) {
if task.Status != TaskStatusPending {
mq.removePendingOperation(taskID)
}
// Periodically cleanup old completed tasks (every 10th completion)
if task.Status == TaskStatusCompleted {
// Simple counter-based trigger for cleanup
if len(mq.tasks)%10 == 0 {
go mq.cleanupCompletedTasks()
}
}
}
// UpdateTaskProgress updates the progress of a running task
@ -283,6 +455,11 @@ func (mq *MaintenanceQueue) UpdateTaskProgress(taskID string, progress float64)
glog.V(1).Infof("Task progress: %s (%s) worker %s, %.1f%% complete",
taskID, task.Type, task.WorkerID, progress)
}
// Save task state after progress update
if progress == 0 || progress >= 100 || progress-oldProgress >= 10 {
mq.saveTaskState(task)
}
} else {
glog.V(2).Infof("Progress update for unknown task: %s (%.1f%%)", taskID, progress)
}
@ -489,9 +666,19 @@ func (mq *MaintenanceQueue) RemoveStaleWorkers(timeout time.Duration) int {
for id, worker := range mq.workers {
if worker.LastHeartbeat.Before(cutoff) {
// Mark any assigned tasks as failed
// Mark any assigned tasks as failed and record unassignment
for _, task := range mq.tasks {
if task.WorkerID == id && (task.Status == TaskStatusAssigned || task.Status == TaskStatusInProgress) {
// Record unassignment due to worker becoming unavailable
if len(task.AssignmentHistory) > 0 {
lastAssignment := task.AssignmentHistory[len(task.AssignmentHistory)-1]
if lastAssignment.UnassignedAt == nil {
unassignedTime := time.Now()
lastAssignment.UnassignedAt = &unassignedTime
lastAssignment.Reason = "Worker became unavailable (stale heartbeat)"
}
}
task.Status = TaskStatusFailed
task.Error = "Worker became unavailable"
completedTime := time.Now()
@ -600,7 +787,10 @@ func (mq *MaintenanceQueue) canExecuteTaskType(taskType MaintenanceTaskType) boo
runningCount := mq.GetRunningTaskCount(taskType)
maxConcurrent := mq.getMaxConcurrentForTaskType(taskType)
return runningCount < maxConcurrent
canExecute := runningCount < maxConcurrent
glog.V(3).Infof("canExecuteTaskType for %s: running=%d, max=%d, canExecute=%v", taskType, runningCount, maxConcurrent, canExecute)
return canExecute
}
// getMaxConcurrentForTaskType returns the maximum concurrent tasks allowed for a task type
@ -684,40 +874,28 @@ func (mq *MaintenanceQueue) trackPendingOperation(task *MaintenanceTask) {
opType = OpTypeVolumeMove
}
// Determine destination node and estimated size from typed parameters
// Determine destination node and estimated size from unified targets
destNode := ""
estimatedSize := uint64(1024 * 1024 * 1024) // Default 1GB estimate
switch params := task.TypedParams.TaskParams.(type) {
case *worker_pb.TaskParams_ErasureCodingParams:
if params.ErasureCodingParams != nil {
if len(params.ErasureCodingParams.Destinations) > 0 {
destNode = params.ErasureCodingParams.Destinations[0].Node
}
if params.ErasureCodingParams.EstimatedShardSize > 0 {
estimatedSize = params.ErasureCodingParams.EstimatedShardSize
}
}
case *worker_pb.TaskParams_BalanceParams:
if params.BalanceParams != nil {
destNode = params.BalanceParams.DestNode
if params.BalanceParams.EstimatedSize > 0 {
estimatedSize = params.BalanceParams.EstimatedSize
}
}
case *worker_pb.TaskParams_ReplicationParams:
if params.ReplicationParams != nil {
destNode = params.ReplicationParams.DestNode
if params.ReplicationParams.EstimatedSize > 0 {
estimatedSize = params.ReplicationParams.EstimatedSize
}
// Use unified targets array - the only source of truth
if len(task.TypedParams.Targets) > 0 {
destNode = task.TypedParams.Targets[0].Node
if task.TypedParams.Targets[0].EstimatedSize > 0 {
estimatedSize = task.TypedParams.Targets[0].EstimatedSize
}
}
// Determine source node from unified sources
sourceNode := ""
if len(task.TypedParams.Sources) > 0 {
sourceNode = task.TypedParams.Sources[0].Node
}
operation := &PendingOperation{
VolumeID: task.VolumeID,
OperationType: opType,
SourceNode: task.Server,
SourceNode: sourceNode,
DestNode: destNode,
TaskID: task.ID,
StartTime: time.Now(),

4
weed/admin/maintenance/maintenance_scanner.go

@ -117,6 +117,8 @@ func (ms *MaintenanceScanner) getVolumeHealthMetrics() ([]*VolumeHealthMetrics,
Server: node.Id,
DiskType: diskType, // Track which disk this volume is on
DiskId: volInfo.DiskId, // Use disk ID from volume info
DataCenter: dc.Id, // Data center from current loop
Rack: rack.Id, // Rack from current loop
Collection: volInfo.Collection,
Size: volInfo.Size,
DeletedBytes: volInfo.DeletedByteCount,
@ -207,6 +209,8 @@ func (ms *MaintenanceScanner) convertToTaskMetrics(metrics []*VolumeHealthMetric
Server: metric.Server,
DiskType: metric.DiskType,
DiskId: metric.DiskId,
DataCenter: metric.DataCenter,
Rack: metric.Rack,
Collection: metric.Collection,
Size: metric.Size,
DeletedBytes: metric.DeletedBytes,

67
weed/admin/maintenance/maintenance_types.go

@ -108,6 +108,57 @@ type MaintenanceTask struct {
Progress float64 `json:"progress"` // 0-100
RetryCount int `json:"retry_count"`
MaxRetries int `json:"max_retries"`
// Enhanced fields for detailed task tracking
CreatedBy string `json:"created_by,omitempty"` // Who/what created this task
CreationContext string `json:"creation_context,omitempty"` // Additional context about creation
AssignmentHistory []*TaskAssignmentRecord `json:"assignment_history,omitempty"` // History of worker assignments
DetailedReason string `json:"detailed_reason,omitempty"` // More detailed explanation than Reason
Tags map[string]string `json:"tags,omitempty"` // Additional metadata tags
}
// TaskAssignmentRecord tracks when a task was assigned to a worker
type TaskAssignmentRecord struct {
WorkerID string `json:"worker_id"`
WorkerAddress string `json:"worker_address"`
AssignedAt time.Time `json:"assigned_at"`
UnassignedAt *time.Time `json:"unassigned_at,omitempty"`
Reason string `json:"reason"` // Why was it assigned/unassigned
}
// TaskExecutionLog represents a log entry from task execution
type TaskExecutionLog struct {
Timestamp time.Time `json:"timestamp"`
Level string `json:"level"` // "info", "warn", "error", "debug"
Message string `json:"message"`
Source string `json:"source"` // Which component logged this
TaskID string `json:"task_id"`
WorkerID string `json:"worker_id"`
// Optional structured fields carried from worker logs
Fields map[string]string `json:"fields,omitempty"`
// Optional progress/status carried from worker logs
Progress *float64 `json:"progress,omitempty"`
Status string `json:"status,omitempty"`
}
// TaskDetailData represents comprehensive information about a task for the detail view
type TaskDetailData struct {
Task *MaintenanceTask `json:"task"`
AssignmentHistory []*TaskAssignmentRecord `json:"assignment_history"`
ExecutionLogs []*TaskExecutionLog `json:"execution_logs"`
RelatedTasks []*MaintenanceTask `json:"related_tasks,omitempty"` // Other tasks on same volume/server
WorkerInfo *MaintenanceWorker `json:"worker_info,omitempty"` // Current or last assigned worker
CreationMetrics *TaskCreationMetrics `json:"creation_metrics,omitempty"` // Metrics that led to task creation
LastUpdated time.Time `json:"last_updated"`
}
// TaskCreationMetrics holds metrics that led to the task being created
type TaskCreationMetrics struct {
TriggerMetric string `json:"trigger_metric"` // What metric triggered this task
MetricValue float64 `json:"metric_value"` // Value of the trigger metric
Threshold float64 `json:"threshold"` // Threshold that was exceeded
VolumeMetrics *VolumeHealthMetrics `json:"volume_metrics,omitempty"`
AdditionalData map[string]interface{} `json:"additional_data,omitempty"`
}
// MaintenanceConfig holds configuration for the maintenance system
@ -122,6 +173,15 @@ type MaintenancePolicy = worker_pb.MaintenancePolicy
// DEPRECATED: Use worker_pb.TaskPolicy instead
type TaskPolicy = worker_pb.TaskPolicy
// TaskPersistence interface for task state persistence
type TaskPersistence interface {
SaveTaskState(task *MaintenanceTask) error
LoadTaskState(taskID string) (*MaintenanceTask, error)
LoadAllTaskStates() ([]*MaintenanceTask, error)
DeleteTaskState(taskID string) error
CleanupCompletedTasks() error
}
// Default configuration values
func DefaultMaintenanceConfig() *MaintenanceConfig {
return DefaultMaintenanceConfigProto()
@ -273,6 +333,7 @@ type MaintenanceQueue struct {
mutex sync.RWMutex
policy *MaintenancePolicy
integration *MaintenanceIntegration
persistence TaskPersistence // Interface for task persistence
}
// MaintenanceScanner analyzes the cluster and generates maintenance tasks
@ -301,8 +362,10 @@ type TaskDetectionResult struct {
type VolumeHealthMetrics struct {
VolumeID uint32 `json:"volume_id"`
Server string `json:"server"`
DiskType string `json:"disk_type"` // Disk type (e.g., "hdd", "ssd") or disk path (e.g., "/data1")
DiskId uint32 `json:"disk_id"` // ID of the disk in Store.Locations array
DiskType string `json:"disk_type"` // Disk type (e.g., "hdd", "ssd") or disk path (e.g., "/data1")
DiskId uint32 `json:"disk_id"` // ID of the disk in Store.Locations array
DataCenter string `json:"data_center"` // Data center of the server
Rack string `json:"rack"` // Rack of the server
Collection string `json:"collection"`
Size uint64 `json:"size"`
DeletedBytes uint64 `json:"deleted_bytes"`

19
weed/admin/topology/structs.go

@ -96,13 +96,12 @@ type ActiveTopology struct {
// DestinationPlan represents a planned destination for a volume/shard operation
type DestinationPlan struct {
TargetNode string `json:"target_node"`
TargetDisk uint32 `json:"target_disk"`
TargetRack string `json:"target_rack"`
TargetDC string `json:"target_dc"`
ExpectedSize uint64 `json:"expected_size"`
PlacementScore float64 `json:"placement_score"`
Conflicts []string `json:"conflicts"`
TargetNode string `json:"target_node"`
TargetDisk uint32 `json:"target_disk"`
TargetRack string `json:"target_rack"`
TargetDC string `json:"target_dc"`
ExpectedSize uint64 `json:"expected_size"`
PlacementScore float64 `json:"placement_score"`
}
// MultiDestinationPlan represents multiple planned destinations for operations like EC
@ -115,6 +114,8 @@ type MultiDestinationPlan struct {
// VolumeReplica represents a replica location with server and disk information
type VolumeReplica struct {
ServerID string `json:"server_id"`
DiskID uint32 `json:"disk_id"`
ServerID string `json:"server_id"`
DiskID uint32 `json:"disk_id"`
DataCenter string `json:"data_center"`
Rack string `json:"rack"`
}

9
weed/admin/topology/task_management.go

@ -233,6 +233,8 @@ const (
type TaskSourceSpec struct {
ServerID string
DiskID uint32
DataCenter string // Data center of the source server
Rack string // Rack of the source server
CleanupType SourceCleanupType // For EC: volume replica vs existing shards
StorageImpact *StorageSlotChange // Optional: manual override
EstimatedSize *int64 // Optional: manual override
@ -255,10 +257,3 @@ type TaskSpec struct {
Sources []TaskSourceSpec // Can be single or multiple
Destinations []TaskDestinationSpec // Can be single or multiple
}
// TaskSourceLocation represents a source location for task creation (DEPRECATED: use TaskSourceSpec)
type TaskSourceLocation struct {
ServerID string
DiskID uint32
CleanupType SourceCleanupType // What type of cleanup is needed
}

12
weed/admin/topology/topology_management.go

@ -188,8 +188,10 @@ func (at *ActiveTopology) GetVolumeLocations(volumeID uint32, collection string)
// Verify collection matches (since index doesn't include collection)
if at.volumeMatchesCollection(disk, volumeID, collection) {
replicas = append(replicas, VolumeReplica{
ServerID: disk.NodeID,
DiskID: disk.DiskID,
ServerID: disk.NodeID,
DiskID: disk.DiskID,
DataCenter: disk.DataCenter,
Rack: disk.Rack,
})
}
}
@ -214,8 +216,10 @@ func (at *ActiveTopology) GetECShardLocations(volumeID uint32, collection string
// Verify collection matches (since index doesn't include collection)
if at.ecShardMatchesCollection(disk, volumeID, collection) {
ecShards = append(ecShards, VolumeReplica{
ServerID: disk.NodeID,
DiskID: disk.DiskID,
ServerID: disk.NodeID,
DiskID: disk.DiskID,
DataCenter: disk.DataCenter,
Rack: disk.Rack,
})
}
}

2
weed/admin/view/app/admin.templ

@ -12,7 +12,7 @@ templ Admin(data dash.AdminData) {
</h1>
<div class="btn-toolbar mb-2 mb-md-0">
<div class="btn-group me-2">
<a href="/s3/buckets" class="btn btn-sm btn-primary">
<a href="/object-store/buckets" class="btn btn-sm btn-primary">
<i class="fas fa-cube me-1"></i>Object Store Buckets
</a>

2
weed/admin/view/app/admin_templ.go

@ -34,7 +34,7 @@ func Admin(data dash.AdminData) templ.Component {
templ_7745c5c3_Var1 = templ.NopComponent
}
ctx = templ.ClearChildren(ctx)
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 1, "<div class=\"d-flex justify-content-between flex-wrap flex-md-nowrap align-items-center pt-3 pb-2 mb-3 border-bottom\"><h1 class=\"h2\"><i class=\"fas fa-tachometer-alt me-2\"></i>Dashboard</h1><div class=\"btn-toolbar mb-2 mb-md-0\"><div class=\"btn-group me-2\"><a href=\"/s3/buckets\" class=\"btn btn-sm btn-primary\"><i class=\"fas fa-cube me-1\"></i>Object Store Buckets</a></div></div></div><div id=\"dashboard-content\"><!-- Summary Cards --><div class=\"row mb-4\"><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\">Total Volumes</div><div class=\"h5 mb-0 font-weight-bold text-gray-800\">")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 1, "<div class=\"d-flex justify-content-between flex-wrap flex-md-nowrap align-items-center pt-3 pb-2 mb-3 border-bottom\"><h1 class=\"h2\"><i class=\"fas fa-tachometer-alt me-2\"></i>Dashboard</h1><div class=\"btn-toolbar mb-2 mb-md-0\"><div class=\"btn-group me-2\"><a href=\"/object-store/buckets\" class=\"btn btn-sm btn-primary\"><i class=\"fas fa-cube me-1\"></i>Object Store Buckets</a></div></div></div><div id=\"dashboard-content\"><!-- Summary Cards --><div class=\"row mb-4\"><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\">Total Volumes</div><div class=\"h5 mb-0 font-weight-bold text-gray-800\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}

13
weed/admin/view/app/cluster_ec_volumes.templ

@ -4,6 +4,7 @@ import (
"fmt"
"strings"
"github.com/seaweedfs/seaweedfs/weed/admin/dash"
"github.com/seaweedfs/seaweedfs/weed/storage/erasure_coding"
)
templ ClusterEcVolumes(data dash.ClusterEcVolumesData) {
@ -99,8 +100,8 @@ templ ClusterEcVolumes(data dash.ClusterEcVolumesData) {
<div class="alert alert-info mb-4" role="alert">
<i class="fas fa-info-circle me-2"></i>
<strong>EC Storage Note:</strong>
EC volumes use erasure coding (10+4) which stores data across 14 shards with redundancy.
Physical storage is approximately 1.4x the original logical data size due to 4 parity shards.
EC volumes use erasure coding ({ fmt.Sprintf("%d+%d", erasure_coding.DataShardsCount, erasure_coding.ParityShardsCount) }) which stores data across { fmt.Sprintf("%d", erasure_coding.TotalShardsCount) } shards with redundancy.
Physical storage is approximately { fmt.Sprintf("%.1fx", float64(erasure_coding.TotalShardsCount)/float64(erasure_coding.DataShardsCount)) } the original logical data size due to { fmt.Sprintf("%d", erasure_coding.ParityShardsCount) } parity shards.
</div>
<!-- Volumes Table -->
@ -688,7 +689,7 @@ func formatIndividualShardSizes(shardSizes map[int]int64) string {
}
var idRanges []string
if len(shardIds) <= 4 {
if len(shardIds) <= erasure_coding.ParityShardsCount {
// Show individual IDs if few shards
for _, id := range shardIds {
idRanges = append(idRanges, fmt.Sprintf("%d", id))
@ -719,11 +720,11 @@ templ displayEcVolumeStatus(volume dash.EcVolumeWithShards) {
if volume.IsComplete {
<span class="badge bg-success"><i class="fas fa-check me-1"></i>Complete</span>
} else {
if len(volume.MissingShards) > 10 {
if len(volume.MissingShards) > erasure_coding.DataShardsCount {
<span class="badge bg-danger"><i class="fas fa-skull me-1"></i>Critical ({fmt.Sprintf("%d", len(volume.MissingShards))} missing)</span>
} else if len(volume.MissingShards) > 6 {
} else if len(volume.MissingShards) > (erasure_coding.DataShardsCount/2) {
<span class="badge bg-warning"><i class="fas fa-exclamation-triangle me-1"></i>Degraded ({fmt.Sprintf("%d", len(volume.MissingShards))} missing)</span>
} else if len(volume.MissingShards) > 2 {
} else if len(volume.MissingShards) > (erasure_coding.ParityShardsCount/2) {
<span class="badge bg-warning"><i class="fas fa-info-circle me-1"></i>Incomplete ({fmt.Sprintf("%d", len(volume.MissingShards))} missing)</span>
} else {
<span class="badge bg-info"><i class="fas fa-info-circle me-1"></i>Minor Issues ({fmt.Sprintf("%d", len(volume.MissingShards))} missing)</span>

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

4
weed/admin/view/app/cluster_volume_servers.templ

@ -98,7 +98,6 @@ templ ClusterVolumeServers(data dash.ClusterVolumeServersData) {
<table class="table table-hover" id="hostsTable">
<thead>
<tr>
<th>Server ID</th>
<th>Address</th>
<th>Data Center</th>
<th>Rack</th>
@ -113,9 +112,6 @@ templ ClusterVolumeServers(data dash.ClusterVolumeServersData) {
<tbody>
for _, host := range data.VolumeServers {
<tr>
<td>
<code>{host.ID}</code>
</td>
<td>
<a href={templ.SafeURL(fmt.Sprintf("http://%s/ui/index.html", host.PublicURL))} target="_blank" class="text-decoration-none">
{host.Address}

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

10
weed/admin/view/app/collection_details.templ

@ -262,6 +262,16 @@ templ CollectionDetails(data dash.CollectionDetailsData) {
</td>
</tr>
}
// Show message when no volumes found
if len(data.RegularVolumes) == 0 && len(data.EcVolumes) == 0 {
<tr>
<td colspan="6" class="text-center text-muted py-4">
<i class="fas fa-info-circle me-2"></i>
No volumes found for collection "{data.CollectionName}"
</td>
</tr>
}
</tbody>
</table>
</div>

99
weed/admin/view/app/collection_details_templ.go

@ -429,134 +429,153 @@ func CollectionDetails(data dash.CollectionDetailsData) templ.Component {
return templ_7745c5c3_Err
}
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 47, "</tbody></table></div><!-- Pagination -->")
if len(data.RegularVolumes) == 0 && len(data.EcVolumes) == 0 {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 47, "<tr><td colspan=\"6\" class=\"text-center text-muted py-4\"><i class=\"fas fa-info-circle me-2\"></i> No volumes found for collection \"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var21 string
templ_7745c5c3_Var21, templ_7745c5c3_Err = templ.JoinStringErrs(data.CollectionName)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 271, Col: 60}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var21))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 48, "\"</td></tr>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 49, "</tbody></table></div><!-- Pagination -->")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if data.TotalPages > 1 {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 48, "<nav aria-label=\"Collection volumes pagination\"><ul class=\"pagination justify-content-center\">")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 50, "<nav aria-label=\"Collection volumes pagination\"><ul class=\"pagination justify-content-center\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if data.Page > 1 {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 49, "<li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"1\">First</a></li><li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 51, "<li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"1\">First</a></li><li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var21 string
templ_7745c5c3_Var21, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.Page-1))
var templ_7745c5c3_Var22 string
templ_7745c5c3_Var22, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.Page-1))
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 278, Col: 104}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 288, Col: 104}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var21))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var22))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 50, "\">Previous</a></li>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 52, "\">Previous</a></li>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
for i := 1; i <= data.TotalPages; i++ {
if i == data.Page {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 51, "<li class=\"page-item active\"><span class=\"page-link\">")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 53, "<li class=\"page-item active\"><span class=\"page-link\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var22 string
templ_7745c5c3_Var22, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", i))
var templ_7745c5c3_Var23 string
templ_7745c5c3_Var23, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", i))
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 285, Col: 52}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 295, Col: 52}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var22))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var23))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 52, "</span></li>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 54, "</span></li>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else if i <= 3 || i > data.TotalPages-3 || (i >= data.Page-2 && i <= data.Page+2) {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 53, "<li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 55, "<li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var23 string
templ_7745c5c3_Var23, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", i))
var templ_7745c5c3_Var24 string
templ_7745c5c3_Var24, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", i))
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 289, Col: 95}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 299, Col: 95}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var23))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var24))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 54, "\">")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 56, "\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var24 string
templ_7745c5c3_Var24, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", i))
var templ_7745c5c3_Var25 string
templ_7745c5c3_Var25, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", i))
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 289, Col: 119}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 299, Col: 119}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var24))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var25))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 55, "</a></li>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 57, "</a></li>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else if i == 4 && data.Page > 6 {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 56, "<li class=\"page-item disabled\"><span class=\"page-link\">...</span></li>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 58, "<li class=\"page-item disabled\"><span class=\"page-link\">...</span></li>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else if i == data.TotalPages-3 && data.Page < data.TotalPages-5 {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 57, "<li class=\"page-item disabled\"><span class=\"page-link\">...</span></li>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 59, "<li class=\"page-item disabled\"><span class=\"page-link\">...</span></li>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
}
if data.Page < data.TotalPages {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 58, "<li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 60, "<li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var25 string
templ_7745c5c3_Var25, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.Page+1))
var templ_7745c5c3_Var26 string
templ_7745c5c3_Var26, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.Page+1))
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 304, Col: 104}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 314, Col: 104}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var25))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var26))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 59, "\">Next</a></li><li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 61, "\">Next</a></li><li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var26 string
templ_7745c5c3_Var26, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.TotalPages))
var templ_7745c5c3_Var27 string
templ_7745c5c3_Var27, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.TotalPages))
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 307, Col: 108}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 317, Col: 108}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var26))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var27))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 60, "\">Last</a></li>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 62, "\">Last</a></li>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 61, "</ul></nav>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 63, "</ul></nav>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 62, "<script>\n\t\t// Sorting functionality\n\t\tfunction sortBy(field) {\n\t\t\tconst currentSort = new URLSearchParams(window.location.search).get('sort_by');\n\t\t\tconst currentOrder = new URLSearchParams(window.location.search).get('sort_order') || 'asc';\n\t\t\t\n\t\t\tlet newOrder = 'asc';\n\t\t\tif (currentSort === field && currentOrder === 'asc') {\n\t\t\t\tnewOrder = 'desc';\n\t\t\t}\n\t\t\t\n\t\t\tconst url = new URL(window.location);\n\t\t\turl.searchParams.set('sort_by', field);\n\t\t\turl.searchParams.set('sort_order', newOrder);\n\t\t\turl.searchParams.set('page', '1'); // Reset to first page\n\t\t\twindow.location.href = url.toString();\n\t\t}\n\n\t\t// Pagination functionality\n\t\tfunction goToPage(event) {\n\t\t\tevent.preventDefault();\n\t\t\tconst page = event.target.closest('a').getAttribute('data-page');\n\t\t\tconst url = new URL(window.location);\n\t\t\turl.searchParams.set('page', page);\n\t\t\twindow.location.href = url.toString();\n\t\t}\n\n\t\t// Page size functionality\n\t\tfunction changePageSize(newPageSize) {\n\t\t\tconst url = new URL(window.location);\n\t\t\turl.searchParams.set('page_size', newPageSize);\n\t\t\turl.searchParams.set('page', '1'); // Reset to first page when changing page size\n\t\t\twindow.location.href = url.toString();\n\t\t}\n\n\t\t// Volume details\n\t\tfunction showVolumeDetails(event) {\n\t\t\tconst volumeId = event.target.closest('button').getAttribute('data-volume-id');\n\t\t\tconst server = event.target.closest('button').getAttribute('data-server');\n\t\t\twindow.location.href = `/cluster/volumes/${volumeId}/${server}`;\n\t\t}\n\n\t\t// EC Volume details\n\t\tfunction showEcVolumeDetails(event) {\n\t\t\tconst volumeId = event.target.closest('button').getAttribute('data-volume-id');\n\t\t\twindow.location.href = `/cluster/ec-volumes/${volumeId}`;\n\t\t}\n\n\t\t// Repair EC Volume\n\t\tfunction repairEcVolume(event) {\n\t\t\tconst volumeId = event.target.closest('button').getAttribute('data-volume-id');\n\t\t\tif (confirm(`Are you sure you want to repair missing shards for EC volume ${volumeId}?`)) {\n\t\t\t\t// TODO: Implement repair functionality\n\t\t\t\talert('Repair functionality will be implemented soon.');\n\t\t\t}\n\t\t}\n\t</script>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 64, "<script>\n\t\t// Sorting functionality\n\t\tfunction sortBy(field) {\n\t\t\tconst currentSort = new URLSearchParams(window.location.search).get('sort_by');\n\t\t\tconst currentOrder = new URLSearchParams(window.location.search).get('sort_order') || 'asc';\n\t\t\t\n\t\t\tlet newOrder = 'asc';\n\t\t\tif (currentSort === field && currentOrder === 'asc') {\n\t\t\t\tnewOrder = 'desc';\n\t\t\t}\n\t\t\t\n\t\t\tconst url = new URL(window.location);\n\t\t\turl.searchParams.set('sort_by', field);\n\t\t\turl.searchParams.set('sort_order', newOrder);\n\t\t\turl.searchParams.set('page', '1'); // Reset to first page\n\t\t\twindow.location.href = url.toString();\n\t\t}\n\n\t\t// Pagination functionality\n\t\tfunction goToPage(event) {\n\t\t\tevent.preventDefault();\n\t\t\tconst page = event.target.closest('a').getAttribute('data-page');\n\t\t\tconst url = new URL(window.location);\n\t\t\turl.searchParams.set('page', page);\n\t\t\twindow.location.href = url.toString();\n\t\t}\n\n\t\t// Page size functionality\n\t\tfunction changePageSize(newPageSize) {\n\t\t\tconst url = new URL(window.location);\n\t\t\turl.searchParams.set('page_size', newPageSize);\n\t\t\turl.searchParams.set('page', '1'); // Reset to first page when changing page size\n\t\t\twindow.location.href = url.toString();\n\t\t}\n\n\t\t// Volume details\n\t\tfunction showVolumeDetails(event) {\n\t\t\tconst volumeId = event.target.closest('button').getAttribute('data-volume-id');\n\t\t\tconst server = event.target.closest('button').getAttribute('data-server');\n\t\t\twindow.location.href = `/cluster/volumes/${volumeId}/${server}`;\n\t\t}\n\n\t\t// EC Volume details\n\t\tfunction showEcVolumeDetails(event) {\n\t\t\tconst volumeId = event.target.closest('button').getAttribute('data-volume-id');\n\t\t\twindow.location.href = `/cluster/ec-volumes/${volumeId}`;\n\t\t}\n\n\t\t// Repair EC Volume\n\t\tfunction repairEcVolume(event) {\n\t\t\tconst volumeId = event.target.closest('button').getAttribute('data-volume-id');\n\t\t\tif (confirm(`Are you sure you want to repair missing shards for EC volume ${volumeId}?`)) {\n\t\t\t\t// TODO: Implement repair functionality\n\t\t\t\talert('Repair functionality will be implemented soon.');\n\t\t\t}\n\t\t}\n\t</script>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}

221
weed/admin/view/app/maintenance_queue.templ

@ -70,6 +70,111 @@ templ MaintenanceQueue(data *maintenance.MaintenanceQueueData) {
</div>
</div>
<!-- Completed Tasks -->
<div class="row mb-4">
<div class="col-12">
<div class="card">
<div class="card-header bg-success text-white">
<h5 class="mb-0">
<i class="fas fa-check-circle me-2"></i>
Completed Tasks
</h5>
</div>
<div class="card-body">
if data.Stats.CompletedToday == 0 && data.Stats.FailedToday == 0 {
<div class="text-center text-muted py-4">
<i class="fas fa-check-circle fa-3x mb-3"></i>
<p>No completed maintenance tasks today</p>
<small>Completed tasks will appear here after workers finish processing them</small>
</div>
} else {
<div class="table-responsive">
<table class="table table-hover">
<thead>
<tr>
<th>Type</th>
<th>Status</th>
<th>Volume</th>
<th>Worker</th>
<th>Duration</th>
<th>Completed</th>
</tr>
</thead>
<tbody>
for _, task := range data.Tasks {
if string(task.Status) == "completed" || string(task.Status) == "failed" || string(task.Status) == "cancelled" {
if string(task.Status) == "failed" {
<tr class="table-danger clickable-row" data-task-id={task.ID} onclick="navigateToTask(this)" style="cursor: pointer;">
<td>
@TaskTypeIcon(task.Type)
{string(task.Type)}
</td>
<td>@StatusBadge(task.Status)</td>
<td>{fmt.Sprintf("%d", task.VolumeID)}</td>
<td>
if task.WorkerID != "" {
<small>{task.WorkerID}</small>
} else {
<span class="text-muted">-</span>
}
</td>
<td>
if task.StartedAt != nil && task.CompletedAt != nil {
{formatDuration(task.CompletedAt.Sub(*task.StartedAt))}
} else {
<span class="text-muted">-</span>
}
</td>
<td>
if task.CompletedAt != nil {
{task.CompletedAt.Format("2006-01-02 15:04")}
} else {
<span class="text-muted">-</span>
}
</td>
</tr>
} else {
<tr class="clickable-row" data-task-id={task.ID} onclick="navigateToTask(this)" style="cursor: pointer;">
<td>
@TaskTypeIcon(task.Type)
{string(task.Type)}
</td>
<td>@StatusBadge(task.Status)</td>
<td>{fmt.Sprintf("%d", task.VolumeID)}</td>
<td>
if task.WorkerID != "" {
<small>{task.WorkerID}</small>
} else {
<span class="text-muted">-</span>
}
</td>
<td>
if task.StartedAt != nil && task.CompletedAt != nil {
{formatDuration(task.CompletedAt.Sub(*task.StartedAt))}
} else {
<span class="text-muted">-</span>
}
</td>
<td>
if task.CompletedAt != nil {
{task.CompletedAt.Format("2006-01-02 15:04")}
} else {
<span class="text-muted">-</span>
}
</td>
</tr>
}
}
}
</tbody>
</table>
</div>
}
</div>
</div>
</div>
</div>
<!-- Pending Tasks -->
<div class="row mb-4">
<div class="col-12">
@ -103,7 +208,7 @@ templ MaintenanceQueue(data *maintenance.MaintenanceQueueData) {
<tbody>
for _, task := range data.Tasks {
if string(task.Status) == "pending" {
<tr>
<tr class="clickable-row" data-task-id={task.ID} onclick="navigateToTask(this)" style="cursor: pointer;">
<td>
@TaskTypeIcon(task.Type)
{string(task.Type)}
@ -158,7 +263,7 @@ templ MaintenanceQueue(data *maintenance.MaintenanceQueueData) {
<tbody>
for _, task := range data.Tasks {
if string(task.Status) == "assigned" || string(task.Status) == "in_progress" {
<tr>
<tr class="clickable-row" data-task-id={task.ID} onclick="navigateToTask(this)" style="cursor: pointer;">
<td>
@TaskTypeIcon(task.Type)
{string(task.Type)}
@ -191,111 +296,6 @@ templ MaintenanceQueue(data *maintenance.MaintenanceQueueData) {
</div>
</div>
</div>
<!-- Completed Tasks -->
<div class="row mb-4">
<div class="col-12">
<div class="card">
<div class="card-header bg-success text-white">
<h5 class="mb-0">
<i class="fas fa-check-circle me-2"></i>
Completed Tasks
</h5>
</div>
<div class="card-body">
if data.Stats.CompletedToday == 0 && data.Stats.FailedToday == 0 {
<div class="text-center text-muted py-4">
<i class="fas fa-check-circle fa-3x mb-3"></i>
<p>No completed maintenance tasks today</p>
<small>Completed tasks will appear here after workers finish processing them</small>
</div>
} else {
<div class="table-responsive">
<table class="table table-hover">
<thead>
<tr>
<th>Type</th>
<th>Status</th>
<th>Volume</th>
<th>Worker</th>
<th>Duration</th>
<th>Completed</th>
</tr>
</thead>
<tbody>
for _, task := range data.Tasks {
if string(task.Status) == "completed" || string(task.Status) == "failed" || string(task.Status) == "cancelled" {
if string(task.Status) == "failed" {
<tr class="table-danger">
<td>
@TaskTypeIcon(task.Type)
{string(task.Type)}
</td>
<td>@StatusBadge(task.Status)</td>
<td>{fmt.Sprintf("%d", task.VolumeID)}</td>
<td>
if task.WorkerID != "" {
<small>{task.WorkerID}</small>
} else {
<span class="text-muted">-</span>
}
</td>
<td>
if task.StartedAt != nil && task.CompletedAt != nil {
{formatDuration(task.CompletedAt.Sub(*task.StartedAt))}
} else {
<span class="text-muted">-</span>
}
</td>
<td>
if task.CompletedAt != nil {
{task.CompletedAt.Format("2006-01-02 15:04")}
} else {
<span class="text-muted">-</span>
}
</td>
</tr>
} else {
<tr>
<td>
@TaskTypeIcon(task.Type)
{string(task.Type)}
</td>
<td>@StatusBadge(task.Status)</td>
<td>{fmt.Sprintf("%d", task.VolumeID)}</td>
<td>
if task.WorkerID != "" {
<small>{task.WorkerID}</small>
} else {
<span class="text-muted">-</span>
}
</td>
<td>
if task.StartedAt != nil && task.CompletedAt != nil {
{formatDuration(task.CompletedAt.Sub(*task.StartedAt))}
} else {
<span class="text-muted">-</span>
}
</td>
<td>
if task.CompletedAt != nil {
{task.CompletedAt.Format("2006-01-02 15:04")}
} else {
<span class="text-muted">-</span>
}
</td>
</tr>
}
}
}
</tbody>
</table>
</div>
}
</div>
</div>
</div>
</div>
</div>
<script>
@ -335,6 +335,13 @@ templ MaintenanceQueue(data *maintenance.MaintenanceQueueData) {
console.log("refreshPage called");
window.location.reload();
};
window.navigateToTask = function(element) {
const taskId = element.getAttribute('data-task-id');
if (taskId) {
window.location.href = '/maintenance/tasks/' + taskId;
}
};
</script>
}

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

3
weed/admin/view/app/task_config_schema.templ

@ -10,6 +10,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/worker/tasks"
"github.com/seaweedfs/seaweedfs/weed/admin/config"
"github.com/seaweedfs/seaweedfs/weed/admin/view/components"
"github.com/seaweedfs/seaweedfs/weed/storage/erasure_coding"
)
// Helper function to convert task schema to JSON string
@ -114,7 +115,7 @@ templ TaskConfigSchema(data *maintenance.TaskConfigData, schema *tasks.TaskConfi
} else if schema.TaskName == "erasure_coding" {
<h6 class="alert-heading">Erasure Coding Operations:</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-2"><strong>Durability:</strong> With 10+4 configuration, can tolerate up to 4 shard failures.</p>
<p class="mb-2"><strong>Durability:</strong> With { fmt.Sprintf("%d+%d", erasure_coding.DataShardsCount, erasure_coding.ParityShardsCount) } configuration, can tolerate up to { fmt.Sprintf("%d", erasure_coding.ParityShardsCount) } shard failures.</p>
<p class="mb-0"><strong>Configuration:</strong> Fullness ratio should be between 0.5 and 1.0 (e.g., 0.90 for 90%).</p>
}
</div>

425
weed/admin/view/app/task_config_schema_templ.go

@ -15,6 +15,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/admin/config"
"github.com/seaweedfs/seaweedfs/weed/admin/maintenance"
"github.com/seaweedfs/seaweedfs/weed/admin/view/components"
"github.com/seaweedfs/seaweedfs/weed/storage/erasure_coding"
"github.com/seaweedfs/seaweedfs/weed/worker/tasks"
"reflect"
"strings"
@ -94,7 +95,7 @@ func TaskConfigSchema(data *maintenance.TaskConfigData, schema *tasks.TaskConfig
var templ_7745c5c3_Var4 string
templ_7745c5c3_Var4, templ_7745c5c3_Err = templ.JoinStringErrs(schema.DisplayName)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 46, Col: 43}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 47, Col: 43}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var4))
if templ_7745c5c3_Err != nil {
@ -107,7 +108,7 @@ func TaskConfigSchema(data *maintenance.TaskConfigData, schema *tasks.TaskConfig
var templ_7745c5c3_Var5 string
templ_7745c5c3_Var5, templ_7745c5c3_Err = templ.JoinStringErrs(schema.Description)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 67, Col: 76}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 68, Col: 76}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var5))
if templ_7745c5c3_Err != nil {
@ -138,25 +139,51 @@ func TaskConfigSchema(data *maintenance.TaskConfigData, schema *tasks.TaskConfig
return templ_7745c5c3_Err
}
} else if schema.TaskName == "erasure_coding" {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 9, "<h6 class=\"alert-heading\">Erasure Coding Operations:</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-2\"><strong>Durability:</strong> With 10+4 configuration, can tolerate up to 4 shard failures.</p><p class=\"mb-0\"><strong>Configuration:</strong> Fullness ratio should be between 0.5 and 1.0 (e.g., 0.90 for 90%).</p>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 9, "<h6 class=\"alert-heading\">Erasure Coding Operations:</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-2\"><strong>Durability:</strong> With ")
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+%d", erasure_coding.DataShardsCount, erasure_coding.ParityShardsCount))
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 118, Col: 170}
}
_, 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, 10, " configuration, can tolerate up to ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var7 string
templ_7745c5c3_Var7, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", erasure_coding.ParityShardsCount))
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 118, Col: 260}
}
_, 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, 11, " shard failures.</p><p class=\"mb-0\"><strong>Configuration:</strong> Fullness ratio should be between 0.5 and 1.0 (e.g., 0.90 for 90%).</p>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 10, "</div></div></div></div></div></div><script>\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 fields to their default values\n const form = document.getElementById('taskConfigForm');\n const schemaFields = window.taskConfigSchema ? window.taskConfigSchema.fields : {};\n \n Object.keys(schemaFields).forEach(fieldName => {\n const field = schemaFields[fieldName];\n const element = document.getElementById(fieldName);\n \n if (element && field.default_value !== undefined) {\n if (field.input_type === 'checkbox') {\n element.checked = field.default_value;\n } else if (field.input_type === 'interval') {\n // Handle interval fields with value and unit\n const valueElement = document.getElementById(fieldName + '_value');\n const unitElement = document.getElementById(fieldName + '_unit');\n if (valueElement && unitElement && field.default_value) {\n const defaultSeconds = field.default_value;\n const { value, unit } = convertSecondsToTaskIntervalValueUnit(defaultSeconds);\n valueElement.value = value;\n unitElement.value = unit;\n }\n } else {\n element.value = field.default_value;\n }\n }\n });\n }\n }\n\n function convertSecondsToTaskIntervalValueUnit(totalSeconds) {\n if (totalSeconds === 0) {\n return { value: 0, unit: 'minutes' };\n }\n\n // Check if it's evenly divisible by days\n if (totalSeconds % (24 * 3600) === 0) {\n return { value: totalSeconds / (24 * 3600), unit: 'days' };\n }\n\n // Check if it's evenly divisible by hours\n if (totalSeconds % 3600 === 0) {\n return { value: totalSeconds / 3600, unit: 'hours' };\n }\n\n // Default to minutes\n return { value: totalSeconds / 60, unit: 'minutes' };\n }\n\n // Store schema data for JavaScript access (moved to after div is created)\n </script><!-- Hidden element to store schema data --><div data-task-schema=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 12, "</div></div></div></div></div></div><script>\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 fields to their default values\n const form = document.getElementById('taskConfigForm');\n const schemaFields = window.taskConfigSchema ? window.taskConfigSchema.fields : {};\n \n Object.keys(schemaFields).forEach(fieldName => {\n const field = schemaFields[fieldName];\n const element = document.getElementById(fieldName);\n \n if (element && field.default_value !== undefined) {\n if (field.input_type === 'checkbox') {\n element.checked = field.default_value;\n } else if (field.input_type === 'interval') {\n // Handle interval fields with value and unit\n const valueElement = document.getElementById(fieldName + '_value');\n const unitElement = document.getElementById(fieldName + '_unit');\n if (valueElement && unitElement && field.default_value) {\n const defaultSeconds = field.default_value;\n const { value, unit } = convertSecondsToTaskIntervalValueUnit(defaultSeconds);\n valueElement.value = value;\n unitElement.value = unit;\n }\n } else {\n element.value = field.default_value;\n }\n }\n });\n }\n }\n\n function convertSecondsToTaskIntervalValueUnit(totalSeconds) {\n if (totalSeconds === 0) {\n return { value: 0, unit: 'minutes' };\n }\n\n // Check if it's evenly divisible by days\n if (totalSeconds % (24 * 3600) === 0) {\n return { value: totalSeconds / (24 * 3600), unit: 'days' };\n }\n\n // Check if it's evenly divisible by hours\n if (totalSeconds % 3600 === 0) {\n return { value: totalSeconds / 3600, unit: 'hours' };\n }\n\n // Default to minutes\n return { value: totalSeconds / 60, unit: 'minutes' };\n }\n\n // Store schema data for JavaScript access (moved to after div is created)\n </script><!-- Hidden element to store schema data --><div data-task-schema=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var6 string
templ_7745c5c3_Var6, templ_7745c5c3_Err = templ.JoinStringErrs(taskSchemaToBase64JSON(schema))
var templ_7745c5c3_Var8 string
templ_7745c5c3_Var8, templ_7745c5c3_Err = templ.JoinStringErrs(taskSchemaToBase64JSON(schema))
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 182, Col: 58}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 183, Col: 58}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var6))
_, 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, "\" style=\"display: none;\"></div><script>\n // Load schema data now that the div exists\n const base64Data = document.querySelector('[data-task-schema]').getAttribute('data-task-schema');\n const jsonStr = atob(base64Data);\n window.taskConfigSchema = JSON.parse(jsonStr);\n </script>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, "\" style=\"display: none;\"></div><script>\n // Load schema data now that the div exists\n const base64Data = document.querySelector('[data-task-schema]').getAttribute('data-task-schema');\n const jsonStr = atob(base64Data);\n window.taskConfigSchema = JSON.parse(jsonStr);\n </script>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
@ -181,584 +208,584 @@ func TaskConfigField(field *config.Field, config interface{}) templ.Component {
}()
}
ctx = templ.InitializeContext(ctx)
templ_7745c5c3_Var7 := templ.GetChildren(ctx)
if templ_7745c5c3_Var7 == nil {
templ_7745c5c3_Var7 = templ.NopComponent
templ_7745c5c3_Var9 := templ.GetChildren(ctx)
if templ_7745c5c3_Var9 == nil {
templ_7745c5c3_Var9 = templ.NopComponent
}
ctx = templ.ClearChildren(ctx)
if field.InputType == "interval" {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 12, "<!-- Interval field with number input + unit dropdown --> <div class=\"mb-3\"><label for=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 14, "<!-- Interval field with number input + unit dropdown --> <div class=\"mb-3\"><label for=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var8 string
templ_7745c5c3_Var8, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
var templ_7745c5c3_Var10 string
templ_7745c5c3_Var10, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 197, Col: 39}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 198, Col: 39}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var8))
_, 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, "\" class=\"form-label\">")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "\" class=\"form-label\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var9 string
templ_7745c5c3_Var9, templ_7745c5c3_Err = templ.JoinStringErrs(field.DisplayName)
var templ_7745c5c3_Var11 string
templ_7745c5c3_Var11, templ_7745c5c3_Err = templ.JoinStringErrs(field.DisplayName)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 198, Col: 35}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 199, Col: 35}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var9))
_, 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, " ")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, " ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Required {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "<span class=\"text-danger\">*</span>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, "<span class=\"text-danger\">*</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, "</label><div class=\"input-group\"><input type=\"number\" class=\"form-control\" id=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 18, "</label><div class=\"input-group\"><input type=\"number\" class=\"form-control\" id=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var10 string
templ_7745c5c3_Var10, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName + "_value")
var templ_7745c5c3_Var12 string
templ_7745c5c3_Var12, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName + "_value")
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 207, Col: 50}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 208, Col: 50}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var10))
_, 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, 17, "\" name=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, "\" name=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var11 string
templ_7745c5c3_Var11, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName + "_value")
var templ_7745c5c3_Var13 string
templ_7745c5c3_Var13, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName + "_value")
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 208, Col: 52}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 209, Col: 52}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var11))
_, 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, 18, "\" value=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, "\" value=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var12 string
templ_7745c5c3_Var12, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%.0f", components.ConvertInt32SecondsToDisplayValue(getTaskConfigInt32Field(config, field.JSONName))))
var templ_7745c5c3_Var14 string
templ_7745c5c3_Var14, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%.0f", components.ConvertInt32SecondsToDisplayValue(getTaskConfigInt32Field(config, field.JSONName))))
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 209, Col: 142}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 210, Col: 142}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var12))
_, 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, 19, "\" step=\"1\" min=\"1\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, "\" step=\"1\" min=\"1\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Required {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, " required")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 22, " required")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, "> <select class=\"form-select\" id=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 23, "> <select class=\"form-select\" id=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var13 string
templ_7745c5c3_Var13, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName + "_unit")
var templ_7745c5c3_Var15 string
templ_7745c5c3_Var15, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName + "_unit")
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 218, Col: 49}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 219, Col: 49}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var13))
_, 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, "\" name=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, "\" name=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var14 string
templ_7745c5c3_Var14, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName + "_unit")
var templ_7745c5c3_Var16 string
templ_7745c5c3_Var16, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName + "_unit")
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 219, Col: 51}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 220, Col: 51}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var14))
_, 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, "\" style=\"max-width: 120px;\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 25, "\" style=\"max-width: 120px;\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Required {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, " required")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, " required")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 25, "><option value=\"minutes\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, "><option value=\"minutes\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if components.GetInt32DisplayUnit(getTaskConfigInt32Field(config, field.JSONName)) == "minutes" {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, " selected")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, " selected")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, ">Minutes</option> <option value=\"hours\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, ">Minutes</option> <option value=\"hours\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if components.GetInt32DisplayUnit(getTaskConfigInt32Field(config, field.JSONName)) == "hours" {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, " selected")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 30, " selected")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, ">Hours</option> <option value=\"days\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 31, ">Hours</option> <option value=\"days\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if components.GetInt32DisplayUnit(getTaskConfigInt32Field(config, field.JSONName)) == "days" {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 30, " selected")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 32, " selected")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 31, ">Days</option></select></div>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 33, ">Days</option></select></div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Description != "" {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 32, "<div class=\"form-text text-muted\">")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 34, "<div class=\"form-text text-muted\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var15 string
templ_7745c5c3_Var15, templ_7745c5c3_Err = templ.JoinStringErrs(field.Description)
var templ_7745c5c3_Var17 string
templ_7745c5c3_Var17, templ_7745c5c3_Err = templ.JoinStringErrs(field.Description)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 252, Col: 69}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 253, Col: 69}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var15))
_, 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, 33, "</div>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 35, "</div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 34, "</div>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 36, "</div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else if field.InputType == "checkbox" {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 35, "<!-- Checkbox field --> <div class=\"mb-3\"><div class=\"form-check form-switch\"><input class=\"form-check-input\" type=\"checkbox\" id=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 37, "<!-- Checkbox field --> <div class=\"mb-3\"><div class=\"form-check form-switch\"><input class=\"form-check-input\" type=\"checkbox\" id=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var16 string
templ_7745c5c3_Var16, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
var templ_7745c5c3_Var18 string
templ_7745c5c3_Var18, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 262, Col: 39}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 263, Col: 39}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var16))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var18))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 36, "\" name=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 38, "\" name=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var17 string
templ_7745c5c3_Var17, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
var templ_7745c5c3_Var19 string
templ_7745c5c3_Var19, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 263, Col: 41}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 264, Col: 41}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var17))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var19))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 37, "\" value=\"on\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 39, "\" value=\"on\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if getTaskConfigBoolField(config, field.JSONName) {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 38, " checked")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 40, " checked")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 39, "> <label class=\"form-check-label\" for=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 41, "> <label class=\"form-check-label\" for=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var18 string
templ_7745c5c3_Var18, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
var templ_7745c5c3_Var20 string
templ_7745c5c3_Var20, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 269, Col: 68}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 270, Col: 68}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var18))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var20))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 40, "\"><strong>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 42, "\"><strong>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var19 string
templ_7745c5c3_Var19, templ_7745c5c3_Err = templ.JoinStringErrs(field.DisplayName)
var templ_7745c5c3_Var21 string
templ_7745c5c3_Var21, templ_7745c5c3_Err = templ.JoinStringErrs(field.DisplayName)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 270, Col: 47}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 271, Col: 47}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var19))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var21))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 41, "</strong></label></div>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 43, "</strong></label></div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Description != "" {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 42, "<div class=\"form-text text-muted\">")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 44, "<div class=\"form-text text-muted\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var20 string
templ_7745c5c3_Var20, templ_7745c5c3_Err = templ.JoinStringErrs(field.Description)
var templ_7745c5c3_Var22 string
templ_7745c5c3_Var22, templ_7745c5c3_Err = templ.JoinStringErrs(field.Description)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 274, Col: 69}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 275, Col: 69}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var20))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var22))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 43, "</div>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 45, "</div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 44, "</div>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 46, "</div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else if field.InputType == "text" {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 45, "<!-- Text field --> <div class=\"mb-3\"><label for=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 47, "<!-- Text field --> <div class=\"mb-3\"><label for=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var21 string
templ_7745c5c3_Var21, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
var templ_7745c5c3_Var23 string
templ_7745c5c3_Var23, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 280, Col: 39}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 281, Col: 39}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var21))
_, 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, 46, "\" class=\"form-label\">")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 48, "\" class=\"form-label\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var22 string
templ_7745c5c3_Var22, templ_7745c5c3_Err = templ.JoinStringErrs(field.DisplayName)
var templ_7745c5c3_Var24 string
templ_7745c5c3_Var24, templ_7745c5c3_Err = templ.JoinStringErrs(field.DisplayName)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 281, Col: 35}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 282, Col: 35}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var22))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var24))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 47, " ")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 49, " ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Required {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 48, "<span class=\"text-danger\">*</span>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 50, "<span class=\"text-danger\">*</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 49, "</label> <input type=\"text\" class=\"form-control\" id=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 51, "</label> <input type=\"text\" class=\"form-control\" id=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var23 string
templ_7745c5c3_Var23, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
var templ_7745c5c3_Var25 string
templ_7745c5c3_Var25, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 289, Col: 35}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 290, Col: 35}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var23))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var25))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 50, "\" name=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 52, "\" name=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var24 string
templ_7745c5c3_Var24, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
var templ_7745c5c3_Var26 string
templ_7745c5c3_Var26, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 290, Col: 37}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 291, Col: 37}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var24))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var26))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 51, "\" value=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 53, "\" value=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var25 string
templ_7745c5c3_Var25, templ_7745c5c3_Err = templ.JoinStringErrs(getTaskConfigStringField(config, field.JSONName))
var templ_7745c5c3_Var27 string
templ_7745c5c3_Var27, templ_7745c5c3_Err = templ.JoinStringErrs(getTaskConfigStringField(config, field.JSONName))
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 291, Col: 72}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 292, Col: 72}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var25))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var27))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 52, "\" placeholder=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 54, "\" placeholder=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var26 string
templ_7745c5c3_Var26, templ_7745c5c3_Err = templ.JoinStringErrs(field.Placeholder)
var templ_7745c5c3_Var28 string
templ_7745c5c3_Var28, templ_7745c5c3_Err = templ.JoinStringErrs(field.Placeholder)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 292, Col: 47}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 293, Col: 47}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var26))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var28))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 53, "\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 55, "\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Required {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 54, " required")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 56, " required")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 55, "> ")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 57, "> ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Description != "" {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 56, "<div class=\"form-text text-muted\">")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 58, "<div class=\"form-text text-muted\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var27 string
templ_7745c5c3_Var27, templ_7745c5c3_Err = templ.JoinStringErrs(field.Description)
var templ_7745c5c3_Var29 string
templ_7745c5c3_Var29, templ_7745c5c3_Err = templ.JoinStringErrs(field.Description)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 298, Col: 69}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 299, Col: 69}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var27))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var29))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 57, "</div>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 59, "</div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 58, "</div>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 60, "</div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 59, "<!-- Number field --> <div class=\"mb-3\"><label for=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 61, "<!-- Number field --> <div class=\"mb-3\"><label for=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var28 string
templ_7745c5c3_Var28, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
var templ_7745c5c3_Var30 string
templ_7745c5c3_Var30, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 304, Col: 39}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 305, Col: 39}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var28))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var30))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 60, "\" class=\"form-label\">")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 62, "\" class=\"form-label\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var29 string
templ_7745c5c3_Var29, templ_7745c5c3_Err = templ.JoinStringErrs(field.DisplayName)
var templ_7745c5c3_Var31 string
templ_7745c5c3_Var31, templ_7745c5c3_Err = templ.JoinStringErrs(field.DisplayName)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 305, Col: 35}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 306, Col: 35}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var29))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var31))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 61, " ")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 63, " ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Required {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 62, "<span class=\"text-danger\">*</span>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 64, "<span class=\"text-danger\">*</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 63, "</label> <input type=\"number\" class=\"form-control\" id=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 65, "</label> <input type=\"number\" class=\"form-control\" id=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var30 string
templ_7745c5c3_Var30, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
var templ_7745c5c3_Var32 string
templ_7745c5c3_Var32, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 313, Col: 35}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 314, Col: 35}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var30))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var32))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 64, "\" name=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 66, "\" name=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var31 string
templ_7745c5c3_Var31, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
var templ_7745c5c3_Var33 string
templ_7745c5c3_Var33, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 314, Col: 37}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 315, Col: 37}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var31))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var33))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 65, "\" value=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 67, "\" value=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var32 string
templ_7745c5c3_Var32, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%.6g", getTaskConfigFloatField(config, field.JSONName)))
var templ_7745c5c3_Var34 string
templ_7745c5c3_Var34, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%.6g", getTaskConfigFloatField(config, field.JSONName)))
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 315, Col: 92}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 316, Col: 92}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var32))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var34))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 66, "\" placeholder=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 68, "\" placeholder=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var33 string
templ_7745c5c3_Var33, templ_7745c5c3_Err = templ.JoinStringErrs(field.Placeholder)
var templ_7745c5c3_Var35 string
templ_7745c5c3_Var35, templ_7745c5c3_Err = templ.JoinStringErrs(field.Placeholder)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 316, Col: 47}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 317, Col: 47}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var33))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var35))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 67, "\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 69, "\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.MinValue != nil {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 68, " min=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 70, " min=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var34 string
templ_7745c5c3_Var34, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%v", field.MinValue))
var templ_7745c5c3_Var36 string
templ_7745c5c3_Var36, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%v", field.MinValue))
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 318, Col: 59}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 319, Col: 59}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var34))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var36))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 69, "\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 71, "\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
if field.MaxValue != nil {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 70, " max=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 72, " max=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var35 string
templ_7745c5c3_Var35, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%v", field.MaxValue))
var templ_7745c5c3_Var37 string
templ_7745c5c3_Var37, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%v", field.MaxValue))
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 321, Col: 59}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 322, Col: 59}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var35))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var37))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 71, "\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 73, "\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 72, " step=\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 74, " step=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var36 string
templ_7745c5c3_Var36, templ_7745c5c3_Err = templ.JoinStringErrs(getTaskNumberStep(field))
var templ_7745c5c3_Var38 string
templ_7745c5c3_Var38, templ_7745c5c3_Err = templ.JoinStringErrs(getTaskNumberStep(field))
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 323, Col: 47}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 324, Col: 47}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var36))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var38))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 73, "\"")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 75, "\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Required {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 74, " required")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 76, " required")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 75, "> ")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 77, "> ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Description != "" {
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 76, "<div class=\"form-text text-muted\">")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 78, "<div class=\"form-text text-muted\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var37 string
templ_7745c5c3_Var37, templ_7745c5c3_Err = templ.JoinStringErrs(field.Description)
var templ_7745c5c3_Var39 string
templ_7745c5c3_Var39, templ_7745c5c3_Err = templ.JoinStringErrs(field.Description)
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 329, Col: 69}
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 330, Col: 69}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var37))
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var39))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 77, "</div>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 79, "</div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 78, "</div>")
templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 80, "</div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}

1118
weed/admin/view/app/task_detail.templ
File diff suppressed because it is too large
View File

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

167
weed/pb/worker.proto

@ -94,21 +94,23 @@ message TaskAssignment {
// TaskParams contains task-specific parameters with typed variants
message TaskParams {
string task_id = 12; // ActiveTopology task ID for lifecycle management
uint32 volume_id = 1;
string server = 2;
string collection = 3;
string data_center = 4;
string rack = 5;
repeated string replicas = 6;
uint64 volume_size = 11; // Original volume size in bytes for tracking size changes
string task_id = 1; // ActiveTopology task ID for lifecycle management
uint32 volume_id = 2; // Primary volume ID for the task
string collection = 3; // Collection name
string data_center = 4; // Primary data center
string rack = 5; // Primary rack
uint64 volume_size = 6; // Original volume size in bytes for tracking size changes
// Unified source and target arrays for all task types
repeated TaskSource sources = 7; // Source locations (volume replicas, EC shards, etc.)
repeated TaskTarget targets = 8; // Target locations (destinations, new replicas, etc.)
// Typed task parameters
oneof task_params {
VacuumTaskParams vacuum_params = 7;
ErasureCodingTaskParams erasure_coding_params = 8;
BalanceTaskParams balance_params = 9;
ReplicationTaskParams replication_params = 10;
VacuumTaskParams vacuum_params = 9;
ErasureCodingTaskParams erasure_coding_params = 10;
BalanceTaskParams balance_params = 11;
ReplicationTaskParams replication_params = 12;
}
}
@ -123,54 +125,48 @@ message VacuumTaskParams {
// ErasureCodingTaskParams for EC encoding operations
message ErasureCodingTaskParams {
uint64 estimated_shard_size = 3; // Estimated size per shard
int32 data_shards = 4; // Number of data shards (default: 10)
int32 parity_shards = 5; // Number of parity shards (default: 4)
string working_dir = 6; // Working directory for EC processing
string master_client = 7; // Master server address
bool cleanup_source = 8; // Whether to cleanup source volume after EC
repeated string placement_conflicts = 9; // Any placement rule conflicts
repeated ECDestination destinations = 10; // Planned destinations with disk information
repeated ExistingECShardLocation existing_shard_locations = 11; // Existing EC shards to cleanup
}
// ECDestination represents a planned destination for EC shards with disk information
message ECDestination {
string node = 1; // Target server address
uint32 disk_id = 2; // Target disk ID
string rack = 3; // Target rack for placement tracking
string data_center = 4; // Target data center for placement tracking
double placement_score = 5; // Quality score of the placement
uint64 estimated_shard_size = 1; // Estimated size per shard
int32 data_shards = 2; // Number of data shards (default: 10)
int32 parity_shards = 3; // Number of parity shards (default: 4)
string working_dir = 4; // Working directory for EC processing
string master_client = 5; // Master server address
bool cleanup_source = 6; // Whether to cleanup source volume after EC
}
// ExistingECShardLocation represents existing EC shards that need cleanup
message ExistingECShardLocation {
string node = 1; // Server address with existing shards
repeated uint32 shard_ids = 2; // List of shard IDs on this server
// TaskSource represents a unified source location for any task type
message TaskSource {
string node = 1; // Source server address
uint32 disk_id = 2; // Source disk ID
string rack = 3; // Source rack for tracking
string data_center = 4; // Source data center for tracking
uint32 volume_id = 5; // Volume ID (for volume operations)
repeated uint32 shard_ids = 6; // Shard IDs (for EC shard operations)
uint64 estimated_size = 7; // Estimated size to be processed
}
// TaskTarget represents a unified target location for any task type
message TaskTarget {
string node = 1; // Target server address
uint32 disk_id = 2; // Target disk ID
string rack = 3; // Target rack for tracking
string data_center = 4; // Target data center for tracking
uint32 volume_id = 5; // Volume ID (for volume operations)
repeated uint32 shard_ids = 6; // Shard IDs (for EC shard operations)
uint64 estimated_size = 7; // Estimated size to be created
}
// BalanceTaskParams for volume balancing operations
message BalanceTaskParams {
string dest_node = 1; // Planned destination node
uint64 estimated_size = 2; // Estimated volume size
string dest_rack = 3; // Destination rack for placement rules
string dest_dc = 4; // Destination data center
double placement_score = 5; // Quality score of the planned placement
repeated string placement_conflicts = 6; // Any placement rule conflicts
bool force_move = 7; // Force move even with conflicts
int32 timeout_seconds = 8; // Operation timeout
bool force_move = 1; // Force move even with conflicts
int32 timeout_seconds = 2; // Operation timeout
}
// ReplicationTaskParams for adding replicas
message ReplicationTaskParams {
string dest_node = 1; // Planned destination node for new replica
uint64 estimated_size = 2; // Estimated replica size
string dest_rack = 3; // Destination rack for placement rules
string dest_dc = 4; // Destination data center
double placement_score = 5; // Quality score of the planned placement
repeated string placement_conflicts = 6; // Any placement rule conflicts
int32 replica_count = 7; // Target replica count
bool verify_consistency = 8; // Verify replica consistency after creation
int32 replica_count = 1; // Target replica count
bool verify_consistency = 2; // Verify replica consistency after creation
}
// TaskUpdate reports task progress
@ -329,4 +325,75 @@ message BalanceTaskConfig {
// ReplicationTaskConfig contains replication-specific configuration
message ReplicationTaskConfig {
int32 target_replica_count = 1; // Target number of replicas
}
// ========== Task Persistence Messages ==========
// MaintenanceTaskData represents complete task state for persistence
message MaintenanceTaskData {
string id = 1;
string type = 2;
string priority = 3;
string status = 4;
uint32 volume_id = 5;
string server = 6;
string collection = 7;
TaskParams typed_params = 8;
string reason = 9;
int64 created_at = 10;
int64 scheduled_at = 11;
int64 started_at = 12;
int64 completed_at = 13;
string worker_id = 14;
string error = 15;
double progress = 16;
int32 retry_count = 17;
int32 max_retries = 18;
// Enhanced fields for detailed task tracking
string created_by = 19;
string creation_context = 20;
repeated TaskAssignmentRecord assignment_history = 21;
string detailed_reason = 22;
map<string, string> tags = 23;
TaskCreationMetrics creation_metrics = 24;
}
// TaskAssignmentRecord tracks worker assignments for a task
message TaskAssignmentRecord {
string worker_id = 1;
string worker_address = 2;
int64 assigned_at = 3;
int64 unassigned_at = 4; // Optional: when worker was unassigned
string reason = 5; // Reason for assignment/unassignment
}
// TaskCreationMetrics tracks why and how a task was created
message TaskCreationMetrics {
string trigger_metric = 1; // Name of metric that triggered creation
double metric_value = 2; // Value that triggered creation
double threshold = 3; // Threshold that was exceeded
VolumeHealthMetrics volume_metrics = 4; // Volume health at creation time
map<string, string> additional_data = 5; // Additional context data
}
// VolumeHealthMetrics captures volume state at task creation
message VolumeHealthMetrics {
uint64 total_size = 1;
uint64 used_size = 2;
uint64 garbage_size = 3;
double garbage_ratio = 4;
int32 file_count = 5;
int32 deleted_file_count = 6;
int64 last_modified = 7;
int32 replica_count = 8;
bool is_ec_volume = 9;
string collection = 10;
}
// TaskStateFile wraps task data with metadata for persistence
message TaskStateFile {
MaintenanceTaskData task = 1;
int64 last_updated = 2;
string admin_version = 3;
}

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

10
weed/worker/client.go

@ -651,7 +651,7 @@ func (c *GrpcAdminClient) RequestTask(workerID string, capabilities []types.Task
Type: types.TaskType(taskAssign.TaskType),
Status: types.TaskStatusAssigned,
VolumeID: taskAssign.Params.VolumeId,
Server: taskAssign.Params.Server,
Server: getServerFromParams(taskAssign.Params),
Collection: taskAssign.Params.Collection,
Priority: types.TaskPriority(taskAssign.Priority),
CreatedAt: time.Unix(taskAssign.CreatedTime, 0),
@ -934,3 +934,11 @@ func (m *MockAdminClient) AddMockTask(task *types.TaskInput) {
func CreateAdminClient(adminServer string, workerID string, dialOption grpc.DialOption) (AdminClient, error) {
return NewGrpcAdminClient(adminServer, workerID, dialOption), nil
}
// getServerFromParams extracts server address from unified sources
func getServerFromParams(params *worker_pb.TaskParams) string {
if len(params.Sources) > 0 {
return params.Sources[0].Node
}
return ""
}

85
weed/worker/log_adapter.go

@ -0,0 +1,85 @@
package worker
import (
"fmt"
wtasks "github.com/seaweedfs/seaweedfs/weed/worker/tasks"
wtypes "github.com/seaweedfs/seaweedfs/weed/worker/types"
)
// taskLoggerAdapter adapts a tasks.TaskLogger to the types.Logger interface used by tasks
// so that structured WithFields logs from task implementations are captured into file logs.
type taskLoggerAdapter struct {
base wtasks.TaskLogger
fields map[string]interface{}
}
func newTaskLoggerAdapter(base wtasks.TaskLogger) *taskLoggerAdapter {
return &taskLoggerAdapter{base: base}
}
// WithFields returns a new adapter instance that includes the provided fields.
func (a *taskLoggerAdapter) WithFields(fields map[string]interface{}) wtypes.Logger {
// copy fields to avoid mutation by caller
copied := make(map[string]interface{}, len(fields))
for k, v := range fields {
copied[k] = v
}
return &taskLoggerAdapter{base: a.base, fields: copied}
}
// Info logs an info message, including any structured fields if present.
func (a *taskLoggerAdapter) Info(msg string, args ...interface{}) {
if a.base == nil {
return
}
if len(a.fields) > 0 {
a.base.LogWithFields("INFO", fmt.Sprintf(msg, args...), toStringMap(a.fields))
return
}
a.base.Info(msg, args...)
}
func (a *taskLoggerAdapter) Warning(msg string, args ...interface{}) {
if a.base == nil {
return
}
if len(a.fields) > 0 {
a.base.LogWithFields("WARNING", fmt.Sprintf(msg, args...), toStringMap(a.fields))
return
}
a.base.Warning(msg, args...)
}
func (a *taskLoggerAdapter) Error(msg string, args ...interface{}) {
if a.base == nil {
return
}
if len(a.fields) > 0 {
a.base.LogWithFields("ERROR", fmt.Sprintf(msg, args...), toStringMap(a.fields))
return
}
a.base.Error(msg, args...)
}
func (a *taskLoggerAdapter) Debug(msg string, args ...interface{}) {
if a.base == nil {
return
}
if len(a.fields) > 0 {
a.base.LogWithFields("DEBUG", fmt.Sprintf(msg, args...), toStringMap(a.fields))
return
}
a.base.Debug(msg, args...)
}
// toStringMap converts map[string]interface{} to map[string]interface{} where values are printable.
// The underlying tasks.TaskLogger handles arbitrary JSON values, but our gRPC conversion later
// expects strings; we rely on existing conversion there. Here we keep interface{} to preserve detail.
func toStringMap(in map[string]interface{}) map[string]interface{} {
out := make(map[string]interface{}, len(in))
for k, v := range in {
out[k] = v
}
return out
}

31
weed/worker/tasks/balance/balance_task.go

@ -48,21 +48,32 @@ func (t *BalanceTask) Execute(ctx context.Context, params *worker_pb.TaskParams)
return fmt.Errorf("balance parameters are required")
}
// Get planned destination
destNode := balanceParams.DestNode
// Get source and destination from unified arrays
if len(params.Sources) == 0 {
return fmt.Errorf("source is required for balance task")
}
if len(params.Targets) == 0 {
return fmt.Errorf("target is required for balance task")
}
sourceNode := params.Sources[0].Node
destNode := params.Targets[0].Node
if sourceNode == "" {
return fmt.Errorf("source node is required for balance task")
}
if destNode == "" {
return fmt.Errorf("destination node is required for balance task")
}
t.GetLogger().WithFields(map[string]interface{}{
"volume_id": t.volumeID,
"source": t.server,
"source": sourceNode,
"destination": destNode,
"collection": t.collection,
}).Info("Starting balance task - moving volume")
sourceServer := pb.ServerAddress(t.server)
sourceServer := pb.ServerAddress(sourceNode)
targetServer := pb.ServerAddress(destNode)
volumeId := needle.VolumeId(t.volumeID)
@ -130,8 +141,16 @@ func (t *BalanceTask) Validate(params *worker_pb.TaskParams) error {
return fmt.Errorf("volume ID mismatch: expected %d, got %d", t.volumeID, params.VolumeId)
}
if params.Server != t.server {
return fmt.Errorf("source server mismatch: expected %s, got %s", t.server, params.Server)
// Validate that at least one source matches our server
found := false
for _, source := range params.Sources {
if source.Node == t.server {
found = true
break
}
}
if !found {
return fmt.Errorf("no source matches expected server %s", t.server)
}
return nil

66
weed/worker/tasks/balance/detection.go

@ -105,36 +105,54 @@ func Detection(metrics []*types.VolumeHealthMetrics, clusterInfo *types.ClusterI
return nil, nil // Skip this task if destination planning fails
}
// Create typed parameters with destination information
// Find the actual disk containing the volume on the source server
sourceDisk, found := base.FindVolumeDisk(clusterInfo.ActiveTopology, selectedVolume.VolumeID, selectedVolume.Collection, selectedVolume.Server)
if !found {
return nil, fmt.Errorf("BALANCE: Could not find volume %d (collection: %s) on source server %s - unable to create balance task",
selectedVolume.VolumeID, selectedVolume.Collection, selectedVolume.Server)
}
// Create typed parameters with unified source and target information
task.TypedParams = &worker_pb.TaskParams{
TaskId: taskID, // Link to ActiveTopology pending task
VolumeId: selectedVolume.VolumeID,
Server: selectedVolume.Server,
Collection: selectedVolume.Collection,
VolumeSize: selectedVolume.Size, // Store original volume size for tracking changes
// Unified sources and targets - the only way to specify locations
Sources: []*worker_pb.TaskSource{
{
Node: selectedVolume.Server,
DiskId: sourceDisk,
VolumeId: selectedVolume.VolumeID,
EstimatedSize: selectedVolume.Size,
DataCenter: selectedVolume.DataCenter,
Rack: selectedVolume.Rack,
},
},
Targets: []*worker_pb.TaskTarget{
{
Node: destinationPlan.TargetNode,
DiskId: destinationPlan.TargetDisk,
VolumeId: selectedVolume.VolumeID,
EstimatedSize: destinationPlan.ExpectedSize,
DataCenter: destinationPlan.TargetDC,
Rack: destinationPlan.TargetRack,
},
},
TaskParams: &worker_pb.TaskParams_BalanceParams{
BalanceParams: &worker_pb.BalanceTaskParams{
DestNode: destinationPlan.TargetNode,
EstimatedSize: destinationPlan.ExpectedSize,
PlacementScore: destinationPlan.PlacementScore,
PlacementConflicts: destinationPlan.Conflicts,
ForceMove: false,
TimeoutSeconds: 600, // 10 minutes default
ForceMove: false,
TimeoutSeconds: 600, // 10 minutes default
},
},
}
glog.V(1).Infof("Planned balance destination for volume %d: %s -> %s (score: %.2f)",
selectedVolume.VolumeID, selectedVolume.Server, destinationPlan.TargetNode, destinationPlan.PlacementScore)
glog.V(1).Infof("Planned balance destination for volume %d: %s -> %s",
selectedVolume.VolumeID, selectedVolume.Server, destinationPlan.TargetNode)
// Add pending balance task to ActiveTopology for capacity management
// Find the actual disk containing the volume on the source server
sourceDisk, found := base.FindVolumeDisk(clusterInfo.ActiveTopology, selectedVolume.VolumeID, selectedVolume.Collection, selectedVolume.Server)
if !found {
return nil, fmt.Errorf("BALANCE: Could not find volume %d (collection: %s) on source server %s - unable to create balance task",
selectedVolume.VolumeID, selectedVolume.Collection, selectedVolume.Server)
}
targetDisk := destinationPlan.TargetDisk
err = clusterInfo.ActiveTopology.AddPendingTask(topology.TaskSpec{
@ -220,7 +238,6 @@ func planBalanceDestination(activeTopology *topology.ActiveTopology, selectedVol
TargetDC: bestDisk.DataCenter,
ExpectedSize: selectedVolume.Size,
PlacementScore: bestScore,
Conflicts: checkPlacementConflicts(bestDisk, sourceRack, sourceDC),
}, nil
}
@ -253,16 +270,3 @@ func calculateBalanceScore(disk *topology.DiskInfo, sourceRack, sourceDC string,
return score
}
// checkPlacementConflicts checks for placement rule conflicts
func checkPlacementConflicts(disk *topology.DiskInfo, sourceRack, sourceDC string) []string {
var conflicts []string
// For now, implement basic conflict detection
// This could be extended with more sophisticated placement rules
if disk.Rack == sourceRack && disk.DataCenter == sourceDC {
conflicts = append(conflicts, "same_rack_as_source")
}
return conflicts
}

72
weed/worker/tasks/balance/execution.go

@ -15,15 +15,13 @@ type TypedTask struct {
*base.BaseTypedTask
// Task state from protobuf
sourceServer string
destNode string
volumeID uint32
collection string
estimatedSize uint64
placementScore float64
forceMove bool
timeoutSeconds int32
placementConflicts []string
sourceServer string
destNode string
volumeID uint32
collection string
estimatedSize uint64
forceMove bool
timeoutSeconds int32
}
// NewTypedTask creates a new typed balance task
@ -47,14 +45,20 @@ func (t *TypedTask) ValidateTyped(params *worker_pb.TaskParams) error {
return fmt.Errorf("balance_params is required for balance task")
}
// Validate destination node
if balanceParams.DestNode == "" {
return fmt.Errorf("dest_node is required for balance task")
// Validate sources and targets
if len(params.Sources) == 0 {
return fmt.Errorf("at least one source is required for balance task")
}
if len(params.Targets) == 0 {
return fmt.Errorf("at least one target is required for balance task")
}
// Validate estimated size
if balanceParams.EstimatedSize == 0 {
return fmt.Errorf("estimated_size must be greater than 0")
// Validate that source and target have volume IDs
if params.Sources[0].VolumeId == 0 {
return fmt.Errorf("source volume_id is required for balance task")
}
if params.Targets[0].VolumeId == 0 {
return fmt.Errorf("target volume_id is required for balance task")
}
// Validate timeout
@ -73,10 +77,13 @@ func (t *TypedTask) EstimateTimeTyped(params *worker_pb.TaskParams) time.Duratio
if balanceParams.TimeoutSeconds > 0 {
return time.Duration(balanceParams.TimeoutSeconds) * time.Second
}
}
// Estimate based on volume size (1 minute per GB)
if balanceParams.EstimatedSize > 0 {
gbSize := balanceParams.EstimatedSize / (1024 * 1024 * 1024)
// Estimate based on volume size from sources (1 minute per GB)
if len(params.Sources) > 0 {
source := params.Sources[0]
if source.EstimatedSize > 0 {
gbSize := source.EstimatedSize / (1024 * 1024 * 1024)
return time.Duration(gbSize) * time.Minute
}
}
@ -89,35 +96,30 @@ func (t *TypedTask) EstimateTimeTyped(params *worker_pb.TaskParams) time.Duratio
func (t *TypedTask) ExecuteTyped(params *worker_pb.TaskParams) error {
// Extract basic parameters
t.volumeID = params.VolumeId
t.sourceServer = params.Server
t.collection = params.Collection
// Ensure sources and targets are present (should be guaranteed by validation)
if len(params.Sources) == 0 {
return fmt.Errorf("at least one source is required for balance task (ExecuteTyped)")
}
if len(params.Targets) == 0 {
return fmt.Errorf("at least one target is required for balance task (ExecuteTyped)")
}
// Extract source and target information
t.sourceServer = params.Sources[0].Node
t.estimatedSize = params.Sources[0].EstimatedSize
t.destNode = params.Targets[0].Node
// Extract balance-specific parameters
balanceParams := params.GetBalanceParams()
if balanceParams != nil {
t.destNode = balanceParams.DestNode
t.estimatedSize = balanceParams.EstimatedSize
t.placementScore = balanceParams.PlacementScore
t.forceMove = balanceParams.ForceMove
t.timeoutSeconds = balanceParams.TimeoutSeconds
t.placementConflicts = balanceParams.PlacementConflicts
}
glog.Infof("Starting typed balance task for volume %d: %s -> %s (collection: %s, size: %d bytes)",
t.volumeID, t.sourceServer, t.destNode, t.collection, t.estimatedSize)
// Log placement information
if t.placementScore > 0 {
glog.V(1).Infof("Placement score: %.2f", t.placementScore)
}
if len(t.placementConflicts) > 0 {
glog.V(1).Infof("Placement conflicts: %v", t.placementConflicts)
if !t.forceMove {
return fmt.Errorf("placement conflicts detected and force_move is false: %v", t.placementConflicts)
}
glog.Warningf("Proceeding with balance despite conflicts (force_move=true): %v", t.placementConflicts)
}
// Simulate balance operation with progress updates
steps := []struct {
name string

5
weed/worker/tasks/balance/register.go

@ -42,9 +42,12 @@ func RegisterBalanceTask() {
if params == nil {
return nil, fmt.Errorf("task parameters are required")
}
if len(params.Sources) == 0 {
return nil, fmt.Errorf("at least one source is required for balance task")
}
return NewBalanceTask(
fmt.Sprintf("balance-%d", params.VolumeId),
params.Server,
params.Sources[0].Node, // Use first source node
params.VolumeId,
params.Collection,
), nil

39
weed/worker/tasks/base/typed_task.go

@ -16,7 +16,8 @@ type BaseTypedTask struct {
taskType types.TaskType
taskID string
progress float64
progressCallback func(float64)
progressCallback func(float64, string)
currentStage string
cancelled bool
mutex sync.RWMutex
@ -75,21 +76,49 @@ func (bt *BaseTypedTask) GetProgress() float64 {
func (bt *BaseTypedTask) SetProgress(progress float64) {
bt.mutex.Lock()
callback := bt.progressCallback
stage := bt.currentStage
bt.progress = progress
bt.mutex.Unlock()
if callback != nil {
callback(progress)
callback(progress, stage)
}
}
// SetProgressCallback sets the progress callback function
func (bt *BaseTypedTask) SetProgressCallback(callback func(float64)) {
func (bt *BaseTypedTask) SetProgressCallback(callback func(float64, string)) {
bt.mutex.Lock()
defer bt.mutex.Unlock()
bt.progressCallback = callback
}
// SetProgressWithStage sets the current progress with a stage description
func (bt *BaseTypedTask) SetProgressWithStage(progress float64, stage string) {
bt.mutex.Lock()
callback := bt.progressCallback
bt.progress = progress
bt.currentStage = stage
bt.mutex.Unlock()
if callback != nil {
callback(progress, stage)
}
}
// SetCurrentStage sets the current stage description
func (bt *BaseTypedTask) SetCurrentStage(stage string) {
bt.mutex.Lock()
defer bt.mutex.Unlock()
bt.currentStage = stage
}
// GetCurrentStage returns the current stage description
func (bt *BaseTypedTask) GetCurrentStage() string {
bt.mutex.RLock()
defer bt.mutex.RUnlock()
return bt.currentStage
}
// SetLoggerConfig sets the logger configuration for this task
func (bt *BaseTypedTask) SetLoggerConfig(config types.TaskLoggerConfig) {
bt.mutex.Lock()
@ -200,8 +229,8 @@ func (bt *BaseTypedTask) ValidateTyped(params *worker_pb.TaskParams) error {
if params.VolumeId == 0 {
return errors.New("volume_id is required")
}
if params.Server == "" {
return errors.New("server is required")
if len(params.Sources) == 0 {
return errors.New("at least one source is required")
}
return nil
}

194
weed/worker/tasks/erasure_coding/detection.go

@ -61,6 +61,8 @@ func Detection(metrics []*types.VolumeHealthMetrics, clusterInfo *types.ClusterI
// Check quiet duration and fullness criteria
if metric.Age >= quietThreshold && metric.FullnessRatio >= ecConfig.FullnessRatio {
glog.Infof("EC Detection: Volume %d meets all criteria, attempting to create task", metric.VolumeID)
// Generate task ID for ActiveTopology integration
taskID := fmt.Sprintf("ec_vol_%d_%d", metric.VolumeID, now.Unix())
@ -79,11 +81,13 @@ func Detection(metrics []*types.VolumeHealthMetrics, clusterInfo *types.ClusterI
// Plan EC destinations if ActiveTopology is available
if clusterInfo.ActiveTopology != nil {
glog.Infof("EC Detection: ActiveTopology available, planning destinations for volume %d", metric.VolumeID)
multiPlan, err := planECDestinations(clusterInfo.ActiveTopology, metric, ecConfig)
if err != nil {
glog.Warningf("Failed to plan EC destinations for volume %d: %v", metric.VolumeID, err)
continue // Skip this volume if destination planning fails
}
glog.Infof("EC Detection: Successfully planned %d destinations for volume %d", len(multiPlan.Plans), metric.VolumeID)
// Calculate expected shard size for EC operation
// Each data shard will be approximately volumeSize / dataShards
@ -100,23 +104,27 @@ func Detection(metrics []*types.VolumeHealthMetrics, clusterInfo *types.ClusterI
}
// Find all volume replica locations (server + disk) from topology
glog.Infof("EC Detection: Looking for replica locations for volume %d", metric.VolumeID)
replicaLocations := findVolumeReplicaLocations(clusterInfo.ActiveTopology, metric.VolumeID, metric.Collection)
if len(replicaLocations) == 0 {
glog.Warningf("No replica locations found for volume %d, skipping EC", metric.VolumeID)
continue
}
glog.Infof("EC Detection: Found %d replica locations for volume %d", len(replicaLocations), metric.VolumeID)
// Find existing EC shards from previous failed attempts
existingECShards := findExistingECShards(clusterInfo.ActiveTopology, metric.VolumeID, metric.Collection)
// Combine volume replicas and existing EC shards for cleanup
var allSourceLocations []topology.TaskSourceLocation
var sources []topology.TaskSourceSpec
// Add volume replicas (will free volume slots)
for _, replica := range replicaLocations {
allSourceLocations = append(allSourceLocations, topology.TaskSourceLocation{
sources = append(sources, topology.TaskSourceSpec{
ServerID: replica.ServerID,
DiskID: replica.DiskID,
DataCenter: replica.DataCenter,
Rack: replica.Rack,
CleanupType: topology.CleanupVolumeReplica,
})
}
@ -131,9 +139,11 @@ func Detection(metrics []*types.VolumeHealthMetrics, clusterInfo *types.ClusterI
for _, shard := range existingECShards {
key := fmt.Sprintf("%s:%d", shard.ServerID, shard.DiskID)
if !duplicateCheck[key] { // Avoid duplicates if EC shards are on same disk as volume replicas
allSourceLocations = append(allSourceLocations, topology.TaskSourceLocation{
sources = append(sources, topology.TaskSourceSpec{
ServerID: shard.ServerID,
DiskID: shard.DiskID,
DataCenter: shard.DataCenter,
Rack: shard.Rack,
CleanupType: topology.CleanupECShards,
})
duplicateCheck[key] = true
@ -141,17 +151,7 @@ func Detection(metrics []*types.VolumeHealthMetrics, clusterInfo *types.ClusterI
}
glog.V(2).Infof("Found %d volume replicas and %d existing EC shards for volume %d (total %d cleanup sources)",
len(replicaLocations), len(existingECShards), metric.VolumeID, len(allSourceLocations))
// Convert TaskSourceLocation to TaskSourceSpec
sources := make([]topology.TaskSourceSpec, len(allSourceLocations))
for i, srcLoc := range allSourceLocations {
sources[i] = topology.TaskSourceSpec{
ServerID: srcLoc.ServerID,
DiskID: srcLoc.DiskID,
CleanupType: srcLoc.CleanupType,
}
}
len(replicaLocations), len(existingECShards), metric.VolumeID, len(sources))
// Convert shard destinations to TaskDestinationSpec
destinations := make([]topology.TaskDestinationSpec, len(shardDestinations))
@ -180,27 +180,21 @@ func Detection(metrics []*types.VolumeHealthMetrics, clusterInfo *types.ClusterI
}
glog.V(2).Infof("Added pending EC shard task %s to ActiveTopology for volume %d with %d cleanup sources and %d shard destinations",
taskID, metric.VolumeID, len(allSourceLocations), len(multiPlan.Plans))
// Find all volume replicas from topology (for legacy worker compatibility)
var replicas []string
serverSet := make(map[string]struct{})
for _, loc := range replicaLocations {
if _, found := serverSet[loc.ServerID]; !found {
replicas = append(replicas, loc.ServerID)
serverSet[loc.ServerID] = struct{}{}
}
}
glog.V(1).Infof("Found %d replicas for volume %d: %v", len(replicas), metric.VolumeID, replicas)
taskID, metric.VolumeID, len(sources), len(multiPlan.Plans))
// Create typed parameters with EC destination information and replicas
// Create unified sources and targets for EC task
result.TypedParams = &worker_pb.TaskParams{
TaskId: taskID, // Link to ActiveTopology pending task
VolumeId: metric.VolumeID,
Server: metric.Server,
Collection: metric.Collection,
VolumeSize: metric.Size, // Store original volume size for tracking changes
Replicas: replicas, // Include all volume replicas for deletion
// Unified sources - all sources that will be processed/cleaned up
Sources: convertTaskSourcesToProtobuf(sources, metric.VolumeID),
// Unified targets - all EC shard destinations
Targets: createECTargets(multiPlan),
TaskParams: &worker_pb.TaskParams_ErasureCodingParams{
ErasureCodingParams: createECTaskParams(multiPlan),
},
@ -213,6 +207,7 @@ func Detection(metrics []*types.VolumeHealthMetrics, clusterInfo *types.ClusterI
continue // Skip this volume if no topology available
}
glog.Infof("EC Detection: Successfully created EC task for volume %d, adding to results", metric.VolumeID)
results = append(results, result)
} else {
// Count debug reasons
@ -283,7 +278,8 @@ func planECDestinations(activeTopology *topology.ActiveTopology, metric *types.V
// Get available disks for EC placement with effective capacity consideration (includes pending tasks)
// For EC, we typically need 1 volume slot per shard, so use minimum capacity of 1
// For EC, we need at least 1 available volume slot on a disk to consider it for placement.
availableDisks := activeTopology.GetDisksWithEffectiveCapacity(topology.TaskTypeErasureCoding, metric.Server, 1)
// Note: We don't exclude the source server since the original volume will be deleted after EC conversion
availableDisks := activeTopology.GetDisksWithEffectiveCapacity(topology.TaskTypeErasureCoding, "", 1)
if len(availableDisks) < erasure_coding.MinTotalDisks {
return nil, fmt.Errorf("insufficient disks for EC placement: need %d, have %d (considering pending/active tasks)", erasure_coding.MinTotalDisks, len(availableDisks))
}
@ -306,7 +302,6 @@ func planECDestinations(activeTopology *topology.ActiveTopology, metric *types.V
TargetDC: disk.DataCenter,
ExpectedSize: expectedShardSize, // Set calculated EC shard size
PlacementScore: calculateECScore(disk, sourceRack, sourceDC),
Conflicts: checkECPlacementConflicts(disk, sourceRack, sourceDC),
}
plans = append(plans, plan)
@ -340,32 +335,96 @@ func planECDestinations(activeTopology *topology.ActiveTopology, metric *types.V
}, nil
}
// createECTaskParams creates EC task parameters from the multi-destination plan
func createECTaskParams(multiPlan *topology.MultiDestinationPlan) *worker_pb.ErasureCodingTaskParams {
var destinations []*worker_pb.ECDestination
for _, plan := range multiPlan.Plans {
destination := &worker_pb.ECDestination{
Node: plan.TargetNode,
DiskId: plan.TargetDisk,
Rack: plan.TargetRack,
DataCenter: plan.TargetDC,
PlacementScore: plan.PlacementScore,
// createECTargets creates unified TaskTarget structures from the multi-destination plan
// with proper shard ID assignment during planning phase
func createECTargets(multiPlan *topology.MultiDestinationPlan) []*worker_pb.TaskTarget {
var targets []*worker_pb.TaskTarget
numTargets := len(multiPlan.Plans)
// Create shard assignment arrays for each target (round-robin distribution)
targetShards := make([][]uint32, numTargets)
for i := range targetShards {
targetShards[i] = make([]uint32, 0)
}
// Distribute shards in round-robin fashion to spread both data and parity shards
// This ensures each target gets a mix of data shards (0-9) and parity shards (10-13)
for shardId := uint32(0); shardId < uint32(erasure_coding.TotalShardsCount); shardId++ {
targetIndex := int(shardId) % numTargets
targetShards[targetIndex] = append(targetShards[targetIndex], shardId)
}
// Create targets with assigned shard IDs
for i, plan := range multiPlan.Plans {
target := &worker_pb.TaskTarget{
Node: plan.TargetNode,
DiskId: plan.TargetDisk,
Rack: plan.TargetRack,
DataCenter: plan.TargetDC,
ShardIds: targetShards[i], // Round-robin assigned shards
EstimatedSize: plan.ExpectedSize,
}
targets = append(targets, target)
// Log shard assignment with data/parity classification
dataShards := make([]uint32, 0)
parityShards := make([]uint32, 0)
for _, shardId := range targetShards[i] {
if shardId < uint32(erasure_coding.DataShardsCount) {
dataShards = append(dataShards, shardId)
} else {
parityShards = append(parityShards, shardId)
}
}
destinations = append(destinations, destination)
glog.V(2).Infof("EC planning: target %s assigned shards %v (data: %v, parity: %v)",
plan.TargetNode, targetShards[i], dataShards, parityShards)
}
// Collect placement conflicts from all destinations
var placementConflicts []string
for _, plan := range multiPlan.Plans {
placementConflicts = append(placementConflicts, plan.Conflicts...)
glog.V(1).Infof("EC planning: distributed %d shards across %d targets using round-robin (data shards 0-%d, parity shards %d-%d)",
erasure_coding.TotalShardsCount, numTargets,
erasure_coding.DataShardsCount-1, erasure_coding.DataShardsCount, erasure_coding.TotalShardsCount-1)
return targets
}
// convertTaskSourcesToProtobuf converts topology.TaskSourceSpec to worker_pb.TaskSource
func convertTaskSourcesToProtobuf(sources []topology.TaskSourceSpec, volumeID uint32) []*worker_pb.TaskSource {
var protobufSources []*worker_pb.TaskSource
for _, source := range sources {
pbSource := &worker_pb.TaskSource{
Node: source.ServerID,
DiskId: source.DiskID,
DataCenter: source.DataCenter,
Rack: source.Rack,
}
// Convert storage impact to estimated size
if source.EstimatedSize != nil {
pbSource.EstimatedSize = uint64(*source.EstimatedSize)
}
// Set appropriate volume ID or shard IDs based on cleanup type
switch source.CleanupType {
case topology.CleanupVolumeReplica:
// This is a volume replica, use the actual volume ID
pbSource.VolumeId = volumeID
case topology.CleanupECShards:
// This is EC shards, also use the volume ID for consistency
pbSource.VolumeId = volumeID
// Note: ShardIds would need to be passed separately if we need specific shard info
}
protobufSources = append(protobufSources, pbSource)
}
return protobufSources
}
// createECTaskParams creates clean EC task parameters (destinations now in unified targets)
func createECTaskParams(multiPlan *topology.MultiDestinationPlan) *worker_pb.ErasureCodingTaskParams {
return &worker_pb.ErasureCodingTaskParams{
Destinations: destinations,
DataShards: erasure_coding.DataShardsCount, // Standard data shards
ParityShards: erasure_coding.ParityShardsCount, // Standard parity shards
PlacementConflicts: placementConflicts,
DataShards: erasure_coding.DataShardsCount, // Standard data shards
ParityShards: erasure_coding.ParityShardsCount, // Standard parity shards
}
}
@ -456,25 +515,19 @@ func calculateECScore(disk *topology.DiskInfo, sourceRack, sourceDC string) floa
score := 0.0
// Prefer disks with available capacity
// Prefer disks with available capacity (primary factor)
if disk.DiskInfo.MaxVolumeCount > 0 {
utilization := float64(disk.DiskInfo.VolumeCount) / float64(disk.DiskInfo.MaxVolumeCount)
score += (1.0 - utilization) * 50.0 // Up to 50 points for available capacity
score += (1.0 - utilization) * 60.0 // Up to 60 points for available capacity
}
// Prefer different racks for better distribution
if disk.Rack != sourceRack {
score += 30.0
}
// Prefer different data centers for better distribution
if disk.DataCenter != sourceDC {
score += 20.0
}
// Consider current load
// Consider current load (secondary factor)
score += (10.0 - float64(disk.LoadCount)) // Up to 10 points for low load
// Note: We don't penalize placing shards on the same rack/DC as source
// since the original volume will be deleted after EC conversion.
// This allows for better network efficiency and storage utilization.
return score
}
@ -492,19 +545,6 @@ func isDiskSuitableForEC(disk *topology.DiskInfo) bool {
return true
}
// checkECPlacementConflicts checks for placement rule conflicts in EC operations
func checkECPlacementConflicts(disk *topology.DiskInfo, sourceRack, sourceDC string) []string {
var conflicts []string
// For EC, being on the same rack as source is often acceptable
// but we note it as potential conflict for monitoring
if disk.Rack == sourceRack && disk.DataCenter == sourceDC {
conflicts = append(conflicts, "same_rack_as_source")
}
return conflicts
}
// findVolumeReplicaLocations finds all replica locations (server + disk) for the specified volume
// Uses O(1) indexed lookup for optimal performance on large clusters.
func findVolumeReplicaLocations(activeTopology *topology.ActiveTopology, volumeID uint32, collection string) []topology.VolumeReplica {

368
weed/worker/tasks/erasure_coding/ec_task.go

@ -7,7 +7,6 @@ import (
"math"
"os"
"path/filepath"
"sort"
"strings"
"time"
@ -36,9 +35,9 @@ type ErasureCodingTask struct {
// EC parameters
dataShards int32
parityShards int32
destinations []*worker_pb.ECDestination
shardAssignment map[string][]string // destination -> assigned shard types
replicas []string // volume replica servers for deletion
targets []*worker_pb.TaskTarget // Unified targets for EC shards
sources []*worker_pb.TaskSource // Unified sources for cleanup
shardAssignment map[string][]string // destination -> assigned shard types
}
// NewErasureCodingTask creates a new unified EC task instance
@ -67,18 +66,43 @@ func (t *ErasureCodingTask) Execute(ctx context.Context, params *worker_pb.TaskP
t.dataShards = ecParams.DataShards
t.parityShards = ecParams.ParityShards
t.workDir = ecParams.WorkingDir
t.destinations = ecParams.Destinations
t.replicas = params.Replicas // Get replicas from task parameters
t.targets = params.Targets // Get unified targets
t.sources = params.Sources // Get unified sources
// Log detailed task information
t.GetLogger().WithFields(map[string]interface{}{
"volume_id": t.volumeID,
"server": t.server,
"collection": t.collection,
"data_shards": t.dataShards,
"parity_shards": t.parityShards,
"destinations": len(t.destinations),
"total_shards": t.dataShards + t.parityShards,
"targets": len(t.targets),
"sources": len(t.sources),
}).Info("Starting erasure coding task")
// Log detailed target server assignments
for i, target := range t.targets {
t.GetLogger().WithFields(map[string]interface{}{
"target_index": i,
"server": target.Node,
"shard_ids": target.ShardIds,
"shard_count": len(target.ShardIds),
}).Info("Target server shard assignment")
}
// Log source information
for i, source := range t.sources {
t.GetLogger().WithFields(map[string]interface{}{
"source_index": i,
"server": source.Node,
"volume_id": source.VolumeId,
"disk_id": source.DiskId,
"rack": source.Rack,
"data_center": source.DataCenter,
}).Info("Source server information")
}
// Use the working directory from task parameters, or fall back to a default
baseWorkDir := t.workDir
@ -112,14 +136,14 @@ func (t *ErasureCodingTask) Execute(ctx context.Context, params *worker_pb.TaskP
}()
// Step 1: Mark volume readonly
t.ReportProgress(10.0)
t.ReportProgressWithStage(10.0, "Marking volume readonly")
t.GetLogger().Info("Marking volume readonly")
if err := t.markVolumeReadonly(); err != nil {
return fmt.Errorf("failed to mark volume readonly: %v", err)
}
// Step 2: Copy volume files to worker
t.ReportProgress(25.0)
t.ReportProgressWithStage(25.0, "Copying volume files to worker")
t.GetLogger().Info("Copying volume files to worker")
localFiles, err := t.copyVolumeFilesToWorker(taskWorkDir)
if err != nil {
@ -127,7 +151,7 @@ func (t *ErasureCodingTask) Execute(ctx context.Context, params *worker_pb.TaskP
}
// Step 3: Generate EC shards locally
t.ReportProgress(40.0)
t.ReportProgressWithStage(40.0, "Generating EC shards locally")
t.GetLogger().Info("Generating EC shards locally")
shardFiles, err := t.generateEcShardsLocally(localFiles, taskWorkDir)
if err != nil {
@ -135,27 +159,27 @@ func (t *ErasureCodingTask) Execute(ctx context.Context, params *worker_pb.TaskP
}
// Step 4: Distribute shards to destinations
t.ReportProgress(60.0)
t.ReportProgressWithStage(60.0, "Distributing EC shards to destinations")
t.GetLogger().Info("Distributing EC shards to destinations")
if err := t.distributeEcShards(shardFiles); err != nil {
return fmt.Errorf("failed to distribute EC shards: %v", err)
}
// Step 5: Mount EC shards
t.ReportProgress(80.0)
t.ReportProgressWithStage(80.0, "Mounting EC shards")
t.GetLogger().Info("Mounting EC shards")
if err := t.mountEcShards(); err != nil {
return fmt.Errorf("failed to mount EC shards: %v", err)
}
// Step 6: Delete original volume
t.ReportProgress(90.0)
t.ReportProgressWithStage(90.0, "Deleting original volume")
t.GetLogger().Info("Deleting original volume")
if err := t.deleteOriginalVolume(); err != nil {
return fmt.Errorf("failed to delete original volume: %v", err)
}
t.ReportProgress(100.0)
t.ReportProgressWithStage(100.0, "EC processing complete")
glog.Infof("EC task completed successfully: volume %d from %s with %d shards distributed",
t.volumeID, t.server, len(shardFiles))
@ -177,8 +201,16 @@ func (t *ErasureCodingTask) Validate(params *worker_pb.TaskParams) error {
return fmt.Errorf("volume ID mismatch: expected %d, got %d", t.volumeID, params.VolumeId)
}
if params.Server != t.server {
return fmt.Errorf("source server mismatch: expected %s, got %s", t.server, params.Server)
// Validate that at least one source matches our server
found := false
for _, source := range params.Sources {
if source.Node == t.server {
found = true
break
}
}
if !found {
return fmt.Errorf("no source matches expected server %s", t.server)
}
if ecParams.DataShards < 1 {
@ -189,8 +221,8 @@ func (t *ErasureCodingTask) Validate(params *worker_pb.TaskParams) error {
return fmt.Errorf("invalid parity shards: %d (must be >= 1)", ecParams.ParityShards)
}
if len(ecParams.Destinations) < int(ecParams.DataShards+ecParams.ParityShards) {
return fmt.Errorf("insufficient destinations: got %d, need %d", len(ecParams.Destinations), ecParams.DataShards+ecParams.ParityShards)
if len(params.Targets) < int(ecParams.DataShards+ecParams.ParityShards) {
return fmt.Errorf("insufficient targets: got %d, need %d", len(params.Targets), ecParams.DataShards+ecParams.ParityShards)
}
return nil
@ -224,6 +256,12 @@ func (t *ErasureCodingTask) markVolumeReadonly() error {
func (t *ErasureCodingTask) copyVolumeFilesToWorker(workDir string) (map[string]string, error) {
localFiles := make(map[string]string)
t.GetLogger().WithFields(map[string]interface{}{
"volume_id": t.volumeID,
"source": t.server,
"working_dir": workDir,
}).Info("Starting volume file copy from source server")
// Copy .dat file
datFile := filepath.Join(workDir, fmt.Sprintf("%d.dat", t.volumeID))
if err := t.copyFileFromSource(".dat", datFile); err != nil {
@ -231,6 +269,16 @@ func (t *ErasureCodingTask) copyVolumeFilesToWorker(workDir string) (map[string]
}
localFiles["dat"] = datFile
// Log .dat file size
if info, err := os.Stat(datFile); err == nil {
t.GetLogger().WithFields(map[string]interface{}{
"file_type": ".dat",
"file_path": datFile,
"size_bytes": info.Size(),
"size_mb": float64(info.Size()) / (1024 * 1024),
}).Info("Volume data file copied successfully")
}
// Copy .idx file
idxFile := filepath.Join(workDir, fmt.Sprintf("%d.idx", t.volumeID))
if err := t.copyFileFromSource(".idx", idxFile); err != nil {
@ -238,6 +286,16 @@ func (t *ErasureCodingTask) copyVolumeFilesToWorker(workDir string) (map[string]
}
localFiles["idx"] = idxFile
// Log .idx file size
if info, err := os.Stat(idxFile); err == nil {
t.GetLogger().WithFields(map[string]interface{}{
"file_type": ".idx",
"file_path": idxFile,
"size_bytes": info.Size(),
"size_mb": float64(info.Size()) / (1024 * 1024),
}).Info("Volume index file copied successfully")
}
return localFiles, nil
}
@ -312,18 +370,38 @@ func (t *ErasureCodingTask) generateEcShardsLocally(localFiles map[string]string
return nil, fmt.Errorf("failed to generate .ecx file: %v", err)
}
// Collect generated shard file paths
// Collect generated shard file paths and log details
var generatedShards []string
var totalShardSize int64
for i := 0; i < erasure_coding.TotalShardsCount; i++ {
shardFile := fmt.Sprintf("%s.ec%02d", baseName, i)
if _, err := os.Stat(shardFile); err == nil {
shardFiles[fmt.Sprintf("ec%02d", i)] = shardFile
if info, err := os.Stat(shardFile); err == nil {
shardKey := fmt.Sprintf("ec%02d", i)
shardFiles[shardKey] = shardFile
generatedShards = append(generatedShards, shardKey)
totalShardSize += info.Size()
// Log individual shard details
t.GetLogger().WithFields(map[string]interface{}{
"shard_id": i,
"shard_type": shardKey,
"file_path": shardFile,
"size_bytes": info.Size(),
"size_kb": float64(info.Size()) / 1024,
}).Info("EC shard generated")
}
}
// Add metadata files
ecxFile := baseName + ".ecx"
if _, err := os.Stat(ecxFile); err == nil {
if info, err := os.Stat(ecxFile); err == nil {
shardFiles["ecx"] = ecxFile
t.GetLogger().WithFields(map[string]interface{}{
"file_type": "ecx",
"file_path": ecxFile,
"size_bytes": info.Size(),
}).Info("EC index file generated")
}
// Generate .vif file (volume info)
@ -335,26 +413,67 @@ func (t *ErasureCodingTask) generateEcShardsLocally(localFiles map[string]string
glog.Warningf("Failed to create .vif file: %v", err)
} else {
shardFiles["vif"] = vifFile
if info, err := os.Stat(vifFile); err == nil {
t.GetLogger().WithFields(map[string]interface{}{
"file_type": "vif",
"file_path": vifFile,
"size_bytes": info.Size(),
}).Info("Volume info file generated")
}
}
glog.V(1).Infof("Generated %d EC files locally", len(shardFiles))
// Log summary of generation
t.GetLogger().WithFields(map[string]interface{}{
"total_files": len(shardFiles),
"ec_shards": len(generatedShards),
"generated_shards": generatedShards,
"total_shard_size_mb": float64(totalShardSize) / (1024 * 1024),
}).Info("EC shard generation completed")
return shardFiles, nil
}
// distributeEcShards distributes locally generated EC shards to destination servers
// using pre-assigned shard IDs from planning phase
func (t *ErasureCodingTask) distributeEcShards(shardFiles map[string]string) error {
if len(t.destinations) == 0 {
return fmt.Errorf("no destinations specified for EC shard distribution")
if len(t.targets) == 0 {
return fmt.Errorf("no targets specified for EC shard distribution")
}
if len(shardFiles) == 0 {
return fmt.Errorf("no shard files available for distribution")
}
// Create shard assignment: assign specific shards to specific destinations
shardAssignment := t.createShardAssignment(shardFiles)
// Build shard assignment from pre-assigned target shard IDs (from planning phase)
shardAssignment := make(map[string][]string)
for _, target := range t.targets {
if len(target.ShardIds) == 0 {
continue // Skip targets with no assigned shards
}
var assignedShards []string
// Convert shard IDs to shard file names (e.g., 0 → "ec00", 1 → "ec01")
for _, shardId := range target.ShardIds {
shardType := fmt.Sprintf("ec%02d", shardId)
assignedShards = append(assignedShards, shardType)
}
// Add metadata files (.ecx, .vif) to targets that have shards
if len(assignedShards) > 0 {
if _, hasEcx := shardFiles["ecx"]; hasEcx {
assignedShards = append(assignedShards, "ecx")
}
if _, hasVif := shardFiles["vif"]; hasVif {
assignedShards = append(assignedShards, "vif")
}
}
shardAssignment[target.Node] = assignedShards
}
if len(shardAssignment) == 0 {
return fmt.Errorf("failed to create shard assignment")
return fmt.Errorf("no shard assignments found from planning phase")
}
// Store assignment for use during mounting
@ -365,100 +484,50 @@ func (t *ErasureCodingTask) distributeEcShards(shardFiles map[string]string) err
t.GetLogger().WithFields(map[string]interface{}{
"destination": destNode,
"assigned_shards": len(assignedShards),
"shard_ids": assignedShards,
}).Info("Distributing assigned EC shards to destination")
"shard_types": assignedShards,
}).Info("Starting shard distribution to destination server")
// Send only the assigned shards to this destination
var transferredBytes int64
for _, shardType := range assignedShards {
filePath, exists := shardFiles[shardType]
if !exists {
return fmt.Errorf("shard file %s not found for destination %s", shardType, destNode)
}
// Log file size before transfer
if info, err := os.Stat(filePath); err == nil {
transferredBytes += info.Size()
t.GetLogger().WithFields(map[string]interface{}{
"destination": destNode,
"shard_type": shardType,
"file_path": filePath,
"size_bytes": info.Size(),
"size_kb": float64(info.Size()) / 1024,
}).Info("Starting shard file transfer")
}
if err := t.sendShardFileToDestination(destNode, filePath, shardType); err != nil {
return fmt.Errorf("failed to send %s to %s: %v", shardType, destNode, err)
}
}
}
glog.V(1).Infof("Successfully distributed EC shards to %d destinations", len(shardAssignment))
return nil
}
// createShardAssignment assigns specific EC shards to specific destination servers
// Each destination gets a subset of shards based on availability and placement rules
func (t *ErasureCodingTask) createShardAssignment(shardFiles map[string]string) map[string][]string {
assignment := make(map[string][]string)
// Collect all available EC shards (ec00-ec13)
var availableShards []string
for shardType := range shardFiles {
if strings.HasPrefix(shardType, "ec") && len(shardType) == 4 {
availableShards = append(availableShards, shardType)
}
}
// Sort shards for consistent assignment
sort.Strings(availableShards)
if len(availableShards) == 0 {
glog.Warningf("No EC shards found for assignment")
return assignment
}
// Calculate shards per destination
numDestinations := len(t.destinations)
if numDestinations == 0 {
return assignment
}
// Strategy: Distribute shards as evenly as possible across destinations
// With 14 shards and N destinations, some destinations get ⌈14/N⌉ shards, others get ⌊14/N⌋
shardsPerDest := len(availableShards) / numDestinations
extraShards := len(availableShards) % numDestinations
shardIndex := 0
for i, dest := range t.destinations {
var destShards []string
// Assign base number of shards
shardsToAssign := shardsPerDest
// Assign one extra shard to first 'extraShards' destinations
if i < extraShards {
shardsToAssign++
}
// Assign the shards
for j := 0; j < shardsToAssign && shardIndex < len(availableShards); j++ {
destShards = append(destShards, availableShards[shardIndex])
shardIndex++
t.GetLogger().WithFields(map[string]interface{}{
"destination": destNode,
"shard_type": shardType,
}).Info("Shard file transfer completed")
}
assignment[dest.Node] = destShards
glog.V(2).Infof("Assigned shards %v to destination %s", destShards, dest.Node)
}
// Assign metadata files (.ecx, .vif) to each destination that has shards
// Note: .ecj files are created during mount, not during initial generation
for destNode, destShards := range assignment {
if len(destShards) > 0 {
// Add .ecx file if available
if _, hasEcx := shardFiles["ecx"]; hasEcx {
assignment[destNode] = append(assignment[destNode], "ecx")
}
// Add .vif file if available
if _, hasVif := shardFiles["vif"]; hasVif {
assignment[destNode] = append(assignment[destNode], "vif")
}
glog.V(2).Infof("Assigned metadata files (.ecx, .vif) to destination %s", destNode)
}
// Log summary for this destination
t.GetLogger().WithFields(map[string]interface{}{
"destination": destNode,
"shards_transferred": len(assignedShards),
"total_bytes": transferredBytes,
"total_mb": float64(transferredBytes) / (1024 * 1024),
}).Info("All shards distributed to destination server")
}
return assignment
glog.V(1).Infof("Successfully distributed EC shards to %d destinations", len(shardAssignment))
return nil
}
// sendShardFileToDestination sends a single shard file to a destination server using ReceiveFile API
@ -565,6 +634,8 @@ func (t *ErasureCodingTask) mountEcShards() error {
for destNode, assignedShards := range t.shardAssignment {
// Convert shard names to shard IDs for mounting
var shardIds []uint32
var metadataFiles []string
for _, shardType := range assignedShards {
// Skip metadata files (.ecx, .vif) - only mount EC shards
if strings.HasPrefix(shardType, "ec") && len(shardType) == 4 {
@ -573,16 +644,26 @@ func (t *ErasureCodingTask) mountEcShards() error {
if _, err := fmt.Sscanf(shardType[2:], "%d", &shardId); err == nil {
shardIds = append(shardIds, shardId)
}
} else {
metadataFiles = append(metadataFiles, shardType)
}
}
t.GetLogger().WithFields(map[string]interface{}{
"destination": destNode,
"shard_ids": shardIds,
"shard_count": len(shardIds),
"metadata_files": metadataFiles,
}).Info("Starting EC shard mount operation")
if len(shardIds) == 0 {
glog.V(1).Infof("No EC shards to mount on %s (only metadata files)", destNode)
t.GetLogger().WithFields(map[string]interface{}{
"destination": destNode,
"metadata_files": metadataFiles,
}).Info("No EC shards to mount (only metadata files)")
continue
}
glog.V(1).Infof("Mounting shards %v on %s", shardIds, destNode)
err := operation.WithVolumeServerClient(false, pb.ServerAddress(destNode), grpc.WithInsecure(),
func(client volume_server_pb.VolumeServerClient) error {
_, mountErr := client.VolumeEcShardsMount(context.Background(), &volume_server_pb.VolumeEcShardsMountRequest{
@ -594,9 +675,18 @@ func (t *ErasureCodingTask) mountEcShards() error {
})
if err != nil {
glog.Warningf("Failed to mount shards %v on %s: %v", shardIds, destNode, err)
t.GetLogger().WithFields(map[string]interface{}{
"destination": destNode,
"shard_ids": shardIds,
"error": err.Error(),
}).Error("Failed to mount EC shards")
} else {
glog.V(1).Infof("Successfully mounted EC shards %v on %s", shardIds, destNode)
t.GetLogger().WithFields(map[string]interface{}{
"destination": destNode,
"shard_ids": shardIds,
"volume_id": t.volumeID,
"collection": t.collection,
}).Info("Successfully mounted EC shards")
}
}
@ -613,13 +703,24 @@ func (t *ErasureCodingTask) deleteOriginalVolume() error {
replicas = []string{t.server}
}
glog.V(1).Infof("Deleting volume %d from %d replica servers: %v", t.volumeID, len(replicas), replicas)
t.GetLogger().WithFields(map[string]interface{}{
"volume_id": t.volumeID,
"replica_count": len(replicas),
"replica_servers": replicas,
}).Info("Starting original volume deletion from replica servers")
// Delete volume from all replica locations
var deleteErrors []string
successCount := 0
for _, replicaServer := range replicas {
for i, replicaServer := range replicas {
t.GetLogger().WithFields(map[string]interface{}{
"replica_index": i + 1,
"total_replicas": len(replicas),
"server": replicaServer,
"volume_id": t.volumeID,
}).Info("Deleting volume from replica server")
err := operation.WithVolumeServerClient(false, pb.ServerAddress(replicaServer), grpc.WithInsecure(),
func(client volume_server_pb.VolumeServerClient) error {
_, err := client.VolumeDelete(context.Background(), &volume_server_pb.VolumeDeleteRequest{
@ -631,27 +732,52 @@ func (t *ErasureCodingTask) deleteOriginalVolume() error {
if err != nil {
deleteErrors = append(deleteErrors, fmt.Sprintf("failed to delete volume %d from %s: %v", t.volumeID, replicaServer, err))
glog.Warningf("Failed to delete volume %d from replica server %s: %v", t.volumeID, replicaServer, err)
t.GetLogger().WithFields(map[string]interface{}{
"server": replicaServer,
"volume_id": t.volumeID,
"error": err.Error(),
}).Error("Failed to delete volume from replica server")
} else {
successCount++
glog.V(1).Infof("Successfully deleted volume %d from replica server %s", t.volumeID, replicaServer)
t.GetLogger().WithFields(map[string]interface{}{
"server": replicaServer,
"volume_id": t.volumeID,
}).Info("Successfully deleted volume from replica server")
}
}
// Report results
if len(deleteErrors) > 0 {
glog.Warningf("Some volume deletions failed (%d/%d successful): %v", successCount, len(replicas), deleteErrors)
t.GetLogger().WithFields(map[string]interface{}{
"volume_id": t.volumeID,
"successful": successCount,
"failed": len(deleteErrors),
"total_replicas": len(replicas),
"success_rate": float64(successCount) / float64(len(replicas)) * 100,
"errors": deleteErrors,
}).Warning("Some volume deletions failed")
// Don't return error - EC task should still be considered successful if shards are mounted
} else {
glog.V(1).Infof("Successfully deleted volume %d from all %d replica servers", t.volumeID, len(replicas))
t.GetLogger().WithFields(map[string]interface{}{
"volume_id": t.volumeID,
"replica_count": len(replicas),
"replica_servers": replicas,
}).Info("Successfully deleted volume from all replica servers")
}
return nil
}
// getReplicas extracts replica servers from task parameters
// getReplicas extracts replica servers from unified sources
func (t *ErasureCodingTask) getReplicas() []string {
// Access replicas from the parameters passed during Execute
// We'll need to store these during Execute - let me add a field to the task
return t.replicas
var replicas []string
for _, source := range t.sources {
// Only include volume replica sources (not EC shard sources)
// Assumption: VolumeId == 0 is considered invalid and should be excluded.
// If volume ID 0 is valid in some contexts, update this check accordingly.
if source.VolumeId > 0 {
replicas = append(replicas, source.Node)
}
}
return replicas
}

5
weed/worker/tasks/erasure_coding/register.go

@ -42,9 +42,12 @@ func RegisterErasureCodingTask() {
if params == nil {
return nil, fmt.Errorf("task parameters are required")
}
if len(params.Sources) == 0 {
return nil, fmt.Errorf("at least one source is required for erasure coding task")
}
return NewErasureCodingTask(
fmt.Sprintf("erasure_coding-%d", params.VolumeId),
params.Server,
params.Sources[0].Node, // Use first source node
params.VolumeId,
params.Collection,
), nil

66
weed/worker/tasks/task.go

@ -7,6 +7,7 @@ import (
"time"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb/worker_pb"
"github.com/seaweedfs/seaweedfs/weed/worker/types"
)
@ -21,7 +22,8 @@ type BaseTask struct {
estimatedDuration time.Duration
logger TaskLogger
loggerConfig TaskLoggerConfig
progressCallback func(float64) // Callback function for progress updates
progressCallback func(float64, string) // Callback function for progress updates
currentStage string // Current stage description
}
// NewBaseTask creates a new base task
@ -90,20 +92,64 @@ func (t *BaseTask) SetProgress(progress float64) {
}
oldProgress := t.progress
callback := t.progressCallback
stage := t.currentStage
t.progress = progress
t.mutex.Unlock()
// Log progress change
if t.logger != nil && progress != oldProgress {
t.logger.LogProgress(progress, fmt.Sprintf("Progress updated from %.1f%% to %.1f%%", oldProgress, progress))
message := stage
if message == "" {
message = fmt.Sprintf("Progress updated from %.1f%% to %.1f%%", oldProgress, progress)
}
t.logger.LogProgress(progress, message)
}
// Call progress callback if set
if callback != nil && progress != oldProgress {
callback(progress)
callback(progress, stage)
}
}
// SetProgressWithStage sets the current progress with a stage description
func (t *BaseTask) SetProgressWithStage(progress float64, stage string) {
t.mutex.Lock()
if progress < 0 {
progress = 0
}
if progress > 100 {
progress = 100
}
callback := t.progressCallback
t.progress = progress
t.currentStage = stage
t.mutex.Unlock()
// Log progress change
if t.logger != nil {
t.logger.LogProgress(progress, stage)
}
// Call progress callback if set
if callback != nil {
callback(progress, stage)
}
}
// SetCurrentStage sets the current stage description
func (t *BaseTask) SetCurrentStage(stage string) {
t.mutex.Lock()
defer t.mutex.Unlock()
t.currentStage = stage
}
// GetCurrentStage returns the current stage description
func (t *BaseTask) GetCurrentStage() string {
t.mutex.RLock()
defer t.mutex.RUnlock()
return t.currentStage
}
// Cancel cancels the task
func (t *BaseTask) Cancel() error {
t.mutex.Lock()
@ -170,7 +216,7 @@ func (t *BaseTask) GetEstimatedDuration() time.Duration {
}
// SetProgressCallback sets the progress callback function
func (t *BaseTask) SetProgressCallback(callback func(float64)) {
func (t *BaseTask) SetProgressCallback(callback func(float64, string)) {
t.mutex.Lock()
defer t.mutex.Unlock()
t.progressCallback = callback
@ -273,7 +319,7 @@ func (t *BaseTask) ExecuteTask(ctx context.Context, params types.TaskParams, exe
if t.logger != nil {
t.logger.LogWithFields("INFO", "Task execution started", map[string]interface{}{
"volume_id": params.VolumeID,
"server": params.Server,
"server": getServerFromSources(params.TypedParams.Sources),
"collection": params.Collection,
})
}
@ -362,7 +408,7 @@ func ValidateParams(params types.TaskParams, requiredFields ...string) error {
return &ValidationError{Field: field, Message: "volume_id is required"}
}
case "server":
if params.Server == "" {
if len(params.TypedParams.Sources) == 0 {
return &ValidationError{Field: field, Message: "server is required"}
}
case "collection":
@ -383,3 +429,11 @@ type ValidationError struct {
func (e *ValidationError) Error() string {
return e.Field + ": " + e.Message
}
// getServerFromSources extracts the server address from unified sources
func getServerFromSources(sources []*worker_pb.TaskSource) string {
if len(sources) > 0 {
return sources[0].Node
}
return ""
}

36
weed/worker/tasks/task_log_handler.go

@ -5,6 +5,7 @@ import (
"os"
"path/filepath"
"strings"
"time"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb/worker_pb"
@ -20,6 +21,10 @@ func NewTaskLogHandler(baseLogDir string) *TaskLogHandler {
if baseLogDir == "" {
baseLogDir = "/tmp/seaweedfs/task_logs"
}
// Best-effort ensure the base directory exists so reads don't fail due to missing dir
if err := os.MkdirAll(baseLogDir, 0755); err != nil {
glog.Warningf("Failed to create base task log directory %s: %v", baseLogDir, err)
}
return &TaskLogHandler{
baseLogDir: baseLogDir,
}
@ -38,6 +43,23 @@ func (h *TaskLogHandler) HandleLogRequest(request *worker_pb.TaskLogRequest) *wo
if err != nil {
response.ErrorMessage = fmt.Sprintf("Task log directory not found: %v", err)
glog.Warningf("Task log request failed for %s: %v", request.TaskId, err)
// Add diagnostic information to help debug the issue
response.LogEntries = []*worker_pb.TaskLogEntry{
{
Timestamp: time.Now().Unix(),
Level: "WARNING",
Message: fmt.Sprintf("Task logs not available: %v", err),
Fields: map[string]string{"source": "task_log_handler"},
},
{
Timestamp: time.Now().Unix(),
Level: "INFO",
Message: fmt.Sprintf("This usually means the task was never executed on this worker or logs were cleaned up. Base log directory: %s", h.baseLogDir),
Fields: map[string]string{"source": "task_log_handler"},
},
}
// response.Success remains false to indicate logs were not found
return response
}
@ -71,17 +93,23 @@ func (h *TaskLogHandler) HandleLogRequest(request *worker_pb.TaskLogRequest) *wo
func (h *TaskLogHandler) findTaskLogDirectory(taskID string) (string, error) {
entries, err := os.ReadDir(h.baseLogDir)
if err != nil {
return "", fmt.Errorf("failed to read base log directory: %w", err)
return "", fmt.Errorf("failed to read base log directory %s: %w", h.baseLogDir, err)
}
// Look for directories that start with the task ID
var candidateDirs []string
for _, entry := range entries {
if entry.IsDir() && strings.HasPrefix(entry.Name(), taskID+"_") {
return filepath.Join(h.baseLogDir, entry.Name()), nil
if entry.IsDir() {
candidateDirs = append(candidateDirs, entry.Name())
if strings.HasPrefix(entry.Name(), taskID+"_") {
return filepath.Join(h.baseLogDir, entry.Name()), nil
}
}
}
return "", fmt.Errorf("task log directory not found for task ID: %s", taskID)
// Enhanced error message with diagnostic information
return "", fmt.Errorf("task log directory not found for task ID: %s (searched %d directories in %s, directories found: %v)",
taskID, len(candidateDirs), h.baseLogDir, candidateDirs)
}
// readTaskMetadata reads task metadata from the log directory

4
weed/worker/tasks/task_logger.go

@ -127,7 +127,7 @@ func NewTaskLogger(taskID string, taskType types.TaskType, workerID string, para
Status: "started",
Progress: 0.0,
VolumeID: params.VolumeID,
Server: params.Server,
Server: getServerFromSources(params.TypedParams.Sources),
Collection: params.Collection,
CustomData: make(map[string]interface{}),
LogFilePath: logFilePath,
@ -149,7 +149,7 @@ func NewTaskLogger(taskID string, taskType types.TaskType, workerID string, para
logger.Info("Task logger initialized for %s (type: %s, worker: %s)", taskID, taskType, workerID)
logger.LogWithFields("INFO", "Task parameters", map[string]interface{}{
"volume_id": params.VolumeID,
"server": params.Server,
"server": getServerFromSources(params.TypedParams.Sources),
"collection": params.Collection,
})

22
weed/worker/tasks/vacuum/detection.go

@ -47,7 +47,7 @@ func Detection(metrics []*types.VolumeHealthMetrics, clusterInfo *types.ClusterI
}
// Create typed parameters for vacuum task
result.TypedParams = createVacuumTaskParams(result, metric, vacuumConfig)
result.TypedParams = createVacuumTaskParams(result, metric, vacuumConfig, clusterInfo)
results = append(results, result)
} else {
// Debug why volume was not selected
@ -85,7 +85,7 @@ func Detection(metrics []*types.VolumeHealthMetrics, clusterInfo *types.ClusterI
// createVacuumTaskParams creates typed parameters for vacuum tasks
// This function is moved from MaintenanceIntegration.createVacuumTaskParams to the detection logic
func createVacuumTaskParams(task *types.TaskDetectionResult, metric *types.VolumeHealthMetrics, vacuumConfig *Config) *worker_pb.TaskParams {
func createVacuumTaskParams(task *types.TaskDetectionResult, metric *types.VolumeHealthMetrics, vacuumConfig *Config, clusterInfo *types.ClusterInfo) *worker_pb.TaskParams {
// Use configured values or defaults
garbageThreshold := 0.3 // Default 30%
verifyChecksum := true // Default to verify
@ -99,13 +99,27 @@ func createVacuumTaskParams(task *types.TaskDetectionResult, metric *types.Volum
// to the protobuf definition if they should be configurable
}
// Create typed protobuf parameters
// Use DC and rack information directly from VolumeHealthMetrics
sourceDC, sourceRack := metric.DataCenter, metric.Rack
// Create typed protobuf parameters with unified sources
return &worker_pb.TaskParams{
TaskId: task.TaskID, // Link to ActiveTopology pending task (if integrated)
VolumeId: task.VolumeID,
Server: task.Server,
Collection: task.Collection,
VolumeSize: metric.Size, // Store original volume size for tracking changes
// Unified sources array
Sources: []*worker_pb.TaskSource{
{
Node: task.Server,
VolumeId: task.VolumeID,
EstimatedSize: metric.Size,
DataCenter: sourceDC,
Rack: sourceRack,
},
},
TaskParams: &worker_pb.TaskParams_VacuumParams{
VacuumParams: &worker_pb.VacuumTaskParams{
GarbageThreshold: garbageThreshold,

5
weed/worker/tasks/vacuum/register.go

@ -42,9 +42,12 @@ func RegisterVacuumTask() {
if params == nil {
return nil, fmt.Errorf("task parameters are required")
}
if len(params.Sources) == 0 {
return nil, fmt.Errorf("at least one source is required for vacuum task")
}
return NewVacuumTask(
fmt.Sprintf("vacuum-%d", params.VolumeId),
params.Server,
params.Sources[0].Node, // Use first source node
params.VolumeId,
params.Collection,
), nil

12
weed/worker/tasks/vacuum/vacuum_task.go

@ -114,8 +114,16 @@ func (t *VacuumTask) Validate(params *worker_pb.TaskParams) error {
return fmt.Errorf("volume ID mismatch: expected %d, got %d", t.volumeID, params.VolumeId)
}
if params.Server != t.server {
return fmt.Errorf("source server mismatch: expected %s, got %s", t.server, params.Server)
// Validate that at least one source matches our server
found := false
for _, source := range params.Sources {
if source.Node == t.server {
found = true
break
}
}
if !found {
return fmt.Errorf("no source matches expected server %s", t.server)
}
if vacuumParams.GarbageThreshold < 0 || vacuumParams.GarbageThreshold > 1.0 {

25
weed/worker/types/base/task.go

@ -12,9 +12,10 @@ import (
type BaseTask struct {
id string
taskType types.TaskType
progressCallback func(float64)
progressCallback func(float64, string) // Modified to include stage description
logger types.Logger
cancelled bool
currentStage string
}
// NewBaseTask creates a new base task
@ -37,17 +38,35 @@ func (t *BaseTask) Type() types.TaskType {
}
// SetProgressCallback sets the progress callback
func (t *BaseTask) SetProgressCallback(callback func(float64)) {
func (t *BaseTask) SetProgressCallback(callback func(float64, string)) {
t.progressCallback = callback
}
// ReportProgress reports current progress through the callback
func (t *BaseTask) ReportProgress(progress float64) {
if t.progressCallback != nil {
t.progressCallback(progress)
t.progressCallback(progress, t.currentStage)
}
}
// ReportProgressWithStage reports current progress with a specific stage description
func (t *BaseTask) ReportProgressWithStage(progress float64, stage string) {
t.currentStage = stage
if t.progressCallback != nil {
t.progressCallback(progress, stage)
}
}
// SetCurrentStage sets the current stage description
func (t *BaseTask) SetCurrentStage(stage string) {
t.currentStage = stage
}
// GetCurrentStage returns the current stage description
func (t *BaseTask) GetCurrentStage() string {
return t.currentStage
}
// GetProgress returns current progress
func (t *BaseTask) GetProgress() float64 {
// Subclasses should override this

2
weed/worker/types/data_types.go

@ -21,6 +21,8 @@ type VolumeHealthMetrics struct {
Server string
DiskType string // Disk type (e.g., "hdd", "ssd") or disk path (e.g., "/data1")
DiskId uint32 // ID of the disk in Store.Locations array
DataCenter string // Data center of the server
Rack string // Rack of the server
Collection string
Size uint64
DeletedBytes uint64

27
weed/worker/types/task.go

@ -28,7 +28,7 @@ type Task interface {
// Progress
GetProgress() float64
SetProgressCallback(func(float64))
SetProgressCallback(func(float64, string))
}
// TaskWithLogging extends Task with logging capabilities
@ -127,9 +127,10 @@ type LoggerFactory interface {
type UnifiedBaseTask struct {
id string
taskType TaskType
progressCallback func(float64)
progressCallback func(float64, string)
logger Logger
cancelled bool
currentStage string
}
// NewBaseTask creates a new base task
@ -151,17 +152,35 @@ func (t *UnifiedBaseTask) Type() TaskType {
}
// SetProgressCallback sets the progress callback
func (t *UnifiedBaseTask) SetProgressCallback(callback func(float64)) {
func (t *UnifiedBaseTask) SetProgressCallback(callback func(float64, string)) {
t.progressCallback = callback
}
// ReportProgress reports current progress through the callback
func (t *UnifiedBaseTask) ReportProgress(progress float64) {
if t.progressCallback != nil {
t.progressCallback(progress)
t.progressCallback(progress, t.currentStage)
}
}
// ReportProgressWithStage reports current progress with a specific stage description
func (t *UnifiedBaseTask) ReportProgressWithStage(progress float64, stage string) {
t.currentStage = stage
if t.progressCallback != nil {
t.progressCallback(progress, stage)
}
}
// SetCurrentStage sets the current stage description
func (t *UnifiedBaseTask) SetCurrentStage(stage string) {
t.currentStage = stage
}
// GetCurrentStage returns the current stage description
func (t *UnifiedBaseTask) GetCurrentStage() string {
return t.currentStage
}
// Cancel marks the task as cancelled
func (t *UnifiedBaseTask) Cancel() error {
t.cancelled = true

1
weed/worker/types/task_types.go

@ -64,7 +64,6 @@ type TaskInput struct {
// 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"`
WorkingDir string `json:"working_dir,omitempty"`
TypedParams *worker_pb.TaskParams `json:"typed_params,omitempty"`

2
weed/worker/types/typed_task_interface.go

@ -54,7 +54,7 @@ type TypedTaskInterface interface {
GetProgress() float64
// Set progress callback for progress updates
SetProgressCallback(callback func(float64))
SetProgressCallback(callback func(float64, string))
// Logger configuration and initialization (all typed tasks support this)
SetLoggerConfig(config TaskLoggerConfig)

95
weed/worker/worker.go

@ -4,7 +4,6 @@ import (
"context"
"crypto/rand"
"fmt"
"net"
"os"
"path/filepath"
"strings"
@ -78,43 +77,39 @@ func GenerateOrLoadWorkerID(workingDir string) (string, error) {
}
}
// Generate new unique worker ID with host information
// Generate simplified worker ID
hostname, _ := os.Hostname()
if hostname == "" {
hostname = "unknown"
}
// Get local IP address for better host identification
var hostIP string
if addrs, err := net.InterfaceAddrs(); err == nil {
for _, addr := range addrs {
if ipnet, ok := addr.(*net.IPNet); ok && !ipnet.IP.IsLoopback() {
if ipnet.IP.To4() != nil {
hostIP = ipnet.IP.String()
break
}
// Use short hostname - take first 6 chars or last part after dots
shortHostname := hostname
if len(hostname) > 6 {
if parts := strings.Split(hostname, "."); len(parts) > 1 {
// Use last part before domain (e.g., "worker1" from "worker1.example.com")
shortHostname = parts[0]
if len(shortHostname) > 6 {
shortHostname = shortHostname[:6]
}
} else {
// Use first 6 characters
shortHostname = hostname[:6]
}
}
if hostIP == "" {
hostIP = "noip"
}
// Create host identifier combining hostname and IP
hostID := fmt.Sprintf("%s@%s", hostname, hostIP)
// Generate random component for uniqueness
randomBytes := make([]byte, 4)
// Generate random component for uniqueness (2 bytes = 4 hex chars)
randomBytes := make([]byte, 2)
var workerID string
if _, err := rand.Read(randomBytes); err != nil {
// Fallback to timestamp if crypto/rand fails
workerID = fmt.Sprintf("worker-%s-%d", hostID, time.Now().Unix())
// Fallback to short timestamp if crypto/rand fails
timestamp := time.Now().Unix() % 10000 // last 4 digits
workerID = fmt.Sprintf("w-%s-%04d", shortHostname, timestamp)
glog.Infof("Generated fallback worker ID: %s", workerID)
} else {
// Use random bytes + timestamp for uniqueness
// Use random hex for uniqueness
randomHex := fmt.Sprintf("%x", randomBytes)
timestamp := time.Now().Unix()
workerID = fmt.Sprintf("worker-%s-%s-%d", hostID, randomHex, timestamp)
workerID = fmt.Sprintf("w-%s-%s", shortHostname, randomHex)
glog.Infof("Generated new worker ID: %s", workerID)
}
@ -145,6 +140,10 @@ func NewWorker(config *types.WorkerConfig) (*Worker, error) {
// Initialize task log handler
logDir := filepath.Join(config.BaseWorkingDir, "task_logs")
// Ensure the base task log directory exists to avoid errors when admin requests logs
if err := os.MkdirAll(logDir, 0755); err != nil {
glog.Warningf("Failed to create task log base directory %s: %v", logDir, err)
}
taskLogHandler := tasks.NewTaskLogHandler(logDir)
worker := &Worker{
@ -407,6 +406,26 @@ func (w *Worker) executeTask(task *types.TaskInput) {
// Use new task execution system with unified Task interface
glog.V(1).Infof("Executing task %s with typed protobuf parameters", task.ID)
// Initialize a file-based task logger so admin can retrieve logs
// Build minimal params for logger metadata
loggerParams := types.TaskParams{
VolumeID: task.VolumeID,
Collection: task.Collection,
TypedParams: task.TypedParams,
}
loggerConfig := w.getTaskLoggerConfig()
fileLogger, logErr := tasks.NewTaskLogger(task.ID, task.Type, w.id, loggerParams, loggerConfig)
if logErr != nil {
glog.Warningf("Failed to initialize file logger for task %s: %v", task.ID, logErr)
} else {
defer func() {
if err := fileLogger.Close(); err != nil {
glog.V(1).Infof("Failed to close task logger for %s: %v", task.ID, err)
}
}()
fileLogger.Info("Task %s started (type=%s, server=%s, collection=%s)", task.ID, task.Type, task.Server, task.Collection)
}
taskFactory := w.registry.Get(task.Type)
if taskFactory == nil {
w.completeTask(task.ID, false, fmt.Sprintf("task factory not available for %s: task type not found", task.Type))
@ -431,13 +450,28 @@ func (w *Worker) executeTask(task *types.TaskInput) {
// Task execution uses the new unified Task interface
glog.V(2).Infof("Executing task %s in working directory: %s", task.ID, taskWorkingDir)
// If we have a file logger, adapt it so task WithFields logs are captured into file
if fileLogger != nil {
if withLogger, ok := taskInstance.(interface{ SetLogger(types.Logger) }); ok {
withLogger.SetLogger(newTaskLoggerAdapter(fileLogger))
}
}
// Set progress callback that reports to admin server
taskInstance.SetProgressCallback(func(progress float64) {
taskInstance.SetProgressCallback(func(progress float64, stage string) {
// Report progress updates to admin server
glog.V(2).Infof("Task %s progress: %.1f%%", task.ID, progress)
glog.V(2).Infof("Task %s progress: %.1f%% - %s", task.ID, progress, stage)
if err := w.adminClient.UpdateTaskProgress(task.ID, progress); err != nil {
glog.V(1).Infof("Failed to report task progress to admin: %v", err)
}
if fileLogger != nil {
// Use meaningful stage description or fallback to generic message
message := stage
if message == "" {
message = fmt.Sprintf("Progress: %.1f%%", progress)
}
fileLogger.LogProgress(progress, message)
}
})
// Execute task with context
@ -449,10 +483,17 @@ func (w *Worker) executeTask(task *types.TaskInput) {
w.completeTask(task.ID, false, err.Error())
w.tasksFailed++
glog.Errorf("Worker %s failed to execute task %s: %v", w.id, task.ID, err)
if fileLogger != nil {
fileLogger.LogStatus("failed", err.Error())
fileLogger.Error("Task %s failed: %v", task.ID, err)
}
} else {
w.completeTask(task.ID, true, "")
w.tasksCompleted++
glog.Infof("Worker %s completed task %s successfully", w.id, task.ID)
if fileLogger != nil {
fileLogger.Info("Task %s completed successfully", task.ID)
}
}
}
@ -696,7 +737,7 @@ func (w *Worker) processAdminMessage(message *worker_pb.AdminMessage) {
Type: types.TaskType(taskAssign.TaskType),
Status: types.TaskStatusAssigned,
VolumeID: taskAssign.Params.VolumeId,
Server: taskAssign.Params.Server,
Server: getServerFromParams(taskAssign.Params),
Collection: taskAssign.Params.Collection,
Priority: types.TaskPriority(taskAssign.Priority),
CreatedAt: time.Unix(taskAssign.CreatedTime, 0),

Loading…
Cancel
Save