From 25bbf4c3d44b1c8a9aa4980e37ed399ec249f771 Mon Sep 17 00:00:00 2001 From: Chris Lu Date: Sat, 9 Aug 2025 21:47:29 -0700 Subject: [PATCH] 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> --- weed/admin/dash/admin_server.go | 265 ++- weed/admin/dash/config_persistence.go | 545 ++++++ weed/admin/dash/ec_shard_management.go | 13 +- weed/admin/dash/volume_management.go | 18 +- weed/admin/dash/worker_grpc_server.go | 164 +- weed/admin/handlers/admin_handlers.go | 6 + weed/admin/handlers/cluster_handlers.go | 8 +- weed/admin/handlers/maintenance_handlers.go | 91 +- weed/admin/maintenance/maintenance_queue.go | 236 ++- weed/admin/maintenance/maintenance_scanner.go | 4 + weed/admin/maintenance/maintenance_types.go | 67 +- weed/admin/topology/structs.go | 19 +- weed/admin/topology/task_management.go | 9 +- weed/admin/topology/topology_management.go | 12 +- weed/admin/view/app/admin.templ | 2 +- weed/admin/view/app/admin_templ.go | 2 +- weed/admin/view/app/cluster_ec_volumes.templ | 13 +- .../view/app/cluster_ec_volumes_templ.go | 501 ++--- .../view/app/cluster_volume_servers.templ | 4 - .../view/app/cluster_volume_servers_templ.go | 217 ++- weed/admin/view/app/collection_details.templ | 10 + .../view/app/collection_details_templ.go | 99 +- weed/admin/view/app/maintenance_queue.templ | 221 +-- .../admin/view/app/maintenance_queue_templ.go | 718 ++++---- weed/admin/view/app/task_config_schema.templ | 3 +- .../view/app/task_config_schema_templ.go | 425 +++-- weed/admin/view/app/task_detail.templ | 1118 +++++++++++ weed/admin/view/app/task_detail_templ.go | 1628 +++++++++++++++++ weed/pb/worker.proto | 167 +- weed/pb/worker_pb/worker.pb.go | 1332 ++++++++++---- weed/worker/client.go | 10 +- weed/worker/log_adapter.go | 85 + weed/worker/tasks/balance/balance_task.go | 31 +- weed/worker/tasks/balance/detection.go | 66 +- weed/worker/tasks/balance/execution.go | 72 +- weed/worker/tasks/balance/register.go | 5 +- weed/worker/tasks/base/typed_task.go | 39 +- weed/worker/tasks/erasure_coding/detection.go | 194 +- weed/worker/tasks/erasure_coding/ec_task.go | 368 ++-- weed/worker/tasks/erasure_coding/register.go | 5 +- weed/worker/tasks/task.go | 66 +- weed/worker/tasks/task_log_handler.go | 36 +- weed/worker/tasks/task_logger.go | 4 +- weed/worker/tasks/vacuum/detection.go | 22 +- weed/worker/tasks/vacuum/register.go | 5 +- weed/worker/tasks/vacuum/vacuum_task.go | 12 +- weed/worker/types/base/task.go | 25 +- weed/worker/types/data_types.go | 2 + weed/worker/types/task.go | 27 +- weed/worker/types/task_types.go | 1 - weed/worker/types/typed_task_interface.go | 2 +- weed/worker/worker.go | 95 +- 52 files changed, 7196 insertions(+), 1893 deletions(-) create mode 100644 weed/admin/view/app/task_detail.templ create mode 100644 weed/admin/view/app/task_detail_templ.go create mode 100644 weed/worker/log_adapter.go diff --git a/weed/admin/dash/admin_server.go b/weed/admin/dash/admin_server.go index 376f3edc7..3f135ee1b 100644 --- a/weed/admin/dash/admin_server.go +++ b/weed/admin/dash/admin_server.go @@ -5,6 +5,7 @@ import ( "context" "fmt" "net/http" + "strconv" "time" "github.com/gin-gonic/gin" @@ -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) } diff --git a/weed/admin/dash/config_persistence.go b/weed/admin/dash/config_persistence.go index b6b3074ab..1fe1a9b42 100644 --- a/weed/admin/dash/config_persistence.go +++ b/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 + } +} diff --git a/weed/admin/dash/ec_shard_management.go b/weed/admin/dash/ec_shard_management.go index 272890cf0..34574ecdb 100644 --- a/weed/admin/dash/ec_shard_management.go +++ b/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) } } diff --git a/weed/admin/dash/volume_management.go b/weed/admin/dash/volume_management.go index 5dabe2674..38b1257a4 100644 --- a/weed/admin/dash/volume_management.go +++ b/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 diff --git a/weed/admin/dash/worker_grpc_server.go b/weed/admin/dash/worker_grpc_server.go index 3b4312235..78ba6d7de 100644 --- a/weed/admin/dash/worker_grpc_server.go +++ b/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) diff --git a/weed/admin/handlers/admin_handlers.go b/weed/admin/handlers/admin_handlers.go index d28dc9e53..215e2a4e5 100644 --- a/weed/admin/handlers/admin_handlers.go +++ b/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) diff --git a/weed/admin/handlers/cluster_handlers.go b/weed/admin/handlers/cluster_handlers.go index 38eebee8b..ee6417954 100644 --- a/weed/admin/handlers/cluster_handlers.go +++ b/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 diff --git a/weed/admin/handlers/maintenance_handlers.go b/weed/admin/handlers/maintenance_handlers.go index 1e2337272..e92a50c9d 100644 --- a/weed/admin/handlers/maintenance_handlers.go +++ b/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 } diff --git a/weed/admin/maintenance/maintenance_queue.go b/weed/admin/maintenance/maintenance_queue.go index ca402bd4d..d39c96a30 100644 --- a/weed/admin/maintenance/maintenance_queue.go +++ b/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(), diff --git a/weed/admin/maintenance/maintenance_scanner.go b/weed/admin/maintenance/maintenance_scanner.go index 3f8a528eb..6f3b46be2 100644 --- a/weed/admin/maintenance/maintenance_scanner.go +++ b/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, diff --git a/weed/admin/maintenance/maintenance_types.go b/weed/admin/maintenance/maintenance_types.go index e863b26e6..fe5d5fa55 100644 --- a/weed/admin/maintenance/maintenance_types.go +++ b/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"` diff --git a/weed/admin/topology/structs.go b/weed/admin/topology/structs.go index f2d29eb5f..103ee5abe 100644 --- a/weed/admin/topology/structs.go +++ b/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"` } diff --git a/weed/admin/topology/task_management.go b/weed/admin/topology/task_management.go index b240adcd8..ada60248b 100644 --- a/weed/admin/topology/task_management.go +++ b/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 -} diff --git a/weed/admin/topology/topology_management.go b/weed/admin/topology/topology_management.go index e12839484..65b7dfe7e 100644 --- a/weed/admin/topology/topology_management.go +++ b/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, }) } } diff --git a/weed/admin/view/app/admin.templ b/weed/admin/view/app/admin.templ index 534c798bd..568db59d7 100644 --- a/weed/admin/view/app/admin.templ +++ b/weed/admin/view/app/admin.templ @@ -12,7 +12,7 @@ templ Admin(data dash.AdminData) {
- + Object Store Buckets diff --git a/weed/admin/view/app/admin_templ.go b/weed/admin/view/app/admin_templ.go index 906c0fd1c..f0257e1d7 100644 --- a/weed/admin/view/app/admin_templ.go +++ b/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, "
Total Volumes
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 1, "
Total Volumes
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } diff --git a/weed/admin/view/app/cluster_ec_volumes.templ b/weed/admin/view/app/cluster_ec_volumes.templ index aafa621aa..c84da45ca 100644 --- a/weed/admin/view/app/cluster_ec_volumes.templ +++ b/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) { @@ -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 { Complete } else { - if len(volume.MissingShards) > 10 { + if len(volume.MissingShards) > erasure_coding.DataShardsCount { Critical ({fmt.Sprintf("%d", len(volume.MissingShards))} missing) - } else if len(volume.MissingShards) > 6 { + } else if len(volume.MissingShards) > (erasure_coding.DataShardsCount/2) { Degraded ({fmt.Sprintf("%d", len(volume.MissingShards))} missing) - } else if len(volume.MissingShards) > 2 { + } else if len(volume.MissingShards) > (erasure_coding.ParityShardsCount/2) { Incomplete ({fmt.Sprintf("%d", len(volume.MissingShards))} missing) } else { Minor Issues ({fmt.Sprintf("%d", len(volume.MissingShards))} missing) diff --git a/weed/admin/view/app/cluster_ec_volumes_templ.go b/weed/admin/view/app/cluster_ec_volumes_templ.go index 419739e7c..932075106 100644 --- a/weed/admin/view/app/cluster_ec_volumes_templ.go +++ b/weed/admin/view/app/cluster_ec_volumes_templ.go @@ -11,6 +11,7 @@ import templruntime "github.com/a-h/templ/runtime" import ( "fmt" "github.com/seaweedfs/seaweedfs/weed/admin/dash" + "github.com/seaweedfs/seaweedfs/weed/storage/erasure_coding" "strings" ) @@ -42,7 +43,7 @@ func ClusterEcVolumes(data dash.ClusterEcVolumesData) templ.Component { var templ_7745c5c3_Var2 string templ_7745c5c3_Var2, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.TotalVolumes)) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 25, Col: 84} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 26, Col: 84} } _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var2)) if templ_7745c5c3_Err != nil { @@ -55,7 +56,7 @@ func ClusterEcVolumes(data dash.ClusterEcVolumesData) templ.Component { var templ_7745c5c3_Var3 string templ_7745c5c3_Var3, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.TotalVolumes)) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 38, Col: 86} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 39, Col: 86} } _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var3)) if templ_7745c5c3_Err != nil { @@ -68,7 +69,7 @@ func ClusterEcVolumes(data dash.ClusterEcVolumesData) templ.Component { var templ_7745c5c3_Var4 string templ_7745c5c3_Var4, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.TotalShards)) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 54, Col: 85} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 55, Col: 85} } _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var4)) if templ_7745c5c3_Err != nil { @@ -81,7 +82,7 @@ func ClusterEcVolumes(data dash.ClusterEcVolumesData) templ.Component { var templ_7745c5c3_Var5 string templ_7745c5c3_Var5, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.CompleteVolumes)) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 70, Col: 89} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 71, Col: 89} } _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var5)) if templ_7745c5c3_Err != nil { @@ -94,31 +95,83 @@ func ClusterEcVolumes(data dash.ClusterEcVolumesData) templ.Component { var templ_7745c5c3_Var6 string templ_7745c5c3_Var6, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.IncompleteVolumes)) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 86, Col: 91} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 87, Col: 91} } _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var6)) if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 6, "Missing shards
EC Storage Note: 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.
Showing ") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 6, "Missing shards
EC Storage Note: EC volumes use erasure coding (") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } var templ_7745c5c3_Var7 string - templ_7745c5c3_Var7, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", (data.Page-1)*data.PageSize+1)) + templ_7745c5c3_Var7, 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/cluster_ec_volumes.templ`, Line: 110, Col: 79} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 103, Col: 131} } _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var7)) if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 7, " to ") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 7, ") which stores data across ") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } var templ_7745c5c3_Var8 string - templ_7745c5c3_Var8, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", func() int { + templ_7745c5c3_Var8, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", erasure_coding.TotalShardsCount)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 103, Col: 212} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var8)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 8, " shards with redundancy. Physical storage is approximately ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var9 string + templ_7745c5c3_Var9, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%.1fx", float64(erasure_coding.TotalShardsCount)/float64(erasure_coding.DataShardsCount))) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 104, Col: 150} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var9)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 9, " the original logical data size due to ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var10 string + templ_7745c5c3_Var10, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", erasure_coding.ParityShardsCount)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 104, Col: 244} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var10)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 10, " parity shards.
Showing ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var11 string + templ_7745c5c3_Var11, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", (data.Page-1)*data.PageSize+1)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 111, Col: 79} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var11)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 11, " to ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var12 string + templ_7745c5c3_Var12, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", func() int { end := data.Page * data.PageSize if end > data.TotalVolumes { return data.TotalVolumes @@ -126,291 +179,291 @@ func ClusterEcVolumes(data dash.ClusterEcVolumesData) templ.Component { return end }())) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 116, Col: 24} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 117, Col: 24} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var8)) + _, 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, 8, " of ") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 12, " of ") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var9 string - templ_7745c5c3_Var9, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.TotalVolumes)) + var templ_7745c5c3_Var13 string + templ_7745c5c3_Var13, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.TotalVolumes)) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 116, Col: 66} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 117, Col: 66} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var9)) + _, 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, 9, " volumes
per page
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 23, ">100 per page
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if data.Collection != "" { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if data.Collection == "default" { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, "Collection: default ") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 25, "Collection: default ") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 22, "Collection: ") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, "Collection: ") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var10 string - templ_7745c5c3_Var10, templ_7745c5c3_Err = templ.JoinStringErrs(data.Collection) + var templ_7745c5c3_Var14 string + templ_7745c5c3_Var14, templ_7745c5c3_Err = templ.JoinStringErrs(data.Collection) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 137, Col: 91} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 138, Col: 91} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var10)) + _, 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, 23, " ") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, " ") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, "Clear Filter
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, "Clear Filter
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 25, "
Volume ID ") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 33, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if data.ShowCollectionColumn { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 30, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 38, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 35, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 47, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if data.ShowDataCenterColumn { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 44, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 48, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 45, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 49, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } for _, volume := range data.EcVolumes { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 46, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 51, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if data.ShowCollectionColumn { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 48, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 56, "") 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, 61, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if data.ShowDataCenterColumn { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 58, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 66, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 63, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 71, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 68, "
Volume ID ") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if data.SortBy == "volume_id" { if data.SortOrder == "asc" { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 30, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 31, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 32, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, "Collection ") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 34, "Collection ") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if data.SortBy == "collection" { if data.SortOrder == "asc" { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 31, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 35, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 32, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 36, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 33, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 37, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 34, "Shard Count ") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 39, "Shard Count ") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if data.SortBy == "total_shards" { if data.SortOrder == "asc" { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 36, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 40, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 37, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 41, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 38, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 42, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 39, "Shard SizeShard LocationsStatus ") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 43, "Shard SizeShard LocationsStatus ") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if data.SortBy == "completeness" { if data.SortOrder == "asc" { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 40, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 44, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 41, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 45, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 42, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 46, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 43, "Data CentersData CentersActions
Actions
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 50, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var11 string - templ_7745c5c3_Var11, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", volume.VolumeID)) + var templ_7745c5c3_Var15 string + templ_7745c5c3_Var15, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", volume.VolumeID)) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 218, Col: 75} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 219, Col: 75} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var11)) + _, 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, 47, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 52, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if volume.Collection != "" { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 49, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 53, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var12 string - templ_7745c5c3_Var12, templ_7745c5c3_Err = templ.JoinStringErrs(volume.Collection) + var templ_7745c5c3_Var16 string + templ_7745c5c3_Var16, templ_7745c5c3_Err = templ.JoinStringErrs(volume.Collection) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 224, Col: 101} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 225, Col: 101} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var12)) + _, 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, 50, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 54, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 51, "default") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 55, "default") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 52, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 57, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var13 string - templ_7745c5c3_Var13, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d/14", volume.TotalShards)) + var templ_7745c5c3_Var17 string + templ_7745c5c3_Var17, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d/14", volume.TotalShards)) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 234, Col: 104} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 235, Col: 104} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var13)) + _, 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, 54, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 58, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } @@ -418,7 +471,7 @@ func ClusterEcVolumes(data dash.ClusterEcVolumesData) templ.Component { 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, 59, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } @@ -426,7 +479,7 @@ func ClusterEcVolumes(data dash.ClusterEcVolumesData) templ.Component { if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 56, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 60, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } @@ -434,214 +487,214 @@ func ClusterEcVolumes(data dash.ClusterEcVolumesData) templ.Component { if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 57, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 62, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } for i, dc := range volume.DataCenters { if i > 0 { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 59, ", ") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 63, ", ") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 60, " ") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 64, " ") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var14 string - templ_7745c5c3_Var14, templ_7745c5c3_Err = templ.JoinStringErrs(dc) + var templ_7745c5c3_Var18 string + templ_7745c5c3_Var18, templ_7745c5c3_Err = templ.JoinStringErrs(dc) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 251, Col: 85} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 252, Col: 85} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var14)) + _, 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, 61, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 65, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 62, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 68, "\" title=\"View EC volume details\"> ") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if !volume.IsComplete { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 65, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 70, "\" title=\"Repair missing shards\">") 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, 72, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if data.TotalPages > 1 { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 69, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 83, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 87, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } @@ -666,60 +719,60 @@ func displayShardLocationsHTML(shardLocations map[int]string) templ.Component { }() } ctx = templ.InitializeContext(ctx) - templ_7745c5c3_Var23 := templ.GetChildren(ctx) - if templ_7745c5c3_Var23 == nil { - templ_7745c5c3_Var23 = templ.NopComponent + templ_7745c5c3_Var27 := templ.GetChildren(ctx) + if templ_7745c5c3_Var27 == nil { + templ_7745c5c3_Var27 = templ.NopComponent } ctx = templ.ClearChildren(ctx) if len(shardLocations) == 0 { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 84, "No shards") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 88, "No shards") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else { for i, serverInfo := range groupShardsByServer(shardLocations) { if i > 0 { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 85, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 89, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 86, " ") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 91, "\" class=\"text-primary text-decoration-none\">") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var25 string - templ_7745c5c3_Var25, templ_7745c5c3_Err = templ.JoinStringErrs(serverInfo.Server) + var templ_7745c5c3_Var29 string + templ_7745c5c3_Var29, templ_7745c5c3_Err = templ.JoinStringErrs(serverInfo.Server) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 391, Col: 24} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 392, Col: 24} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var25)) + _, 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, 88, ": ") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 92, ": ") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var26 string - templ_7745c5c3_Var26, templ_7745c5c3_Err = templ.JoinStringErrs(serverInfo.ShardRanges) + var templ_7745c5c3_Var30 string + templ_7745c5c3_Var30, templ_7745c5c3_Err = templ.JoinStringErrs(serverInfo.ShardRanges) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 393, Col: 37} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 394, Col: 37} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var26)) + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var30)) if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } @@ -746,13 +799,13 @@ func displayShardSizes(shardSizes map[int]int64) templ.Component { }() } ctx = templ.InitializeContext(ctx) - templ_7745c5c3_Var27 := templ.GetChildren(ctx) - if templ_7745c5c3_Var27 == nil { - templ_7745c5c3_Var27 = templ.NopComponent + templ_7745c5c3_Var31 := templ.GetChildren(ctx) + if templ_7745c5c3_Var31 == nil { + templ_7745c5c3_Var31 = templ.NopComponent } ctx = templ.ClearChildren(ctx) if len(shardSizes) == 0 { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 89, "-") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 93, "-") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } @@ -783,44 +836,44 @@ func renderShardSizesContent(shardSizes map[int]int64) templ.Component { }() } ctx = templ.InitializeContext(ctx) - templ_7745c5c3_Var28 := templ.GetChildren(ctx) - if templ_7745c5c3_Var28 == nil { - templ_7745c5c3_Var28 = templ.NopComponent + templ_7745c5c3_Var32 := templ.GetChildren(ctx) + if templ_7745c5c3_Var32 == nil { + templ_7745c5c3_Var32 = templ.NopComponent } ctx = templ.ClearChildren(ctx) if areAllShardSizesSame(shardSizes) { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 90, " ") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 94, " ") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var29 string - templ_7745c5c3_Var29, templ_7745c5c3_Err = templ.JoinStringErrs(getCommonShardSize(shardSizes)) + var templ_7745c5c3_Var33 string + templ_7745c5c3_Var33, templ_7745c5c3_Err = templ.JoinStringErrs(getCommonShardSize(shardSizes)) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 411, Col: 60} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 412, Col: 60} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var29)) + _, 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, 91, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 95, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 92, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 96, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var30 string - templ_7745c5c3_Var30, templ_7745c5c3_Err = templ.JoinStringErrs(formatIndividualShardSizes(shardSizes)) + var templ_7745c5c3_Var34 string + templ_7745c5c3_Var34, templ_7745c5c3_Err = templ.JoinStringErrs(formatIndividualShardSizes(shardSizes)) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 415, Col: 43} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 416, Col: 43} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var30)) + _, 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, 93, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 97, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } @@ -1100,7 +1153,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)) @@ -1135,25 +1188,25 @@ func displayVolumeDistribution(volume dash.EcVolumeWithShards) templ.Component { }() } ctx = templ.InitializeContext(ctx) - templ_7745c5c3_Var31 := templ.GetChildren(ctx) - if templ_7745c5c3_Var31 == nil { - templ_7745c5c3_Var31 = templ.NopComponent + templ_7745c5c3_Var35 := templ.GetChildren(ctx) + if templ_7745c5c3_Var35 == nil { + templ_7745c5c3_Var35 = templ.NopComponent } ctx = templ.ClearChildren(ctx) - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 94, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 98, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var32 string - templ_7745c5c3_Var32, templ_7745c5c3_Err = templ.JoinStringErrs(calculateVolumeDistributionSummary(volume)) + var templ_7745c5c3_Var36 string + templ_7745c5c3_Var36, templ_7745c5c3_Err = templ.JoinStringErrs(calculateVolumeDistributionSummary(volume)) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 713, Col: 52} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 714, Col: 52} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var32)) + _, 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, 95, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 99, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } @@ -1178,86 +1231,86 @@ func displayEcVolumeStatus(volume dash.EcVolumeWithShards) templ.Component { }() } ctx = templ.InitializeContext(ctx) - templ_7745c5c3_Var33 := templ.GetChildren(ctx) - if templ_7745c5c3_Var33 == nil { - templ_7745c5c3_Var33 = templ.NopComponent + templ_7745c5c3_Var37 := templ.GetChildren(ctx) + if templ_7745c5c3_Var37 == nil { + templ_7745c5c3_Var37 = templ.NopComponent } ctx = templ.ClearChildren(ctx) if volume.IsComplete { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 96, "Complete") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 100, "Complete") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else { - if len(volume.MissingShards) > 10 { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 97, "Critical (") + if len(volume.MissingShards) > erasure_coding.DataShardsCount { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 101, "Critical (") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var34 string - templ_7745c5c3_Var34, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", len(volume.MissingShards))) + var templ_7745c5c3_Var38 string + templ_7745c5c3_Var38, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", len(volume.MissingShards))) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 723, Col: 130} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 724, Col: 130} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var34)) + _, 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, 98, " missing)") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 102, " missing)") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - } else if len(volume.MissingShards) > 6 { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 99, "Degraded (") + } else if len(volume.MissingShards) > (erasure_coding.DataShardsCount / 2) { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 103, "Degraded (") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var35 string - templ_7745c5c3_Var35, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", len(volume.MissingShards))) + var templ_7745c5c3_Var39 string + templ_7745c5c3_Var39, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", len(volume.MissingShards))) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 725, Col: 146} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 726, Col: 146} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var35)) + _, 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, 100, " missing)") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 104, " missing)") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - } else if len(volume.MissingShards) > 2 { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 101, "Incomplete (") + } else if len(volume.MissingShards) > (erasure_coding.ParityShardsCount / 2) { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 105, "Incomplete (") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var36 string - templ_7745c5c3_Var36, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", len(volume.MissingShards))) + var templ_7745c5c3_Var40 string + templ_7745c5c3_Var40, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", len(volume.MissingShards))) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 727, Col: 139} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 728, Col: 139} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var36)) + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var40)) if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 102, " missing)") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 106, " missing)") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 103, "Minor Issues (") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 107, "Minor Issues (") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var37 string - templ_7745c5c3_Var37, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", len(volume.MissingShards))) + var templ_7745c5c3_Var41 string + templ_7745c5c3_Var41, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", len(volume.MissingShards))) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 729, Col: 138} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 730, Col: 138} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var37)) + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var41)) if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 104, " missing)") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 108, " missing)") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } diff --git a/weed/admin/view/app/cluster_volume_servers.templ b/weed/admin/view/app/cluster_volume_servers.templ index 26cb659c5..14b952dce 100644 --- a/weed/admin/view/app/cluster_volume_servers.templ +++ b/weed/admin/view/app/cluster_volume_servers.templ @@ -98,7 +98,6 @@ templ ClusterVolumeServers(data dash.ClusterVolumeServersData) { - @@ -113,9 +112,6 @@ templ ClusterVolumeServers(data dash.ClusterVolumeServersData) { for _, host := range data.VolumeServers { -
Server ID Address Data Center Rack
- {host.ID} - {host.Address} diff --git a/weed/admin/view/app/cluster_volume_servers_templ.go b/weed/admin/view/app/cluster_volume_servers_templ.go index b25f86880..7ebced18d 100644 --- a/weed/admin/view/app/cluster_volume_servers_templ.go +++ b/weed/admin/view/app/cluster_volume_servers_templ.go @@ -78,386 +78,373 @@ func ClusterVolumeServers(data dash.ClusterVolumeServersData) templ.Component { return templ_7745c5c3_Err } if len(data.VolumeServers) > 0 { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 5, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 5, "
Server IDAddressData CenterRackVolumesMax VolumesEC ShardsCapacityUsageActions
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } for _, host := range data.VolumeServers { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 6, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 34, "\">") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 36, "
AddressData CenterRackVolumesMax VolumesEC ShardsCapacityUsageActions
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 6, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var6 templ.SafeURL - templ_7745c5c3_Var6, templ_7745c5c3_Err = templ.JoinURLErrs(templ.SafeURL(fmt.Sprintf("http://%s/ui/index.html", host.PublicURL))) + var templ_7745c5c3_Var6 string + templ_7745c5c3_Var6, templ_7745c5c3_Err = templ.JoinStringErrs(host.Address) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 120, Col: 122} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 117, Col: 61} } _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var6)) if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 8, "\" target=\"_blank\" class=\"text-decoration-none\">") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 8, " ") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } var templ_7745c5c3_Var7 string - templ_7745c5c3_Var7, templ_7745c5c3_Err = templ.JoinStringErrs(host.Address) + templ_7745c5c3_Var7, templ_7745c5c3_Err = templ.JoinStringErrs(host.DataCenter) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 121, Col: 61} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 122, Col: 99} } _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var7)) if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 9, " ") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 9, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } var templ_7745c5c3_Var8 string - templ_7745c5c3_Var8, templ_7745c5c3_Err = templ.JoinStringErrs(host.DataCenter) + templ_7745c5c3_Var8, templ_7745c5c3_Err = templ.JoinStringErrs(host.Rack) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 126, Col: 99} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 125, Col: 93} } _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var8)) if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 10, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 10, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } var templ_7745c5c3_Var10 string - templ_7745c5c3_Var10, templ_7745c5c3_Err = templruntime.SanitizeStyleAttributeValues(fmt.Sprintf("width: %d%%", calculatePercent(host.Volumes, host.MaxVolumes))) + templ_7745c5c3_Var10, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", host.Volumes)) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 135, Col: 139} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 134, Col: 111} } _, 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, 12, "\">
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 12, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } var templ_7745c5c3_Var11 string - templ_7745c5c3_Var11, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", host.Volumes)) + templ_7745c5c3_Var11, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", host.MaxVolumes)) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 138, Col: 111} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 138, Col: 112} } _, 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, 13, "") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - var templ_7745c5c3_Var12 string - templ_7745c5c3_Var12, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", host.MaxVolumes)) - if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 142, Col: 112} - } - _, 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, 14, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if host.EcShards > 0 { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 14, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var13 string - templ_7745c5c3_Var13, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", host.EcShards)) + var templ_7745c5c3_Var12 string + templ_7745c5c3_Var12, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", host.EcShards)) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 148, Col: 129} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 144, Col: 129} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var13)) + _, 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, 16, " shards
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "
shards
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if host.EcVolumes > 0 { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var14 string - templ_7745c5c3_Var14, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d EC volumes", host.EcVolumes)) + var templ_7745c5c3_Var13 string + templ_7745c5c3_Var13, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d EC volumes", host.EcVolumes)) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 153, Col: 127} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 149, Col: 127} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var14)) + _, 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, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, "-") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 18, "-") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var14 string + templ_7745c5c3_Var14, templ_7745c5c3_Err = templ.JoinStringErrs(formatBytes(host.DiskCapacity)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 156, Col: 75} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var14)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } var templ_7745c5c3_Var16 string - templ_7745c5c3_Var16, templ_7745c5c3_Err = templruntime.SanitizeStyleAttributeValues(fmt.Sprintf("width: %d%%", calculatePercent(int(host.DiskUsage), int(host.DiskCapacity)))) + templ_7745c5c3_Var16, templ_7745c5c3_Err = templ.JoinStringErrs(formatBytes(host.DiskUsage)) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 165, Col: 153} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 164, Col: 83} } _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var16)) if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 22, "\">
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 22, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 35, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 37, "
No Volume Servers Found

No volume servers are currently available in the cluster.

") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 36, "
No Volume Servers Found

No volume servers are currently available in the cluster.

") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 38, "
Last updated: ") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 37, "
Last updated: ") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var30 string - templ_7745c5c3_Var30, templ_7745c5c3_Err = templ.JoinStringErrs(data.LastUpdated.Format("2006-01-02 15:04:05")) + var templ_7745c5c3_Var29 string + templ_7745c5c3_Var29, templ_7745c5c3_Err = templ.JoinStringErrs(data.LastUpdated.Format("2006-01-02 15:04:05")) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 211, Col: 81} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 207, Col: 81} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var30)) + _, 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, 39, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 38, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } diff --git a/weed/admin/view/app/collection_details.templ b/weed/admin/view/app/collection_details.templ index bd11cca81..b5c86ba18 100644 --- a/weed/admin/view/app/collection_details.templ +++ b/weed/admin/view/app/collection_details.templ @@ -262,6 +262,16 @@ templ CollectionDetails(data dash.CollectionDetailsData) { } + + // Show message when no volumes found + if len(data.RegularVolumes) == 0 && len(data.EcVolumes) == 0 { + + + + No volumes found for collection "{data.CollectionName}" + + + } diff --git a/weed/admin/view/app/collection_details_templ.go b/weed/admin/view/app/collection_details_templ.go index bb1ed9e36..b91ddebb2 100644 --- a/weed/admin/view/app/collection_details_templ.go +++ b/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, "") + if len(data.RegularVolumes) == 0 && len(data.EcVolumes) == 0 { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 47, " 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, "\"") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 49, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if data.TotalPages > 1 { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 48, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 62, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 64, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } diff --git a/weed/admin/view/app/maintenance_queue.templ b/weed/admin/view/app/maintenance_queue.templ index f16a72381..74540f285 100644 --- a/weed/admin/view/app/maintenance_queue.templ +++ b/weed/admin/view/app/maintenance_queue.templ @@ -70,6 +70,111 @@ templ MaintenanceQueue(data *maintenance.MaintenanceQueueData) { + +
+
+
+
+
+ + Completed Tasks +
+
+
+ if data.Stats.CompletedToday == 0 && data.Stats.FailedToday == 0 { +
+ +

No completed maintenance tasks today

+ Completed tasks will appear here after workers finish processing them +
+ } else { +
+ + + + + + + + + + + + + for _, task := range data.Tasks { + if string(task.Status) == "completed" || string(task.Status) == "failed" || string(task.Status) == "cancelled" { + if string(task.Status) == "failed" { + + + + + + + + + } else { + + + + + + + + + } + } + } + +
TypeStatusVolumeWorkerDurationCompleted
+ @TaskTypeIcon(task.Type) + {string(task.Type)} + @StatusBadge(task.Status){fmt.Sprintf("%d", task.VolumeID)} + if task.WorkerID != "" { + {task.WorkerID} + } else { + - + } + + if task.StartedAt != nil && task.CompletedAt != nil { + {formatDuration(task.CompletedAt.Sub(*task.StartedAt))} + } else { + - + } + + if task.CompletedAt != nil { + {task.CompletedAt.Format("2006-01-02 15:04")} + } else { + - + } +
+ @TaskTypeIcon(task.Type) + {string(task.Type)} + @StatusBadge(task.Status){fmt.Sprintf("%d", task.VolumeID)} + if task.WorkerID != "" { + {task.WorkerID} + } else { + - + } + + if task.StartedAt != nil && task.CompletedAt != nil { + {formatDuration(task.CompletedAt.Sub(*task.StartedAt))} + } else { + - + } + + if task.CompletedAt != nil { + {task.CompletedAt.Format("2006-01-02 15:04")} + } else { + - + } +
+
+ } +
+
+
+
+
@@ -103,7 +208,7 @@ templ MaintenanceQueue(data *maintenance.MaintenanceQueueData) { for _, task := range data.Tasks { if string(task.Status) == "pending" { - + @TaskTypeIcon(task.Type) {string(task.Type)} @@ -158,7 +263,7 @@ templ MaintenanceQueue(data *maintenance.MaintenanceQueueData) { for _, task := range data.Tasks { if string(task.Status) == "assigned" || string(task.Status) == "in_progress" { - + @TaskTypeIcon(task.Type) {string(task.Type)} @@ -191,111 +296,6 @@ templ MaintenanceQueue(data *maintenance.MaintenanceQueueData) {
- - -
-
-
-
-
- - Completed Tasks -
-
-
- if data.Stats.CompletedToday == 0 && data.Stats.FailedToday == 0 { -
- -

No completed maintenance tasks today

- Completed tasks will appear here after workers finish processing them -
- } else { -
- - - - - - - - - - - - - for _, task := range data.Tasks { - if string(task.Status) == "completed" || string(task.Status) == "failed" || string(task.Status) == "cancelled" { - if string(task.Status) == "failed" { - - - - - - - - - } else { - - - - - - - - - } - } - } - -
TypeStatusVolumeWorkerDurationCompleted
- @TaskTypeIcon(task.Type) - {string(task.Type)} - @StatusBadge(task.Status){fmt.Sprintf("%d", task.VolumeID)} - if task.WorkerID != "" { - {task.WorkerID} - } else { - - - } - - if task.StartedAt != nil && task.CompletedAt != nil { - {formatDuration(task.CompletedAt.Sub(*task.StartedAt))} - } else { - - - } - - if task.CompletedAt != nil { - {task.CompletedAt.Format("2006-01-02 15:04")} - } else { - - - } -
- @TaskTypeIcon(task.Type) - {string(task.Type)} - @StatusBadge(task.Status){fmt.Sprintf("%d", task.VolumeID)} - if task.WorkerID != "" { - {task.WorkerID} - } else { - - - } - - if task.StartedAt != nil && task.CompletedAt != nil { - {formatDuration(task.CompletedAt.Sub(*task.StartedAt))} - } else { - - - } - - if task.CompletedAt != nil { - {task.CompletedAt.Format("2006-01-02 15:04")} - } else { - - - } -
-
- } -
-
-
-
} diff --git a/weed/admin/view/app/maintenance_queue_templ.go b/weed/admin/view/app/maintenance_queue_templ.go index 35ee421af..f4d8d1ea6 100644 --- a/weed/admin/view/app/maintenance_queue_templ.go +++ b/weed/admin/view/app/maintenance_queue_templ.go @@ -87,249 +87,37 @@ func MaintenanceQueue(data *maintenance.MaintenanceQueueData) templ.Component { if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 5, "

Failed Today

Pending Tasks
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 5, "

Failed Today

Completed Tasks
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - if data.Stats.PendingTasks == 0 { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 6, "

No pending maintenance tasks

Pending tasks will appear here when the system detects maintenance needs
") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 7, "
") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - for _, task := range data.Tasks { - if string(task.Status) == "pending" { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 8, "") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - } - } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "
TypePriorityVolumeServerReasonCreated
") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - templ_7745c5c3_Err = TaskTypeIcon(task.Type).Render(ctx, templ_7745c5c3_Buffer) - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - var templ_7745c5c3_Var6 string - templ_7745c5c3_Var6, templ_7745c5c3_Err = templ.JoinStringErrs(string(task.Type)) - if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 109, Col: 74} - } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var6)) - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 9, "") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - templ_7745c5c3_Err = PriorityBadge(task.Priority).Render(ctx, templ_7745c5c3_Buffer) - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 10, "") - 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", task.VolumeID)) - if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 112, Col: 89} - } - _, 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, "") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - var templ_7745c5c3_Var8 string - templ_7745c5c3_Var8, templ_7745c5c3_Err = templ.JoinStringErrs(task.Server) - if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 113, Col: 75} - } - _, 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, 12, "") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - var templ_7745c5c3_Var9 string - templ_7745c5c3_Var9, templ_7745c5c3_Err = templ.JoinStringErrs(task.Reason) - if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 114, Col: 75} - } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var9)) - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, "") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - var templ_7745c5c3_Var10 string - templ_7745c5c3_Var10, templ_7745c5c3_Err = templ.JoinStringErrs(task.CreatedAt.Format("2006-01-02 15:04")) - if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 115, Col: 98} - } - _, 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, 14, "
") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, "
Active Tasks
") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - if data.Stats.RunningTasks == 0 { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, "

No active maintenance tasks

Active tasks will appear here when workers start processing them
") + if data.Stats.CompletedToday == 0 && data.Stats.FailedToday == 0 { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 6, "

No completed maintenance tasks today

Completed tasks will appear here after workers finish processing them
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 18, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 7, "
TypeStatusProgressVolumeWorkerStarted
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } for _, task := range data.Tasks { - if string(task.Status) == "assigned" || string(task.Status) == "in_progress" { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, "") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - } - } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 30, "
TypeStatusVolumeWorkerDurationCompleted
") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - templ_7745c5c3_Err = TaskTypeIcon(task.Type).Render(ctx, templ_7745c5c3_Buffer) - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - var templ_7745c5c3_Var11 string - templ_7745c5c3_Var11, templ_7745c5c3_Err = templ.JoinStringErrs(string(task.Type)) - if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 164, Col: 74} - } - _, 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, 20, "") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - templ_7745c5c3_Err = StatusBadge(task.Status).Render(ctx, templ_7745c5c3_Buffer) - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, "") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - templ_7745c5c3_Err = ProgressBar(task.Progress, task.Status).Render(ctx, templ_7745c5c3_Buffer) - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 22, "") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - var templ_7745c5c3_Var12 string - templ_7745c5c3_Var12, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", task.VolumeID)) - if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 168, Col: 89} - } - _, 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, 23, "") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - if task.WorkerID != "" { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, "") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - var templ_7745c5c3_Var13 string - templ_7745c5c3_Var13, templ_7745c5c3_Err = templ.JoinStringErrs(task.WorkerID) - if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 171, Col: 81} - } - _, 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, 25, "") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, "-") + if string(task.Status) == "completed" || string(task.Status) == "failed" || string(task.Status) == "cancelled" { + if string(task.Status) == "failed" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 8, "
") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - if task.StartedAt != nil { - var templ_7745c5c3_Var14 string - templ_7745c5c3_Var14, templ_7745c5c3_Err = templ.JoinStringErrs(task.StartedAt.Format("2006-01-02 15:04")) + var templ_7745c5c3_Var6 string + templ_7745c5c3_Var6, templ_7745c5c3_Err = templ.JoinStringErrs(task.ID) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 178, Col: 102} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 107, Col: 112} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var14)) + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var6)) if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, "-") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, "
") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 31, "
Completed Tasks
") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - if data.Stats.CompletedToday == 0 && data.Stats.FailedToday == 0 { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 32, "

No completed maintenance tasks today

Completed tasks will appear here after workers finish processing them
") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 33, "
") - if templ_7745c5c3_Err != nil { - return templ_7745c5c3_Err - } - for _, task := range data.Tasks { - if string(task.Status) == "completed" || string(task.Status) == "failed" || string(task.Status) == "cancelled" { - if string(task.Status) == "failed" { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 34, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 46, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 33, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 34, "
TypeStatusVolumeWorkerDurationCompleted
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 9, "\" onclick=\"navigateToTask(this)\" style=\"cursor: pointer;\">") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } @@ -337,16 +125,16 @@ func MaintenanceQueue(data *maintenance.MaintenanceQueueData) templ.Component { if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var15 string - templ_7745c5c3_Var15, templ_7745c5c3_Err = templ.JoinStringErrs(string(task.Type)) + var templ_7745c5c3_Var7 string + templ_7745c5c3_Var7, templ_7745c5c3_Err = templ.JoinStringErrs(string(task.Type)) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 232, Col: 78} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 110, Col: 78} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var15)) + _, 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, 35, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 10, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } @@ -354,93 +142,106 @@ func MaintenanceQueue(data *maintenance.MaintenanceQueueData) templ.Component { if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 36, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 11, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var16 string - templ_7745c5c3_Var16, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", task.VolumeID)) + var templ_7745c5c3_Var8 string + templ_7745c5c3_Var8, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", task.VolumeID)) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 235, Col: 93} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 113, Col: 93} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var16)) + _, 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, 37, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 12, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if task.WorkerID != "" { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 38, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var17 string - templ_7745c5c3_Var17, templ_7745c5c3_Err = templ.JoinStringErrs(task.WorkerID) + var templ_7745c5c3_Var9 string + templ_7745c5c3_Var9, templ_7745c5c3_Err = templ.JoinStringErrs(task.WorkerID) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 238, Col: 85} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 116, Col: 85} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var17)) + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var9)) if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 39, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 14, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 40, "-") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "-") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 41, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if task.StartedAt != nil && task.CompletedAt != nil { - var templ_7745c5c3_Var18 string - templ_7745c5c3_Var18, templ_7745c5c3_Err = templ.JoinStringErrs(formatDuration(task.CompletedAt.Sub(*task.StartedAt))) + var templ_7745c5c3_Var10 string + templ_7745c5c3_Var10, templ_7745c5c3_Err = templ.JoinStringErrs(formatDuration(task.CompletedAt.Sub(*task.StartedAt))) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 245, Col: 118} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 123, Col: 118} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var18)) + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var10)) if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 42, "-") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, "-") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 43, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 18, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if task.CompletedAt != nil { - var templ_7745c5c3_Var19 string - templ_7745c5c3_Var19, templ_7745c5c3_Err = templ.JoinStringErrs(task.CompletedAt.Format("2006-01-02 15:04")) + var templ_7745c5c3_Var11 string + templ_7745c5c3_Var11, templ_7745c5c3_Err = templ.JoinStringErrs(task.CompletedAt.Format("2006-01-02 15:04")) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 252, Col: 108} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 130, Col: 108} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var19)) + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var11)) if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 44, "-") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, "-") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 45, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } @@ -448,16 +249,16 @@ func MaintenanceQueue(data *maintenance.MaintenanceQueueData) templ.Component { if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var20 string - templ_7745c5c3_Var20, templ_7745c5c3_Err = templ.JoinStringErrs(string(task.Type)) + var templ_7745c5c3_Var13 string + templ_7745c5c3_Var13, templ_7745c5c3_Err = templ.JoinStringErrs(string(task.Type)) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 262, Col: 78} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 140, Col: 78} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var20)) + _, 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, 47, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 23, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } @@ -465,100 +266,351 @@ func MaintenanceQueue(data *maintenance.MaintenanceQueueData) templ.Component { if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 48, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, "") 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", task.VolumeID)) + var templ_7745c5c3_Var14 string + templ_7745c5c3_Var14, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", task.VolumeID)) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 265, Col: 93} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 143, Col: 93} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var21)) + _, 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, 49, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 25, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if task.WorkerID != "" { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 50, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var22 string - templ_7745c5c3_Var22, templ_7745c5c3_Err = templ.JoinStringErrs(task.WorkerID) + var templ_7745c5c3_Var15 string + templ_7745c5c3_Var15, templ_7745c5c3_Err = templ.JoinStringErrs(task.WorkerID) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 268, Col: 85} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 146, Col: 85} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var22)) + _, 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, 51, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 52, "-") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, "-") 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, 29, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if task.StartedAt != nil && task.CompletedAt != nil { - var templ_7745c5c3_Var23 string - templ_7745c5c3_Var23, templ_7745c5c3_Err = templ.JoinStringErrs(formatDuration(task.CompletedAt.Sub(*task.StartedAt))) + var templ_7745c5c3_Var16 string + templ_7745c5c3_Var16, templ_7745c5c3_Err = templ.JoinStringErrs(formatDuration(task.CompletedAt.Sub(*task.StartedAt))) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 275, Col: 118} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 153, Col: 118} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var23)) + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var16)) if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 54, "-") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 30, "-") 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, 31, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if task.CompletedAt != nil { - var templ_7745c5c3_Var24 string - templ_7745c5c3_Var24, templ_7745c5c3_Err = templ.JoinStringErrs(task.CompletedAt.Format("2006-01-02 15:04")) + var templ_7745c5c3_Var17 string + templ_7745c5c3_Var17, templ_7745c5c3_Err = templ.JoinStringErrs(task.CompletedAt.Format("2006-01-02 15:04")) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 282, Col: 108} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 160, Col: 108} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var24)) + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var17)) if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 56, "-") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 32, "-") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 57, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 35, "
Pending Tasks
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if data.Stats.PendingTasks == 0 { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 36, "

No pending maintenance tasks

Pending tasks will appear here when the system detects maintenance needs
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 37, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + for _, task := range data.Tasks { + if string(task.Status) == "pending" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 38, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 46, "
TypePriorityVolumeServerReasonCreated
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = TaskTypeIcon(task.Type).Render(ctx, templ_7745c5c3_Buffer) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var19 string + templ_7745c5c3_Var19, templ_7745c5c3_Err = templ.JoinStringErrs(string(task.Type)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 214, Col: 74} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var19)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 40, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = PriorityBadge(task.Priority).Render(ctx, templ_7745c5c3_Buffer) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 41, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var20 string + templ_7745c5c3_Var20, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", task.VolumeID)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 217, Col: 89} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var20)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 42, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var21 string + templ_7745c5c3_Var21, templ_7745c5c3_Err = templ.JoinStringErrs(task.Server) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 218, Col: 75} + } + _, 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, 43, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var22 string + templ_7745c5c3_Var22, templ_7745c5c3_Err = templ.JoinStringErrs(task.Reason) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 219, Col: 75} + } + _, 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, 44, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var23 string + templ_7745c5c3_Var23, templ_7745c5c3_Err = templ.JoinStringErrs(task.CreatedAt.Format("2006-01-02 15:04")) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 220, Col: 98} + } + _, 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, 45, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 47, "
Active Tasks
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if data.Stats.RunningTasks == 0 { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 48, "

No active maintenance tasks

Active tasks will appear here when workers start processing them
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 49, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + for _, task := range data.Tasks { + if string(task.Status) == "assigned" || string(task.Status) == "in_progress" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 50, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 58, "
TypeStatusProgressVolumeWorkerStarted
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = TaskTypeIcon(task.Type).Render(ctx, templ_7745c5c3_Buffer) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var25 string + templ_7745c5c3_Var25, templ_7745c5c3_Err = templ.JoinStringErrs(string(task.Type)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 269, Col: 74} + } + _, 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, 52, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = StatusBadge(task.Status).Render(ctx, templ_7745c5c3_Buffer) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 53, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = ProgressBar(task.Progress, task.Status).Render(ctx, templ_7745c5c3_Buffer) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 54, "") + 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", task.VolumeID)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 273, Col: 89} + } + _, 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, 55, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if task.WorkerID != "" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 56, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var27 string + templ_7745c5c3_Var27, templ_7745c5c3_Err = templ.JoinStringErrs(task.WorkerID) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 276, Col: 81} + } + _, 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, 57, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 58, "-") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 59, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if task.StartedAt != nil { + var templ_7745c5c3_Var28 string + templ_7745c5c3_Var28, templ_7745c5c3_Err = templ.JoinStringErrs(task.StartedAt.Format("2006-01-02 15:04")) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 283, Col: 102} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var28)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 60, "-") + 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, 62, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 59, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 63, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } @@ -583,30 +635,30 @@ func TaskTypeIcon(taskType maintenance.MaintenanceTaskType) templ.Component { }() } ctx = templ.InitializeContext(ctx) - templ_7745c5c3_Var25 := templ.GetChildren(ctx) - if templ_7745c5c3_Var25 == nil { - templ_7745c5c3_Var25 = templ.NopComponent + templ_7745c5c3_Var29 := templ.GetChildren(ctx) + if templ_7745c5c3_Var29 == nil { + templ_7745c5c3_Var29 = templ.NopComponent } ctx = templ.ClearChildren(ctx) - var templ_7745c5c3_Var26 = []any{maintenance.GetTaskIcon(taskType) + " me-1"} - templ_7745c5c3_Err = templ.RenderCSSItems(ctx, templ_7745c5c3_Buffer, templ_7745c5c3_Var26...) + var templ_7745c5c3_Var30 = []any{maintenance.GetTaskIcon(taskType) + " me-1"} + templ_7745c5c3_Err = templ.RenderCSSItems(ctx, templ_7745c5c3_Buffer, templ_7745c5c3_Var30...) if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 60, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 65, "\">") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } @@ -630,34 +682,34 @@ func PriorityBadge(priority maintenance.MaintenanceTaskPriority) templ.Component }() } ctx = templ.InitializeContext(ctx) - templ_7745c5c3_Var28 := templ.GetChildren(ctx) - if templ_7745c5c3_Var28 == nil { - templ_7745c5c3_Var28 = templ.NopComponent + templ_7745c5c3_Var32 := templ.GetChildren(ctx) + if templ_7745c5c3_Var32 == nil { + templ_7745c5c3_Var32 = templ.NopComponent } ctx = templ.ClearChildren(ctx) switch priority { case maintenance.PriorityCritical: - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 62, "Critical") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 66, "Critical") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } case maintenance.PriorityHigh: - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 63, "High") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 67, "High") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } case maintenance.PriorityNormal: - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 64, "Normal") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 68, "Normal") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } case maintenance.PriorityLow: - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 65, "Low") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 69, "Low") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } default: - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 66, "Unknown") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 70, "Unknown") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } @@ -682,44 +734,44 @@ func StatusBadge(status maintenance.MaintenanceTaskStatus) templ.Component { }() } ctx = templ.InitializeContext(ctx) - templ_7745c5c3_Var29 := templ.GetChildren(ctx) - if templ_7745c5c3_Var29 == nil { - templ_7745c5c3_Var29 = templ.NopComponent + templ_7745c5c3_Var33 := templ.GetChildren(ctx) + if templ_7745c5c3_Var33 == nil { + templ_7745c5c3_Var33 = templ.NopComponent } ctx = templ.ClearChildren(ctx) switch status { case maintenance.TaskStatusPending: - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 67, "Pending") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 71, "Pending") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } case maintenance.TaskStatusAssigned: - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 68, "Assigned") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 72, "Assigned") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } case maintenance.TaskStatusInProgress: - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 69, "Running") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 73, "Running") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } case maintenance.TaskStatusCompleted: - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 70, "Completed") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 74, "Completed") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } case maintenance.TaskStatusFailed: - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 71, "Failed") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 75, "Failed") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } case maintenance.TaskStatusCancelled: - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 72, "Cancelled") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 76, "Cancelled") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } default: - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 73, "Unknown") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 77, "Unknown") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } @@ -744,49 +796,49 @@ func ProgressBar(progress float64, status maintenance.MaintenanceTaskStatus) tem }() } ctx = templ.InitializeContext(ctx) - templ_7745c5c3_Var30 := templ.GetChildren(ctx) - if templ_7745c5c3_Var30 == nil { - templ_7745c5c3_Var30 = templ.NopComponent + templ_7745c5c3_Var34 := templ.GetChildren(ctx) + if templ_7745c5c3_Var34 == nil { + templ_7745c5c3_Var34 = templ.NopComponent } ctx = templ.ClearChildren(ctx) if status == maintenance.TaskStatusInProgress || status == maintenance.TaskStatusAssigned { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 74, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 79, "\">
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } - var templ_7745c5c3_Var32 string - templ_7745c5c3_Var32, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%.1f%%", progress)) + var templ_7745c5c3_Var36 string + templ_7745c5c3_Var36, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%.1f%%", progress)) if templ_7745c5c3_Err != nil { - return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 386, Col: 66} + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 393, Col: 66} } - _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var32)) + _, 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, 76, "") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 80, "") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else if status == maintenance.TaskStatusCompleted { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 77, "
100%") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 81, "
100%") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 78, "-") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 82, "-") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } diff --git a/weed/admin/view/app/task_config_schema.templ b/weed/admin/view/app/task_config_schema.templ index 174a8f580..bc2f29661 100644 --- a/weed/admin/view/app/task_config_schema.templ +++ b/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" {
Erasure Coding Operations:

Performance: Erasure coding is CPU and I/O intensive. Consider running during off-peak hours.

-

Durability: With 10+4 configuration, can tolerate up to 4 shard failures.

+

Durability: With { fmt.Sprintf("%d+%d", erasure_coding.DataShardsCount, erasure_coding.ParityShardsCount) } configuration, can tolerate up to { fmt.Sprintf("%d", erasure_coding.ParityShardsCount) } shard failures.

Configuration: Fullness ratio should be between 0.5 and 1.0 (e.g., 0.90 for 90%).

} diff --git a/weed/admin/view/app/task_config_schema_templ.go b/weed/admin/view/app/task_config_schema_templ.go index eae4683d9..258542e39 100644 --- a/weed/admin/view/app/task_config_schema_templ.go +++ b/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, "
Erasure Coding Operations:

Performance: Erasure coding is CPU and I/O intensive. Consider running during off-peak hours.

Durability: With 10+4 configuration, can tolerate up to 4 shard failures.

Configuration: Fullness ratio should be between 0.5 and 1.0 (e.g., 0.90 for 90%).

") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 9, "
Erasure Coding Operations:

Performance: Erasure coding is CPU and I/O intensive. Consider running during off-peak hours.

Durability: 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.

Configuration: Fullness ratio should be between 0.5 and 1.0 (e.g., 0.90 for 90%).

") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 10, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, "\" style=\"display: none;\">") 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, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else if field.InputType == "checkbox" { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 35, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 43, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } if field.Description != "" { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 42, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 44, "
") 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, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 45, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 44, "
") + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 46, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else if field.InputType == "text" { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 45, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } } else { - templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 59, "
") if templ_7745c5c3_Err != nil { return templ_7745c5c3_Err } diff --git a/weed/admin/view/app/task_detail.templ b/weed/admin/view/app/task_detail.templ new file mode 100644 index 000000000..6045a5301 --- /dev/null +++ b/weed/admin/view/app/task_detail.templ @@ -0,0 +1,1118 @@ +package app + +import ( + "fmt" + "sort" + "github.com/seaweedfs/seaweedfs/weed/admin/maintenance" + "github.com/seaweedfs/seaweedfs/weed/storage/erasure_coding" +) + +// sortedKeys returns the sorted keys for a string map +func sortedKeys(m map[string]string) []string { + keys := make([]string, 0, len(m)) + for k := range m { + keys = append(keys, k) + } + sort.Strings(keys) + return keys +} + +templ TaskDetail(data *maintenance.TaskDetailData) { +
+ +
+
+
+
+ +

+ + Task Detail: {data.Task.ID} +

+
+
+ + +
+
+
+
+ + +
+
+
+
+
+ + Task Overview +
+
+
+
+
+
+
Task ID:
+
{data.Task.ID}
+ +
Type:
+
+ {string(data.Task.Type)} +
+ +
Status:
+
+ if data.Task.Status == maintenance.TaskStatusPending { + Pending + } else if data.Task.Status == maintenance.TaskStatusAssigned { + Assigned + } else if data.Task.Status == maintenance.TaskStatusInProgress { + In Progress + } else if data.Task.Status == maintenance.TaskStatusCompleted { + Completed + } else if data.Task.Status == maintenance.TaskStatusFailed { + Failed + } else if data.Task.Status == maintenance.TaskStatusCancelled { + Cancelled + } +
+ +
Priority:
+
+ if data.Task.Priority == maintenance.PriorityHigh { + High + } else if data.Task.Priority == maintenance.PriorityCritical { + Critical + } else if data.Task.Priority == maintenance.PriorityNormal { + Normal + } else { + Low + } +
+ + if data.Task.Reason != "" { +
Reason:
+
+ {data.Task.Reason} +
+ } +
+
+
+ +
+
+ Task Timeline +
+
+
+
+
+ +
+
+
+ Created + {data.Task.CreatedAt.Format("01-02 15:04:05")} +
+
+ +
+
+ +
+ if data.Task.StartedAt != nil { +
+ } else { +
+ } +
+ Scheduled + {data.Task.ScheduledAt.Format("01-02 15:04:05")} +
+
+ +
+ if data.Task.StartedAt != nil { +
+ +
+ } else { +
+ +
+ } + if data.Task.CompletedAt != nil { +
+ } else { +
+ } +
+ Started + + if data.Task.StartedAt != nil { + {data.Task.StartedAt.Format("01-02 15:04:05")} + } else { + — + } + +
+
+ +
+ if data.Task.CompletedAt != nil { +
+ if data.Task.Status == maintenance.TaskStatusCompleted { + + } else if data.Task.Status == maintenance.TaskStatusFailed { + + } else { + + } +
+ } else { +
+ +
+ } +
+ + if data.Task.Status == maintenance.TaskStatusCompleted { + Completed + } else if data.Task.Status == maintenance.TaskStatusFailed { + Failed + } else if data.Task.Status == maintenance.TaskStatusCancelled { + Cancelled + } else { + Pending + } + + + if data.Task.CompletedAt != nil { + {data.Task.CompletedAt.Format("01-02 15:04:05")} + } else { + — + } + +
+
+
+
+
+ + + if data.Task.WorkerID != "" { +
+
Worker:
+
{data.Task.WorkerID}
+
+ } + +
+ if data.Task.TypedParams != nil && data.Task.TypedParams.VolumeSize > 0 { +
Volume Size:
+
+ {formatBytes(int64(data.Task.TypedParams.VolumeSize))} +
+ } + + if data.Task.TypedParams != nil && data.Task.TypedParams.Collection != "" { +
Collection:
+
+ {data.Task.TypedParams.Collection} +
+ } + + if data.Task.TypedParams != nil && data.Task.TypedParams.DataCenter != "" { +
Data Center:
+
+ {data.Task.TypedParams.DataCenter} +
+ } + + if data.Task.Progress > 0 { +
Progress:
+
+
+
+ {fmt.Sprintf("%.1f%%", data.Task.Progress)} +
+
+
+ } +
+
+
+ + + + if data.Task.DetailedReason != "" { +
+
+
Detailed Reason:
+

{data.Task.DetailedReason}

+
+
+ } + + if data.Task.Error != "" { +
+
+
Error:
+
+ {data.Task.Error} +
+
+
+ } +
+
+
+
+ + + if data.Task.TypedParams != nil { +
+
+
+
+
+ + Task Configuration +
+
+
+ + if len(data.Task.TypedParams.Sources) > 0 { +
+
+ + Source Servers + {fmt.Sprintf("%d", len(data.Task.TypedParams.Sources))} +
+
+
+ for i, source := range data.Task.TypedParams.Sources { +
+ {fmt.Sprintf("#%d", i+1)} + {source.Node} +
+ if source.DataCenter != "" { + + {source.DataCenter} + + } +
+
+ if source.Rack != "" { + + {source.Rack} + + } +
+
+ if source.VolumeId > 0 { + + Vol:{fmt.Sprintf("%d", source.VolumeId)} + + } +
+
+ if len(source.ShardIds) > 0 { + + Shards: + for j, shardId := range source.ShardIds { + if j > 0 { + , + } + if shardId < erasure_coding.DataShardsCount { + {fmt.Sprintf("%d", shardId)} + } else { + {fmt.Sprintf("P%d", shardId-erasure_coding.DataShardsCount)} + } + } + + } +
+
+ } +
+
+
+ } + + + if len(data.Task.TypedParams.Sources) > 0 || len(data.Task.TypedParams.Targets) > 0 { +
+ +
+ Task: {string(data.Task.Type)} +
+ } + + + if len(data.Task.TypedParams.Targets) > 0 { +
+
+ + Target Servers + {fmt.Sprintf("%d", len(data.Task.TypedParams.Targets))} +
+
+
+ for i, target := range data.Task.TypedParams.Targets { +
+ {fmt.Sprintf("#%d", i+1)} + {target.Node} +
+ if target.DataCenter != "" { + + {target.DataCenter} + + } +
+
+ if target.Rack != "" { + + {target.Rack} + + } +
+
+ if target.VolumeId > 0 { + + Vol:{fmt.Sprintf("%d", target.VolumeId)} + + } +
+
+ if len(target.ShardIds) > 0 { + + Shards: + for j, shardId := range target.ShardIds { + if j > 0 { + , + } + if shardId < erasure_coding.DataShardsCount { + {fmt.Sprintf("%d", shardId)} + } else { + {fmt.Sprintf("P%d", shardId-erasure_coding.DataShardsCount)} + } + } + + } +
+
+ } +
+
+
+ } +
+
+
+
+ } + + + if data.WorkerInfo != nil { +
+
+
+
+
+ + Worker Information +
+
+
+
+
+
+
Worker ID:
+
{data.WorkerInfo.ID}
+ +
Address:
+
{data.WorkerInfo.Address}
+ +
Status:
+
+ if data.WorkerInfo.Status == "active" { + Active + } else if data.WorkerInfo.Status == "busy" { + Busy + } else { + Inactive + } +
+
+
+
+
+
Last Heartbeat:
+
{data.WorkerInfo.LastHeartbeat.Format("2006-01-02 15:04:05")}
+ +
Current Load:
+
{fmt.Sprintf("%d/%d", data.WorkerInfo.CurrentLoad, data.WorkerInfo.MaxConcurrent)}
+ +
Capabilities:
+
+ for _, capability := range data.WorkerInfo.Capabilities { + {string(capability)} + } +
+
+
+
+
+
+
+
+ } + + + if len(data.AssignmentHistory) > 0 { +
+
+
+
+
+ + Assignment History +
+
+
+
+ + + + + + + + + + + + for _, assignment := range data.AssignmentHistory { + + + + + + + + } + +
Worker IDWorker AddressAssigned AtUnassigned AtReason
{assignment.WorkerID}{assignment.WorkerAddress}{assignment.AssignedAt.Format("2006-01-02 15:04:05")} + if assignment.UnassignedAt != nil { + {assignment.UnassignedAt.Format("2006-01-02 15:04:05")} + } else { + + } + {assignment.Reason}
+
+
+
+
+
+ } + + + if len(data.ExecutionLogs) > 0 { +
+
+
+
+
+ + Execution Logs +
+
+
+
+ + + + + + + + + + + for _, log := range data.ExecutionLogs { + + + + + + + } + +
TimestampLevelMessageDetails
{log.Timestamp.Format("15:04:05")} + if log.Level == "error" { + {log.Level} + } else if log.Level == "warn" { + {log.Level} + } else if log.Level == "info" { + {log.Level} + } else { + {log.Level} + } + {log.Message} + if log.Fields != nil && len(log.Fields) > 0 { + + for _, k := range sortedKeys(log.Fields) { + {k}={log.Fields[k]} + } + + } else if log.Progress != nil || log.Status != "" { + + if log.Progress != nil { + progress={fmt.Sprintf("%.0f%%", *log.Progress)} + } + if log.Status != "" { + status={log.Status} + } + + } else { + - + } +
+
+
+
+
+
+ } + + + if len(data.RelatedTasks) > 0 { +
+
+
+
+
+ + Related Tasks +
+
+
+
+ + + + + + + + + + + + + for _, relatedTask := range data.RelatedTasks { + + + + + + + + + } + +
Task IDTypeStatusVolume IDServerCreated
+ + {relatedTask.ID} + + {string(relatedTask.Type)} + if relatedTask.Status == maintenance.TaskStatusCompleted { + Completed + } else if relatedTask.Status == maintenance.TaskStatusFailed { + Failed + } else if relatedTask.Status == maintenance.TaskStatusInProgress { + In Progress + } else { + {string(relatedTask.Status)} + } + + if relatedTask.VolumeID != 0 { + {fmt.Sprintf("%d", relatedTask.VolumeID)} + } else { + - + } + + if relatedTask.Server != "" { + {relatedTask.Server} + } else { + - + } + {relatedTask.CreatedAt.Format("2006-01-02 15:04:05")}
+
+
+
+
+
+ } + + +
+
+
+
+
+ + Actions +
+
+
+ if data.Task.Status == maintenance.TaskStatusPending || data.Task.Status == maintenance.TaskStatusAssigned { + + } + if data.Task.WorkerID != "" { + + } + +
+
+
+
+
+ + + + + + + +} diff --git a/weed/admin/view/app/task_detail_templ.go b/weed/admin/view/app/task_detail_templ.go new file mode 100644 index 000000000..43103e6a9 --- /dev/null +++ b/weed/admin/view/app/task_detail_templ.go @@ -0,0 +1,1628 @@ +// Code generated by templ - DO NOT EDIT. + +// templ: version: v0.3.906 +package app + +//lint:file-ignore SA4006 This context is only used if a nested component is present. + +import "github.com/a-h/templ" +import templruntime "github.com/a-h/templ/runtime" + +import ( + "fmt" + "github.com/seaweedfs/seaweedfs/weed/admin/maintenance" + "github.com/seaweedfs/seaweedfs/weed/storage/erasure_coding" + "sort" +) + +// sortedKeys returns the sorted keys for a string map +func sortedKeys(m map[string]string) []string { + keys := make([]string, 0, len(m)) + for k := range m { + keys = append(keys, k) + } + sort.Strings(keys) + return keys +} + +func TaskDetail(data *maintenance.TaskDetailData) templ.Component { + return templruntime.GeneratedTemplate(func(templ_7745c5c3_Input templruntime.GeneratedComponentInput) (templ_7745c5c3_Err error) { + templ_7745c5c3_W, ctx := templ_7745c5c3_Input.Writer, templ_7745c5c3_Input.Context + if templ_7745c5c3_CtxErr := ctx.Err(); templ_7745c5c3_CtxErr != nil { + return templ_7745c5c3_CtxErr + } + templ_7745c5c3_Buffer, templ_7745c5c3_IsBuffer := templruntime.GetBuffer(templ_7745c5c3_W) + if !templ_7745c5c3_IsBuffer { + defer func() { + templ_7745c5c3_BufErr := templruntime.ReleaseBuffer(templ_7745c5c3_Buffer) + if templ_7745c5c3_Err == nil { + templ_7745c5c3_Err = templ_7745c5c3_BufErr + } + }() + } + ctx = templ.InitializeContext(ctx) + templ_7745c5c3_Var1 := templ.GetChildren(ctx) + if templ_7745c5c3_Var1 == nil { + templ_7745c5c3_Var1 = templ.NopComponent + } + ctx = templ.ClearChildren(ctx) + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 1, "

Task Detail: ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var2 string + templ_7745c5c3_Var2, templ_7745c5c3_Err = templ.JoinStringErrs(data.Task.ID) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 35, Col: 54} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var2)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 2, "

Task Overview
Task ID:
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var3 string + templ_7745c5c3_Var3, templ_7745c5c3_Err = templ.JoinStringErrs(data.Task.ID) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 67, Col: 76} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var3)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 3, "
Type:
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var4 string + templ_7745c5c3_Var4, templ_7745c5c3_Err = templ.JoinStringErrs(string(data.Task.Type)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 71, Col: 91} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var4)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 4, "
Status:
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if data.Task.Status == maintenance.TaskStatusPending { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 5, "Pending") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else if data.Task.Status == maintenance.TaskStatusAssigned { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 6, "Assigned") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else if data.Task.Status == maintenance.TaskStatusInProgress { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 7, "In Progress") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else if data.Task.Status == maintenance.TaskStatusCompleted { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 8, "Completed") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else if data.Task.Status == maintenance.TaskStatusFailed { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 9, "Failed") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else if data.Task.Status == maintenance.TaskStatusCancelled { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 10, "Cancelled") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 11, "
Priority:
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if data.Task.Priority == maintenance.PriorityHigh { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 12, "High") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else if data.Task.Priority == maintenance.PriorityCritical { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, "Critical") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else if data.Task.Priority == maintenance.PriorityNormal { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 14, "Normal") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "Low") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if data.Task.Reason != "" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, "
Reason:
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var5 string + templ_7745c5c3_Var5, templ_7745c5c3_Err = templ.JoinStringErrs(data.Task.Reason) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 107, Col: 86} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var5)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 18, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, "
Task Timeline
Created ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var6 string + templ_7745c5c3_Var6, templ_7745c5c3_Err = templ.JoinStringErrs(data.Task.CreatedAt.Format("01-02 15:04:05")) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 127, Col: 131} + } + _, 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, 20, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if data.Task.StartedAt != nil { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 22, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 23, "
Scheduled ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var7 string + templ_7745c5c3_Var7, templ_7745c5c3_Err = templ.JoinStringErrs(data.Task.ScheduledAt.Format("01-02 15:04:05")) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 142, Col: 133} + } + _, 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, 24, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if data.Task.StartedAt != nil { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 25, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + if data.Task.CompletedAt != nil { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, "
Started ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if data.Task.StartedAt != nil { + var templ_7745c5c3_Var8 string + templ_7745c5c3_Var8, templ_7745c5c3_Err = templ.JoinStringErrs(data.Task.StartedAt.Format("01-02 15:04:05")) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 165, Col: 105} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var8)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 30, "—") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 31, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if data.Task.CompletedAt != nil { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 32, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if data.Task.Status == maintenance.TaskStatusCompleted { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 33, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else if data.Task.Status == maintenance.TaskStatusFailed { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 34, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 35, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 36, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 37, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 38, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if data.Task.Status == maintenance.TaskStatusCompleted { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 39, "Completed") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else if data.Task.Status == maintenance.TaskStatusFailed { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 40, "Failed") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else if data.Task.Status == maintenance.TaskStatusCancelled { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 41, "Cancelled") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 42, "Pending") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 43, " ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if data.Task.CompletedAt != nil { + var templ_7745c5c3_Var9 string + templ_7745c5c3_Var9, templ_7745c5c3_Err = templ.JoinStringErrs(data.Task.CompletedAt.Format("01-02 15:04:05")) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 203, Col: 107} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var9)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 44, "—") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 45, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if data.Task.WorkerID != "" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 46, "
Worker:
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var10 string + templ_7745c5c3_Var10, templ_7745c5c3_Err = templ.JoinStringErrs(data.Task.WorkerID) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 218, Col: 86} + } + _, 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, 47, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 48, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if data.Task.TypedParams != nil && data.Task.TypedParams.VolumeSize > 0 { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 49, "
Volume Size:
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var11 string + templ_7745c5c3_Var11, templ_7745c5c3_Err = templ.JoinStringErrs(formatBytes(int64(data.Task.TypedParams.VolumeSize))) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 226, Col: 128} + } + _, 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, 50, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + if data.Task.TypedParams != nil && data.Task.TypedParams.Collection != "" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 51, "
Collection:
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var12 string + templ_7745c5c3_Var12, templ_7745c5c3_Err = templ.JoinStringErrs(data.Task.TypedParams.Collection) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 233, Col: 139} + } + _, 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, 52, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + if data.Task.TypedParams != nil && data.Task.TypedParams.DataCenter != "" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 53, "
Data Center:
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var13 string + templ_7745c5c3_Var13, templ_7745c5c3_Err = templ.JoinStringErrs(data.Task.TypedParams.DataCenter) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 240, Col: 146} + } + _, 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, 54, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + if data.Task.Progress > 0 { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 55, "
Progress:
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var16 string + templ_7745c5c3_Var16, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%.1f%%", data.Task.Progress)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 252, Col: 94} + } + _, 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, 58, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 59, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if data.Task.DetailedReason != "" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 60, "
Detailed Reason:

") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var17 string + templ_7745c5c3_Var17, templ_7745c5c3_Err = templ.JoinStringErrs(data.Task.DetailedReason) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 267, Col: 83} + } + _, 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, 61, "

") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + if data.Task.Error != "" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 62, "
Error:
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var18 string + templ_7745c5c3_Var18, templ_7745c5c3_Err = templ.JoinStringErrs(data.Task.Error) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 277, Col: 62} + } + _, 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, 63, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 64, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if data.Task.TypedParams != nil { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 65, "
Task Configuration
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if len(data.Task.TypedParams.Sources) > 0 { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 66, "
Source Servers ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var19 string + templ_7745c5c3_Var19, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", len(data.Task.TypedParams.Sources))) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 305, Col: 127} + } + _, 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, 67, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + for i, source := range data.Task.TypedParams.Sources { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 68, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var20 string + templ_7745c5c3_Var20, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("#%d", i+1)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 311, Col: 91} + } + _, 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, 69, " ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var21 string + templ_7745c5c3_Var21, templ_7745c5c3_Err = templ.JoinStringErrs(source.Node) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 312, Col: 54} + } + _, 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, 70, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if source.DataCenter != "" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 71, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var22 string + templ_7745c5c3_Var22, templ_7745c5c3_Err = templ.JoinStringErrs(source.DataCenter) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 316, Col: 102} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var22)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 72, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 73, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if source.Rack != "" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 74, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var23 string + templ_7745c5c3_Var23, templ_7745c5c3_Err = templ.JoinStringErrs(source.Rack) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 323, Col: 94} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var23)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 75, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 76, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if source.VolumeId > 0 { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 77, "Vol:") + 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", source.VolumeId)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 330, Col: 118} + } + _, 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, 78, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 79, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if len(source.ShardIds) > 0 { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 80, "Shards: ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + for j, shardId := range source.ShardIds { + if j > 0 { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 81, ", ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 82, " ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if shardId < erasure_coding.DataShardsCount { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 83, "") + 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", shardId)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 343, Col: 202} + } + _, 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, 85, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 86, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var28 string + templ_7745c5c3_Var28, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("P%d", shardId-erasure_coding.DataShardsCount)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 345, Col: 246} + } + _, 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, 88, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 89, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 90, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 91, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 92, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if len(data.Task.TypedParams.Sources) > 0 || len(data.Task.TypedParams.Targets) > 0 { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 93, "

Task: ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var29 string + templ_7745c5c3_Var29, templ_7745c5c3_Err = templ.JoinStringErrs(string(data.Task.Type)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 363, Col: 91} + } + _, 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, 94, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 95, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if len(data.Task.TypedParams.Targets) > 0 { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 96, "
Target Servers ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var30 string + templ_7745c5c3_Var30, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", len(data.Task.TypedParams.Targets))) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 373, Col: 130} + } + _, 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, 97, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + for i, target := range data.Task.TypedParams.Targets { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 98, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var31 string + templ_7745c5c3_Var31, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("#%d", i+1)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 379, Col: 91} + } + _, 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, 99, " ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var32 string + templ_7745c5c3_Var32, templ_7745c5c3_Err = templ.JoinStringErrs(target.Node) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 380, Col: 54} + } + _, 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, 100, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if target.DataCenter != "" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 101, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var33 string + templ_7745c5c3_Var33, templ_7745c5c3_Err = templ.JoinStringErrs(target.DataCenter) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 384, Col: 102} + } + _, 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, 102, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 103, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if target.Rack != "" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 104, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var34 string + templ_7745c5c3_Var34, templ_7745c5c3_Err = templ.JoinStringErrs(target.Rack) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 391, Col: 94} + } + _, 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, 105, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 106, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if target.VolumeId > 0 { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 107, "Vol:") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var35 string + templ_7745c5c3_Var35, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", target.VolumeId)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 398, Col: 118} + } + _, 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, 108, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 109, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if len(target.ShardIds) > 0 { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 110, "Shards: ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + for j, shardId := range target.ShardIds { + if j > 0 { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 111, ", ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 112, " ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if shardId < erasure_coding.DataShardsCount { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 113, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var37 string + templ_7745c5c3_Var37, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", shardId)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 411, Col: 202} + } + _, 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, 115, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 116, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var39 string + templ_7745c5c3_Var39, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("P%d", shardId-erasure_coding.DataShardsCount)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 413, Col: 246} + } + _, 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, 118, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 119, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 120, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 121, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 122, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 123, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if data.WorkerInfo != nil { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 124, "
Worker Information
Worker ID:
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var40 string + templ_7745c5c3_Var40, templ_7745c5c3_Err = templ.JoinStringErrs(data.WorkerInfo.ID) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 447, Col: 86} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var40)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 125, "
Address:
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var41 string + templ_7745c5c3_Var41, templ_7745c5c3_Err = templ.JoinStringErrs(data.WorkerInfo.Address) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 450, Col: 91} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var41)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 126, "
Status:
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if data.WorkerInfo.Status == "active" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 127, "Active") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else if data.WorkerInfo.Status == "busy" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 128, "Busy") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 129, "Inactive") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 130, "
Last Heartbeat:
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var42 string + templ_7745c5c3_Var42, templ_7745c5c3_Err = templ.JoinStringErrs(data.WorkerInfo.LastHeartbeat.Format("2006-01-02 15:04:05")) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 467, Col: 121} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var42)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 131, "
Current Load:
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var43 string + templ_7745c5c3_Var43, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d/%d", data.WorkerInfo.CurrentLoad, data.WorkerInfo.MaxConcurrent)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 470, Col: 142} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var43)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 132, "
Capabilities:
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + for _, capability := range data.WorkerInfo.Capabilities { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 133, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var44 string + templ_7745c5c3_Var44, templ_7745c5c3_Err = templ.JoinStringErrs(string(capability)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 475, Col: 100} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var44)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 134, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 135, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 136, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if len(data.AssignmentHistory) > 0 { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 137, "
Assignment History
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + for _, assignment := range data.AssignmentHistory { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 138, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 145, "
Worker IDWorker AddressAssigned AtUnassigned AtReason
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var45 string + templ_7745c5c3_Var45, templ_7745c5c3_Err = templ.JoinStringErrs(assignment.WorkerID) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 513, Col: 78} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var45)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 139, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var46 string + templ_7745c5c3_Var46, templ_7745c5c3_Err = templ.JoinStringErrs(assignment.WorkerAddress) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 514, Col: 83} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var46)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 140, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var47 string + templ_7745c5c3_Var47, templ_7745c5c3_Err = templ.JoinStringErrs(assignment.AssignedAt.Format("2006-01-02 15:04:05")) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 515, Col: 104} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var47)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 141, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if assignment.UnassignedAt != nil { + var templ_7745c5c3_Var48 string + templ_7745c5c3_Var48, templ_7745c5c3_Err = templ.JoinStringErrs(assignment.UnassignedAt.Format("2006-01-02 15:04:05")) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 518, Col: 110} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var48)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 142, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 143, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var49 string + templ_7745c5c3_Var49, templ_7745c5c3_Err = templ.JoinStringErrs(assignment.Reason) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 523, Col: 70} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var49)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 144, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 146, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if len(data.ExecutionLogs) > 0 { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 147, "
Execution Logs
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + for _, log := range data.ExecutionLogs { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 148, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 173, "
TimestampLevelMessageDetails
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var50 string + templ_7745c5c3_Var50, templ_7745c5c3_Err = templ.JoinStringErrs(log.Timestamp.Format("15:04:05")) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 560, Col: 92} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var50)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 149, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if log.Level == "error" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 150, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var51 string + templ_7745c5c3_Var51, templ_7745c5c3_Err = templ.JoinStringErrs(log.Level) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 563, Col: 96} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var51)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 151, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else if log.Level == "warn" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 152, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var52 string + templ_7745c5c3_Var52, templ_7745c5c3_Err = templ.JoinStringErrs(log.Level) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 565, Col: 97} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var52)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 153, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else if log.Level == "info" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 154, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var53 string + templ_7745c5c3_Var53, templ_7745c5c3_Err = templ.JoinStringErrs(log.Level) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 567, Col: 94} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var53)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 155, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 156, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var54 string + templ_7745c5c3_Var54, templ_7745c5c3_Err = templ.JoinStringErrs(log.Level) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 569, Col: 99} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var54)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 157, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 158, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var55 string + templ_7745c5c3_Var55, templ_7745c5c3_Err = templ.JoinStringErrs(log.Message) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 572, Col: 70} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var55)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 159, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if log.Fields != nil && len(log.Fields) > 0 { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 160, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + for _, k := range sortedKeys(log.Fields) { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 161, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var56 string + templ_7745c5c3_Var56, templ_7745c5c3_Err = templ.JoinStringErrs(k) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 577, Col: 110} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var56)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 162, "=") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var57 string + templ_7745c5c3_Var57, templ_7745c5c3_Err = templ.JoinStringErrs(log.Fields[k]) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 577, Col: 129} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var57)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 163, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 164, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else if log.Progress != nil || log.Status != "" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 165, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if log.Progress != nil { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 166, "progress=") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var58 string + templ_7745c5c3_Var58, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%.0f%%", *log.Progress)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 583, Col: 151} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var58)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 167, " ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + if log.Status != "" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 168, "status=") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var59 string + templ_7745c5c3_Var59, templ_7745c5c3_Err = templ.JoinStringErrs(log.Status) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 586, Col: 118} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var59)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 169, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 170, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 171, "-") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 172, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 174, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if len(data.RelatedTasks) > 0 { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 175, "
Related Tasks
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + for _, relatedTask := range data.RelatedTasks { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 176, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 193, "
Task IDTypeStatusVolume IDServerCreated
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var61 string + templ_7745c5c3_Var61, templ_7745c5c3_Err = templ.JoinStringErrs(relatedTask.ID) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 633, Col: 77} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var61)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 178, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var62 string + templ_7745c5c3_Var62, templ_7745c5c3_Err = templ.JoinStringErrs(string(relatedTask.Type)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 636, Col: 105} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var62)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 179, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if relatedTask.Status == maintenance.TaskStatusCompleted { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 180, "Completed") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else if relatedTask.Status == maintenance.TaskStatusFailed { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 181, "Failed") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else if relatedTask.Status == maintenance.TaskStatusInProgress { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 182, "In Progress") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 183, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var63 string + templ_7745c5c3_Var63, templ_7745c5c3_Err = templ.JoinStringErrs(string(relatedTask.Status)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 645, Col: 116} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var63)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 184, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 185, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if relatedTask.VolumeID != 0 { + var templ_7745c5c3_Var64 string + templ_7745c5c3_Var64, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", relatedTask.VolumeID)) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 650, Col: 96} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var64)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 186, "-") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 187, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if relatedTask.Server != "" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 188, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var65 string + templ_7745c5c3_Var65, templ_7745c5c3_Err = templ.JoinStringErrs(relatedTask.Server) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 657, Col: 81} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var65)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 189, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } else { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 190, "-") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 191, "") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + var templ_7745c5c3_Var66 string + templ_7745c5c3_Var66, templ_7745c5c3_Err = templ.JoinStringErrs(relatedTask.CreatedAt.Format("2006-01-02 15:04:05")) + if templ_7745c5c3_Err != nil { + return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 662, Col: 111} + } + _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var66)) + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 192, "
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 194, "
Actions
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + if data.Task.Status == maintenance.TaskStatusPending || data.Task.Status == maintenance.TaskStatusAssigned { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 195, " ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + if data.Task.WorkerID != "" { + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 197, " ") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + } + templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 200, "
Task Logs
Loading logs...

Fetching logs from worker...

Task: | Worker: | Entries:
Log Entries (Last 100) Newest entries first
") + if templ_7745c5c3_Err != nil { + return templ_7745c5c3_Err + } + return nil + }) +} + +var _ = templruntime.GeneratedTemplate diff --git a/weed/pb/worker.proto b/weed/pb/worker.proto index 811f94591..b9e3d61d0 100644 --- a/weed/pb/worker.proto +++ b/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 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 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; } \ No newline at end of file diff --git a/weed/pb/worker_pb/worker.pb.go b/weed/pb/worker_pb/worker.pb.go index ff7d60545..7ff5a8a36 100644 --- a/weed/pb/worker_pb/worker.pb.go +++ b/weed/pb/worker_pb/worker.pb.go @@ -804,14 +804,15 @@ func (x *TaskAssignment) GetMetadata() map[string]string { // TaskParams contains task-specific parameters with typed variants type TaskParams struct { state protoimpl.MessageState `protogen:"open.v1"` - TaskId string `protobuf:"bytes,12,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"` // ActiveTopology task ID for lifecycle management - VolumeId uint32 `protobuf:"varint,1,opt,name=volume_id,json=volumeId,proto3" json:"volume_id,omitempty"` - Server string `protobuf:"bytes,2,opt,name=server,proto3" json:"server,omitempty"` - Collection string `protobuf:"bytes,3,opt,name=collection,proto3" json:"collection,omitempty"` - DataCenter string `protobuf:"bytes,4,opt,name=data_center,json=dataCenter,proto3" json:"data_center,omitempty"` - Rack string `protobuf:"bytes,5,opt,name=rack,proto3" json:"rack,omitempty"` - Replicas []string `protobuf:"bytes,6,rep,name=replicas,proto3" json:"replicas,omitempty"` - VolumeSize uint64 `protobuf:"varint,11,opt,name=volume_size,json=volumeSize,proto3" json:"volume_size,omitempty"` // Original volume size in bytes for tracking size changes + TaskId string `protobuf:"bytes,1,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"` // ActiveTopology task ID for lifecycle management + VolumeId uint32 `protobuf:"varint,2,opt,name=volume_id,json=volumeId,proto3" json:"volume_id,omitempty"` // Primary volume ID for the task + Collection string `protobuf:"bytes,3,opt,name=collection,proto3" json:"collection,omitempty"` // Collection name + DataCenter string `protobuf:"bytes,4,opt,name=data_center,json=dataCenter,proto3" json:"data_center,omitempty"` // Primary data center + Rack string `protobuf:"bytes,5,opt,name=rack,proto3" json:"rack,omitempty"` // Primary rack + VolumeSize uint64 `protobuf:"varint,6,opt,name=volume_size,json=volumeSize,proto3" json:"volume_size,omitempty"` // Original volume size in bytes for tracking size changes + // Unified source and target arrays for all task types + Sources []*TaskSource `protobuf:"bytes,7,rep,name=sources,proto3" json:"sources,omitempty"` // Source locations (volume replicas, EC shards, etc.) + Targets []*TaskTarget `protobuf:"bytes,8,rep,name=targets,proto3" json:"targets,omitempty"` // Target locations (destinations, new replicas, etc.) // Typed task parameters // // Types that are valid to be assigned to TaskParams: @@ -869,13 +870,6 @@ func (x *TaskParams) GetVolumeId() uint32 { return 0 } -func (x *TaskParams) GetServer() string { - if x != nil { - return x.Server - } - return "" -} - func (x *TaskParams) GetCollection() string { if x != nil { return x.Collection @@ -897,18 +891,25 @@ func (x *TaskParams) GetRack() string { return "" } -func (x *TaskParams) GetReplicas() []string { +func (x *TaskParams) GetVolumeSize() uint64 { + if x != nil { + return x.VolumeSize + } + return 0 +} + +func (x *TaskParams) GetSources() []*TaskSource { if x != nil { - return x.Replicas + return x.Sources } return nil } -func (x *TaskParams) GetVolumeSize() uint64 { +func (x *TaskParams) GetTargets() []*TaskTarget { if x != nil { - return x.VolumeSize + return x.Targets } - return 0 + return nil } func (x *TaskParams) GetTaskParams() isTaskParams_TaskParams { @@ -959,19 +960,19 @@ type isTaskParams_TaskParams interface { } type TaskParams_VacuumParams struct { - VacuumParams *VacuumTaskParams `protobuf:"bytes,7,opt,name=vacuum_params,json=vacuumParams,proto3,oneof"` + VacuumParams *VacuumTaskParams `protobuf:"bytes,9,opt,name=vacuum_params,json=vacuumParams,proto3,oneof"` } type TaskParams_ErasureCodingParams struct { - ErasureCodingParams *ErasureCodingTaskParams `protobuf:"bytes,8,opt,name=erasure_coding_params,json=erasureCodingParams,proto3,oneof"` + ErasureCodingParams *ErasureCodingTaskParams `protobuf:"bytes,10,opt,name=erasure_coding_params,json=erasureCodingParams,proto3,oneof"` } type TaskParams_BalanceParams struct { - BalanceParams *BalanceTaskParams `protobuf:"bytes,9,opt,name=balance_params,json=balanceParams,proto3,oneof"` + BalanceParams *BalanceTaskParams `protobuf:"bytes,11,opt,name=balance_params,json=balanceParams,proto3,oneof"` } type TaskParams_ReplicationParams struct { - ReplicationParams *ReplicationTaskParams `protobuf:"bytes,10,opt,name=replication_params,json=replicationParams,proto3,oneof"` + ReplicationParams *ReplicationTaskParams `protobuf:"bytes,12,opt,name=replication_params,json=replicationParams,proto3,oneof"` } func (*TaskParams_VacuumParams) isTaskParams_TaskParams() {} @@ -1061,18 +1062,15 @@ func (x *VacuumTaskParams) GetVerifyChecksum() bool { // ErasureCodingTaskParams for EC encoding operations type ErasureCodingTaskParams struct { - state protoimpl.MessageState `protogen:"open.v1"` - EstimatedShardSize uint64 `protobuf:"varint,3,opt,name=estimated_shard_size,json=estimatedShardSize,proto3" json:"estimated_shard_size,omitempty"` // Estimated size per shard - DataShards int32 `protobuf:"varint,4,opt,name=data_shards,json=dataShards,proto3" json:"data_shards,omitempty"` // Number of data shards (default: 10) - ParityShards int32 `protobuf:"varint,5,opt,name=parity_shards,json=parityShards,proto3" json:"parity_shards,omitempty"` // Number of parity shards (default: 4) - WorkingDir string `protobuf:"bytes,6,opt,name=working_dir,json=workingDir,proto3" json:"working_dir,omitempty"` // Working directory for EC processing - MasterClient string `protobuf:"bytes,7,opt,name=master_client,json=masterClient,proto3" json:"master_client,omitempty"` // Master server address - CleanupSource bool `protobuf:"varint,8,opt,name=cleanup_source,json=cleanupSource,proto3" json:"cleanup_source,omitempty"` // Whether to cleanup source volume after EC - PlacementConflicts []string `protobuf:"bytes,9,rep,name=placement_conflicts,json=placementConflicts,proto3" json:"placement_conflicts,omitempty"` // Any placement rule conflicts - Destinations []*ECDestination `protobuf:"bytes,10,rep,name=destinations,proto3" json:"destinations,omitempty"` // Planned destinations with disk information - ExistingShardLocations []*ExistingECShardLocation `protobuf:"bytes,11,rep,name=existing_shard_locations,json=existingShardLocations,proto3" json:"existing_shard_locations,omitempty"` // Existing EC shards to cleanup - unknownFields protoimpl.UnknownFields - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"open.v1"` + EstimatedShardSize uint64 `protobuf:"varint,1,opt,name=estimated_shard_size,json=estimatedShardSize,proto3" json:"estimated_shard_size,omitempty"` // Estimated size per shard + DataShards int32 `protobuf:"varint,2,opt,name=data_shards,json=dataShards,proto3" json:"data_shards,omitempty"` // Number of data shards (default: 10) + ParityShards int32 `protobuf:"varint,3,opt,name=parity_shards,json=parityShards,proto3" json:"parity_shards,omitempty"` // Number of parity shards (default: 4) + WorkingDir string `protobuf:"bytes,4,opt,name=working_dir,json=workingDir,proto3" json:"working_dir,omitempty"` // Working directory for EC processing + MasterClient string `protobuf:"bytes,5,opt,name=master_client,json=masterClient,proto3" json:"master_client,omitempty"` // Master server address + CleanupSource bool `protobuf:"varint,6,opt,name=cleanup_source,json=cleanupSource,proto3" json:"cleanup_source,omitempty"` // Whether to cleanup source volume after EC + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ErasureCodingTaskParams) Reset() { @@ -1147,53 +1145,34 @@ func (x *ErasureCodingTaskParams) GetCleanupSource() bool { return false } -func (x *ErasureCodingTaskParams) GetPlacementConflicts() []string { - if x != nil { - return x.PlacementConflicts - } - return nil -} - -func (x *ErasureCodingTaskParams) GetDestinations() []*ECDestination { - if x != nil { - return x.Destinations - } - return nil -} - -func (x *ErasureCodingTaskParams) GetExistingShardLocations() []*ExistingECShardLocation { - if x != nil { - return x.ExistingShardLocations - } - return nil -} - -// ECDestination represents a planned destination for EC shards with disk information -type ECDestination struct { - state protoimpl.MessageState `protogen:"open.v1"` - Node string `protobuf:"bytes,1,opt,name=node,proto3" json:"node,omitempty"` // Target server address - DiskId uint32 `protobuf:"varint,2,opt,name=disk_id,json=diskId,proto3" json:"disk_id,omitempty"` // Target disk ID - Rack string `protobuf:"bytes,3,opt,name=rack,proto3" json:"rack,omitempty"` // Target rack for placement tracking - DataCenter string `protobuf:"bytes,4,opt,name=data_center,json=dataCenter,proto3" json:"data_center,omitempty"` // Target data center for placement tracking - PlacementScore float64 `protobuf:"fixed64,5,opt,name=placement_score,json=placementScore,proto3" json:"placement_score,omitempty"` // Quality score of the placement - unknownFields protoimpl.UnknownFields - sizeCache protoimpl.SizeCache +// TaskSource represents a unified source location for any task type +type TaskSource struct { + state protoimpl.MessageState `protogen:"open.v1"` + Node string `protobuf:"bytes,1,opt,name=node,proto3" json:"node,omitempty"` // Source server address + DiskId uint32 `protobuf:"varint,2,opt,name=disk_id,json=diskId,proto3" json:"disk_id,omitempty"` // Source disk ID + Rack string `protobuf:"bytes,3,opt,name=rack,proto3" json:"rack,omitempty"` // Source rack for tracking + DataCenter string `protobuf:"bytes,4,opt,name=data_center,json=dataCenter,proto3" json:"data_center,omitempty"` // Source data center for tracking + VolumeId uint32 `protobuf:"varint,5,opt,name=volume_id,json=volumeId,proto3" json:"volume_id,omitempty"` // Volume ID (for volume operations) + ShardIds []uint32 `protobuf:"varint,6,rep,packed,name=shard_ids,json=shardIds,proto3" json:"shard_ids,omitempty"` // Shard IDs (for EC shard operations) + EstimatedSize uint64 `protobuf:"varint,7,opt,name=estimated_size,json=estimatedSize,proto3" json:"estimated_size,omitempty"` // Estimated size to be processed + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } -func (x *ECDestination) Reset() { - *x = ECDestination{} +func (x *TaskSource) Reset() { + *x = TaskSource{} mi := &file_worker_proto_msgTypes[11] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ECDestination) String() string { +func (x *TaskSource) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ECDestination) ProtoMessage() {} +func (*TaskSource) ProtoMessage() {} -func (x *ECDestination) ProtoReflect() protoreflect.Message { +func (x *TaskSource) ProtoReflect() protoreflect.Message { mi := &file_worker_proto_msgTypes[11] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -1205,69 +1184,88 @@ func (x *ECDestination) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ECDestination.ProtoReflect.Descriptor instead. -func (*ECDestination) Descriptor() ([]byte, []int) { +// Deprecated: Use TaskSource.ProtoReflect.Descriptor instead. +func (*TaskSource) Descriptor() ([]byte, []int) { return file_worker_proto_rawDescGZIP(), []int{11} } -func (x *ECDestination) GetNode() string { +func (x *TaskSource) GetNode() string { if x != nil { return x.Node } return "" } -func (x *ECDestination) GetDiskId() uint32 { +func (x *TaskSource) GetDiskId() uint32 { if x != nil { return x.DiskId } return 0 } -func (x *ECDestination) GetRack() string { +func (x *TaskSource) GetRack() string { if x != nil { return x.Rack } return "" } -func (x *ECDestination) GetDataCenter() string { +func (x *TaskSource) GetDataCenter() string { if x != nil { return x.DataCenter } return "" } -func (x *ECDestination) GetPlacementScore() float64 { +func (x *TaskSource) GetVolumeId() uint32 { + if x != nil { + return x.VolumeId + } + return 0 +} + +func (x *TaskSource) GetShardIds() []uint32 { + if x != nil { + return x.ShardIds + } + return nil +} + +func (x *TaskSource) GetEstimatedSize() uint64 { if x != nil { - return x.PlacementScore + return x.EstimatedSize } return 0 } -// ExistingECShardLocation represents existing EC shards that need cleanup -type ExistingECShardLocation struct { +// TaskTarget represents a unified target location for any task type +type TaskTarget struct { state protoimpl.MessageState `protogen:"open.v1"` - Node string `protobuf:"bytes,1,opt,name=node,proto3" json:"node,omitempty"` // Server address with existing shards - ShardIds []uint32 `protobuf:"varint,2,rep,packed,name=shard_ids,json=shardIds,proto3" json:"shard_ids,omitempty"` // List of shard IDs on this server + Node string `protobuf:"bytes,1,opt,name=node,proto3" json:"node,omitempty"` // Target server address + DiskId uint32 `protobuf:"varint,2,opt,name=disk_id,json=diskId,proto3" json:"disk_id,omitempty"` // Target disk ID + Rack string `protobuf:"bytes,3,opt,name=rack,proto3" json:"rack,omitempty"` // Target rack for tracking + DataCenter string `protobuf:"bytes,4,opt,name=data_center,json=dataCenter,proto3" json:"data_center,omitempty"` // Target data center for tracking + VolumeId uint32 `protobuf:"varint,5,opt,name=volume_id,json=volumeId,proto3" json:"volume_id,omitempty"` // Volume ID (for volume operations) + ShardIds []uint32 `protobuf:"varint,6,rep,packed,name=shard_ids,json=shardIds,proto3" json:"shard_ids,omitempty"` // Shard IDs (for EC shard operations) + EstimatedSize uint64 `protobuf:"varint,7,opt,name=estimated_size,json=estimatedSize,proto3" json:"estimated_size,omitempty"` // Estimated size to be created unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } -func (x *ExistingECShardLocation) Reset() { - *x = ExistingECShardLocation{} +func (x *TaskTarget) Reset() { + *x = TaskTarget{} mi := &file_worker_proto_msgTypes[12] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } -func (x *ExistingECShardLocation) String() string { +func (x *TaskTarget) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ExistingECShardLocation) ProtoMessage() {} +func (*TaskTarget) ProtoMessage() {} -func (x *ExistingECShardLocation) ProtoReflect() protoreflect.Message { +func (x *TaskTarget) ProtoReflect() protoreflect.Message { mi := &file_worker_proto_msgTypes[12] if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -1279,38 +1277,67 @@ func (x *ExistingECShardLocation) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ExistingECShardLocation.ProtoReflect.Descriptor instead. -func (*ExistingECShardLocation) Descriptor() ([]byte, []int) { +// Deprecated: Use TaskTarget.ProtoReflect.Descriptor instead. +func (*TaskTarget) Descriptor() ([]byte, []int) { return file_worker_proto_rawDescGZIP(), []int{12} } -func (x *ExistingECShardLocation) GetNode() string { +func (x *TaskTarget) GetNode() string { if x != nil { return x.Node } return "" } -func (x *ExistingECShardLocation) GetShardIds() []uint32 { +func (x *TaskTarget) GetDiskId() uint32 { + if x != nil { + return x.DiskId + } + return 0 +} + +func (x *TaskTarget) GetRack() string { + if x != nil { + return x.Rack + } + return "" +} + +func (x *TaskTarget) GetDataCenter() string { + if x != nil { + return x.DataCenter + } + return "" +} + +func (x *TaskTarget) GetVolumeId() uint32 { + if x != nil { + return x.VolumeId + } + return 0 +} + +func (x *TaskTarget) GetShardIds() []uint32 { if x != nil { return x.ShardIds } return nil } +func (x *TaskTarget) GetEstimatedSize() uint64 { + if x != nil { + return x.EstimatedSize + } + return 0 +} + // BalanceTaskParams for volume balancing operations type BalanceTaskParams struct { - state protoimpl.MessageState `protogen:"open.v1"` - DestNode string `protobuf:"bytes,1,opt,name=dest_node,json=destNode,proto3" json:"dest_node,omitempty"` // Planned destination node - EstimatedSize uint64 `protobuf:"varint,2,opt,name=estimated_size,json=estimatedSize,proto3" json:"estimated_size,omitempty"` // Estimated volume size - DestRack string `protobuf:"bytes,3,opt,name=dest_rack,json=destRack,proto3" json:"dest_rack,omitempty"` // Destination rack for placement rules - DestDc string `protobuf:"bytes,4,opt,name=dest_dc,json=destDc,proto3" json:"dest_dc,omitempty"` // Destination data center - PlacementScore float64 `protobuf:"fixed64,5,opt,name=placement_score,json=placementScore,proto3" json:"placement_score,omitempty"` // Quality score of the planned placement - PlacementConflicts []string `protobuf:"bytes,6,rep,name=placement_conflicts,json=placementConflicts,proto3" json:"placement_conflicts,omitempty"` // Any placement rule conflicts - ForceMove bool `protobuf:"varint,7,opt,name=force_move,json=forceMove,proto3" json:"force_move,omitempty"` // Force move even with conflicts - TimeoutSeconds int32 `protobuf:"varint,8,opt,name=timeout_seconds,json=timeoutSeconds,proto3" json:"timeout_seconds,omitempty"` // Operation timeout - unknownFields protoimpl.UnknownFields - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"open.v1"` + ForceMove bool `protobuf:"varint,1,opt,name=force_move,json=forceMove,proto3" json:"force_move,omitempty"` // Force move even with conflicts + TimeoutSeconds int32 `protobuf:"varint,2,opt,name=timeout_seconds,json=timeoutSeconds,proto3" json:"timeout_seconds,omitempty"` // Operation timeout + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *BalanceTaskParams) Reset() { @@ -1343,48 +1370,6 @@ func (*BalanceTaskParams) Descriptor() ([]byte, []int) { return file_worker_proto_rawDescGZIP(), []int{13} } -func (x *BalanceTaskParams) GetDestNode() string { - if x != nil { - return x.DestNode - } - return "" -} - -func (x *BalanceTaskParams) GetEstimatedSize() uint64 { - if x != nil { - return x.EstimatedSize - } - return 0 -} - -func (x *BalanceTaskParams) GetDestRack() string { - if x != nil { - return x.DestRack - } - return "" -} - -func (x *BalanceTaskParams) GetDestDc() string { - if x != nil { - return x.DestDc - } - return "" -} - -func (x *BalanceTaskParams) GetPlacementScore() float64 { - if x != nil { - return x.PlacementScore - } - return 0 -} - -func (x *BalanceTaskParams) GetPlacementConflicts() []string { - if x != nil { - return x.PlacementConflicts - } - return nil -} - func (x *BalanceTaskParams) GetForceMove() bool { if x != nil { return x.ForceMove @@ -1401,17 +1386,11 @@ func (x *BalanceTaskParams) GetTimeoutSeconds() int32 { // ReplicationTaskParams for adding replicas type ReplicationTaskParams struct { - state protoimpl.MessageState `protogen:"open.v1"` - DestNode string `protobuf:"bytes,1,opt,name=dest_node,json=destNode,proto3" json:"dest_node,omitempty"` // Planned destination node for new replica - EstimatedSize uint64 `protobuf:"varint,2,opt,name=estimated_size,json=estimatedSize,proto3" json:"estimated_size,omitempty"` // Estimated replica size - DestRack string `protobuf:"bytes,3,opt,name=dest_rack,json=destRack,proto3" json:"dest_rack,omitempty"` // Destination rack for placement rules - DestDc string `protobuf:"bytes,4,opt,name=dest_dc,json=destDc,proto3" json:"dest_dc,omitempty"` // Destination data center - PlacementScore float64 `protobuf:"fixed64,5,opt,name=placement_score,json=placementScore,proto3" json:"placement_score,omitempty"` // Quality score of the planned placement - PlacementConflicts []string `protobuf:"bytes,6,rep,name=placement_conflicts,json=placementConflicts,proto3" json:"placement_conflicts,omitempty"` // Any placement rule conflicts - ReplicaCount int32 `protobuf:"varint,7,opt,name=replica_count,json=replicaCount,proto3" json:"replica_count,omitempty"` // Target replica count - VerifyConsistency bool `protobuf:"varint,8,opt,name=verify_consistency,json=verifyConsistency,proto3" json:"verify_consistency,omitempty"` // Verify replica consistency after creation - unknownFields protoimpl.UnknownFields - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"open.v1"` + ReplicaCount int32 `protobuf:"varint,1,opt,name=replica_count,json=replicaCount,proto3" json:"replica_count,omitempty"` // Target replica count + VerifyConsistency bool `protobuf:"varint,2,opt,name=verify_consistency,json=verifyConsistency,proto3" json:"verify_consistency,omitempty"` // Verify replica consistency after creation + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ReplicationTaskParams) Reset() { @@ -1444,48 +1423,6 @@ func (*ReplicationTaskParams) Descriptor() ([]byte, []int) { return file_worker_proto_rawDescGZIP(), []int{14} } -func (x *ReplicationTaskParams) GetDestNode() string { - if x != nil { - return x.DestNode - } - return "" -} - -func (x *ReplicationTaskParams) GetEstimatedSize() uint64 { - if x != nil { - return x.EstimatedSize - } - return 0 -} - -func (x *ReplicationTaskParams) GetDestRack() string { - if x != nil { - return x.DestRack - } - return "" -} - -func (x *ReplicationTaskParams) GetDestDc() string { - if x != nil { - return x.DestDc - } - return "" -} - -func (x *ReplicationTaskParams) GetPlacementScore() float64 { - if x != nil { - return x.PlacementScore - } - return 0 -} - -func (x *ReplicationTaskParams) GetPlacementConflicts() []string { - if x != nil { - return x.PlacementConflicts - } - return nil -} - func (x *ReplicationTaskParams) GetReplicaCount() int32 { if x != nil { return x.ReplicaCount @@ -2812,151 +2749,707 @@ func (x *ReplicationTaskConfig) GetTargetReplicaCount() int32 { return 0 } -var File_worker_proto protoreflect.FileDescriptor +// MaintenanceTaskData represents complete task state for persistence +type MaintenanceTaskData struct { + state protoimpl.MessageState `protogen:"open.v1"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Type string `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"` + Priority string `protobuf:"bytes,3,opt,name=priority,proto3" json:"priority,omitempty"` + Status string `protobuf:"bytes,4,opt,name=status,proto3" json:"status,omitempty"` + VolumeId uint32 `protobuf:"varint,5,opt,name=volume_id,json=volumeId,proto3" json:"volume_id,omitempty"` + Server string `protobuf:"bytes,6,opt,name=server,proto3" json:"server,omitempty"` + Collection string `protobuf:"bytes,7,opt,name=collection,proto3" json:"collection,omitempty"` + TypedParams *TaskParams `protobuf:"bytes,8,opt,name=typed_params,json=typedParams,proto3" json:"typed_params,omitempty"` + Reason string `protobuf:"bytes,9,opt,name=reason,proto3" json:"reason,omitempty"` + CreatedAt int64 `protobuf:"varint,10,opt,name=created_at,json=createdAt,proto3" json:"created_at,omitempty"` + ScheduledAt int64 `protobuf:"varint,11,opt,name=scheduled_at,json=scheduledAt,proto3" json:"scheduled_at,omitempty"` + StartedAt int64 `protobuf:"varint,12,opt,name=started_at,json=startedAt,proto3" json:"started_at,omitempty"` + CompletedAt int64 `protobuf:"varint,13,opt,name=completed_at,json=completedAt,proto3" json:"completed_at,omitempty"` + WorkerId string `protobuf:"bytes,14,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"` + Error string `protobuf:"bytes,15,opt,name=error,proto3" json:"error,omitempty"` + Progress float64 `protobuf:"fixed64,16,opt,name=progress,proto3" json:"progress,omitempty"` + RetryCount int32 `protobuf:"varint,17,opt,name=retry_count,json=retryCount,proto3" json:"retry_count,omitempty"` + MaxRetries int32 `protobuf:"varint,18,opt,name=max_retries,json=maxRetries,proto3" json:"max_retries,omitempty"` + // Enhanced fields for detailed task tracking + CreatedBy string `protobuf:"bytes,19,opt,name=created_by,json=createdBy,proto3" json:"created_by,omitempty"` + CreationContext string `protobuf:"bytes,20,opt,name=creation_context,json=creationContext,proto3" json:"creation_context,omitempty"` + AssignmentHistory []*TaskAssignmentRecord `protobuf:"bytes,21,rep,name=assignment_history,json=assignmentHistory,proto3" json:"assignment_history,omitempty"` + DetailedReason string `protobuf:"bytes,22,opt,name=detailed_reason,json=detailedReason,proto3" json:"detailed_reason,omitempty"` + Tags map[string]string `protobuf:"bytes,23,rep,name=tags,proto3" json:"tags,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + CreationMetrics *TaskCreationMetrics `protobuf:"bytes,24,opt,name=creation_metrics,json=creationMetrics,proto3" json:"creation_metrics,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MaintenanceTaskData) Reset() { + *x = MaintenanceTaskData{} + mi := &file_worker_proto_msgTypes[31] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} -const file_worker_proto_rawDesc = "" + - "\n" + - "\fworker.proto\x12\tworker_pb\"\x90\x04\n" + - "\rWorkerMessage\x12\x1b\n" + - "\tworker_id\x18\x01 \x01(\tR\bworkerId\x12\x1c\n" + - "\ttimestamp\x18\x02 \x01(\x03R\ttimestamp\x12C\n" + - "\fregistration\x18\x03 \x01(\v2\x1d.worker_pb.WorkerRegistrationH\x00R\fregistration\x12:\n" + - "\theartbeat\x18\x04 \x01(\v2\x1a.worker_pb.WorkerHeartbeatH\x00R\theartbeat\x12;\n" + - "\ftask_request\x18\x05 \x01(\v2\x16.worker_pb.TaskRequestH\x00R\vtaskRequest\x128\n" + - "\vtask_update\x18\x06 \x01(\v2\x15.worker_pb.TaskUpdateH\x00R\n" + - "taskUpdate\x12>\n" + - "\rtask_complete\x18\a \x01(\v2\x17.worker_pb.TaskCompleteH\x00R\ftaskComplete\x127\n" + - "\bshutdown\x18\b \x01(\v2\x19.worker_pb.WorkerShutdownH\x00R\bshutdown\x12H\n" + - "\x11task_log_response\x18\t \x01(\v2\x1a.worker_pb.TaskLogResponseH\x00R\x0ftaskLogResponseB\t\n" + - "\amessage\"\x95\x04\n" + - "\fAdminMessage\x12\x19\n" + - "\badmin_id\x18\x01 \x01(\tR\aadminId\x12\x1c\n" + - "\ttimestamp\x18\x02 \x01(\x03R\ttimestamp\x12V\n" + - "\x15registration_response\x18\x03 \x01(\v2\x1f.worker_pb.RegistrationResponseH\x00R\x14registrationResponse\x12M\n" + - "\x12heartbeat_response\x18\x04 \x01(\v2\x1c.worker_pb.HeartbeatResponseH\x00R\x11heartbeatResponse\x12D\n" + - "\x0ftask_assignment\x18\x05 \x01(\v2\x19.worker_pb.TaskAssignmentH\x00R\x0etaskAssignment\x12J\n" + - "\x11task_cancellation\x18\x06 \x01(\v2\x1b.worker_pb.TaskCancellationH\x00R\x10taskCancellation\x12A\n" + - "\x0eadmin_shutdown\x18\a \x01(\v2\x18.worker_pb.AdminShutdownH\x00R\radminShutdown\x12E\n" + - "\x10task_log_request\x18\b \x01(\v2\x19.worker_pb.TaskLogRequestH\x00R\x0etaskLogRequestB\t\n" + - "\amessage\"\x9c\x02\n" + - "\x12WorkerRegistration\x12\x1b\n" + - "\tworker_id\x18\x01 \x01(\tR\bworkerId\x12\x18\n" + - "\aaddress\x18\x02 \x01(\tR\aaddress\x12\"\n" + - "\fcapabilities\x18\x03 \x03(\tR\fcapabilities\x12%\n" + - "\x0emax_concurrent\x18\x04 \x01(\x05R\rmaxConcurrent\x12G\n" + - "\bmetadata\x18\x05 \x03(\v2+.worker_pb.WorkerRegistration.MetadataEntryR\bmetadata\x1a;\n" + - "\rMetadataEntry\x12\x10\n" + - "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + - "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"x\n" + - "\x14RegistrationResponse\x12\x18\n" + - "\asuccess\x18\x01 \x01(\bR\asuccess\x12\x18\n" + - "\amessage\x18\x02 \x01(\tR\amessage\x12,\n" + - "\x12assigned_worker_id\x18\x03 \x01(\tR\x10assignedWorkerId\"\xad\x02\n" + - "\x0fWorkerHeartbeat\x12\x1b\n" + - "\tworker_id\x18\x01 \x01(\tR\bworkerId\x12\x16\n" + - "\x06status\x18\x02 \x01(\tR\x06status\x12!\n" + - "\fcurrent_load\x18\x03 \x01(\x05R\vcurrentLoad\x12%\n" + - "\x0emax_concurrent\x18\x04 \x01(\x05R\rmaxConcurrent\x12(\n" + - "\x10current_task_ids\x18\x05 \x03(\tR\x0ecurrentTaskIds\x12'\n" + - "\x0ftasks_completed\x18\x06 \x01(\x05R\x0etasksCompleted\x12!\n" + - "\ftasks_failed\x18\a \x01(\x05R\vtasksFailed\x12%\n" + - "\x0euptime_seconds\x18\b \x01(\x03R\ruptimeSeconds\"G\n" + - "\x11HeartbeatResponse\x12\x18\n" + - "\asuccess\x18\x01 \x01(\bR\asuccess\x12\x18\n" + - "\amessage\x18\x02 \x01(\tR\amessage\"w\n" + - "\vTaskRequest\x12\x1b\n" + - "\tworker_id\x18\x01 \x01(\tR\bworkerId\x12\"\n" + - "\fcapabilities\x18\x02 \x03(\tR\fcapabilities\x12'\n" + - "\x0favailable_slots\x18\x03 \x01(\x05R\x0eavailableSlots\"\xb6\x02\n" + - "\x0eTaskAssignment\x12\x17\n" + - "\atask_id\x18\x01 \x01(\tR\x06taskId\x12\x1b\n" + - "\ttask_type\x18\x02 \x01(\tR\btaskType\x12-\n" + - "\x06params\x18\x03 \x01(\v2\x15.worker_pb.TaskParamsR\x06params\x12\x1a\n" + - "\bpriority\x18\x04 \x01(\x05R\bpriority\x12!\n" + - "\fcreated_time\x18\x05 \x01(\x03R\vcreatedTime\x12C\n" + - "\bmetadata\x18\x06 \x03(\v2'.worker_pb.TaskAssignment.MetadataEntryR\bmetadata\x1a;\n" + - "\rMetadataEntry\x12\x10\n" + - "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + - "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"\xb3\x04\n" + - "\n" + - "TaskParams\x12\x17\n" + - "\atask_id\x18\f \x01(\tR\x06taskId\x12\x1b\n" + - "\tvolume_id\x18\x01 \x01(\rR\bvolumeId\x12\x16\n" + - "\x06server\x18\x02 \x01(\tR\x06server\x12\x1e\n" + - "\n" + - "collection\x18\x03 \x01(\tR\n" + - "collection\x12\x1f\n" + - "\vdata_center\x18\x04 \x01(\tR\n" + - "dataCenter\x12\x12\n" + - "\x04rack\x18\x05 \x01(\tR\x04rack\x12\x1a\n" + - "\breplicas\x18\x06 \x03(\tR\breplicas\x12\x1f\n" + - "\vvolume_size\x18\v \x01(\x04R\n" + - "volumeSize\x12B\n" + - "\rvacuum_params\x18\a \x01(\v2\x1b.worker_pb.VacuumTaskParamsH\x00R\fvacuumParams\x12X\n" + - "\x15erasure_coding_params\x18\b \x01(\v2\".worker_pb.ErasureCodingTaskParamsH\x00R\x13erasureCodingParams\x12E\n" + - "\x0ebalance_params\x18\t \x01(\v2\x1c.worker_pb.BalanceTaskParamsH\x00R\rbalanceParams\x12Q\n" + - "\x12replication_params\x18\n" + - " \x01(\v2 .worker_pb.ReplicationTaskParamsH\x00R\x11replicationParamsB\r\n" + - "\vtask_params\"\xcb\x01\n" + - "\x10VacuumTaskParams\x12+\n" + - "\x11garbage_threshold\x18\x01 \x01(\x01R\x10garbageThreshold\x12!\n" + - "\fforce_vacuum\x18\x02 \x01(\bR\vforceVacuum\x12\x1d\n" + - "\n" + - "batch_size\x18\x03 \x01(\x05R\tbatchSize\x12\x1f\n" + - "\vworking_dir\x18\x04 \x01(\tR\n" + - "workingDir\x12'\n" + - "\x0fverify_checksum\x18\x05 \x01(\bR\x0everifyChecksum\"\xcb\x03\n" + - "\x17ErasureCodingTaskParams\x120\n" + - "\x14estimated_shard_size\x18\x03 \x01(\x04R\x12estimatedShardSize\x12\x1f\n" + - "\vdata_shards\x18\x04 \x01(\x05R\n" + - "dataShards\x12#\n" + - "\rparity_shards\x18\x05 \x01(\x05R\fparityShards\x12\x1f\n" + - "\vworking_dir\x18\x06 \x01(\tR\n" + - "workingDir\x12#\n" + - "\rmaster_client\x18\a \x01(\tR\fmasterClient\x12%\n" + - "\x0ecleanup_source\x18\b \x01(\bR\rcleanupSource\x12/\n" + - "\x13placement_conflicts\x18\t \x03(\tR\x12placementConflicts\x12<\n" + - "\fdestinations\x18\n" + - " \x03(\v2\x18.worker_pb.ECDestinationR\fdestinations\x12\\\n" + - "\x18existing_shard_locations\x18\v \x03(\v2\".worker_pb.ExistingECShardLocationR\x16existingShardLocations\"\x9a\x01\n" + - "\rECDestination\x12\x12\n" + - "\x04node\x18\x01 \x01(\tR\x04node\x12\x17\n" + - "\adisk_id\x18\x02 \x01(\rR\x06diskId\x12\x12\n" + - "\x04rack\x18\x03 \x01(\tR\x04rack\x12\x1f\n" + - "\vdata_center\x18\x04 \x01(\tR\n" + - "dataCenter\x12'\n" + - "\x0fplacement_score\x18\x05 \x01(\x01R\x0eplacementScore\"J\n" + - "\x17ExistingECShardLocation\x12\x12\n" + - "\x04node\x18\x01 \x01(\tR\x04node\x12\x1b\n" + - "\tshard_ids\x18\x02 \x03(\rR\bshardIds\"\xaf\x02\n" + - "\x11BalanceTaskParams\x12\x1b\n" + - "\tdest_node\x18\x01 \x01(\tR\bdestNode\x12%\n" + - "\x0eestimated_size\x18\x02 \x01(\x04R\restimatedSize\x12\x1b\n" + - "\tdest_rack\x18\x03 \x01(\tR\bdestRack\x12\x17\n" + - "\adest_dc\x18\x04 \x01(\tR\x06destDc\x12'\n" + - "\x0fplacement_score\x18\x05 \x01(\x01R\x0eplacementScore\x12/\n" + - "\x13placement_conflicts\x18\x06 \x03(\tR\x12placementConflicts\x12\x1d\n" + - "\n" + - "force_move\x18\a \x01(\bR\tforceMove\x12'\n" + - "\x0ftimeout_seconds\x18\b \x01(\x05R\x0etimeoutSeconds\"\xbf\x02\n" + - "\x15ReplicationTaskParams\x12\x1b\n" + - "\tdest_node\x18\x01 \x01(\tR\bdestNode\x12%\n" + - "\x0eestimated_size\x18\x02 \x01(\x04R\restimatedSize\x12\x1b\n" + - "\tdest_rack\x18\x03 \x01(\tR\bdestRack\x12\x17\n" + - "\adest_dc\x18\x04 \x01(\tR\x06destDc\x12'\n" + - "\x0fplacement_score\x18\x05 \x01(\x01R\x0eplacementScore\x12/\n" + - "\x13placement_conflicts\x18\x06 \x03(\tR\x12placementConflicts\x12#\n" + - "\rreplica_count\x18\a \x01(\x05R\freplicaCount\x12-\n" + - "\x12verify_consistency\x18\b \x01(\bR\x11verifyConsistency\"\x8e\x02\n" + - "\n" + - "TaskUpdate\x12\x17\n" + - "\atask_id\x18\x01 \x01(\tR\x06taskId\x12\x1b\n" + - "\tworker_id\x18\x02 \x01(\tR\bworkerId\x12\x16\n" + - "\x06status\x18\x03 \x01(\tR\x06status\x12\x1a\n" + - "\bprogress\x18\x04 \x01(\x02R\bprogress\x12\x18\n" + - "\amessage\x18\x05 \x01(\tR\amessage\x12?\n" + - "\bmetadata\x18\x06 \x03(\v2#.worker_pb.TaskUpdate.MetadataEntryR\bmetadata\x1a;\n" + - "\rMetadataEntry\x12\x10\n" + +func (x *MaintenanceTaskData) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MaintenanceTaskData) ProtoMessage() {} + +func (x *MaintenanceTaskData) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[31] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MaintenanceTaskData.ProtoReflect.Descriptor instead. +func (*MaintenanceTaskData) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{31} +} + +func (x *MaintenanceTaskData) GetId() string { + if x != nil { + return x.Id + } + return "" +} + +func (x *MaintenanceTaskData) GetType() string { + if x != nil { + return x.Type + } + return "" +} + +func (x *MaintenanceTaskData) GetPriority() string { + if x != nil { + return x.Priority + } + return "" +} + +func (x *MaintenanceTaskData) GetStatus() string { + if x != nil { + return x.Status + } + return "" +} + +func (x *MaintenanceTaskData) GetVolumeId() uint32 { + if x != nil { + return x.VolumeId + } + return 0 +} + +func (x *MaintenanceTaskData) GetServer() string { + if x != nil { + return x.Server + } + return "" +} + +func (x *MaintenanceTaskData) GetCollection() string { + if x != nil { + return x.Collection + } + return "" +} + +func (x *MaintenanceTaskData) GetTypedParams() *TaskParams { + if x != nil { + return x.TypedParams + } + return nil +} + +func (x *MaintenanceTaskData) GetReason() string { + if x != nil { + return x.Reason + } + return "" +} + +func (x *MaintenanceTaskData) GetCreatedAt() int64 { + if x != nil { + return x.CreatedAt + } + return 0 +} + +func (x *MaintenanceTaskData) GetScheduledAt() int64 { + if x != nil { + return x.ScheduledAt + } + return 0 +} + +func (x *MaintenanceTaskData) GetStartedAt() int64 { + if x != nil { + return x.StartedAt + } + return 0 +} + +func (x *MaintenanceTaskData) GetCompletedAt() int64 { + if x != nil { + return x.CompletedAt + } + return 0 +} + +func (x *MaintenanceTaskData) GetWorkerId() string { + if x != nil { + return x.WorkerId + } + return "" +} + +func (x *MaintenanceTaskData) GetError() string { + if x != nil { + return x.Error + } + return "" +} + +func (x *MaintenanceTaskData) GetProgress() float64 { + if x != nil { + return x.Progress + } + return 0 +} + +func (x *MaintenanceTaskData) GetRetryCount() int32 { + if x != nil { + return x.RetryCount + } + return 0 +} + +func (x *MaintenanceTaskData) GetMaxRetries() int32 { + if x != nil { + return x.MaxRetries + } + return 0 +} + +func (x *MaintenanceTaskData) GetCreatedBy() string { + if x != nil { + return x.CreatedBy + } + return "" +} + +func (x *MaintenanceTaskData) GetCreationContext() string { + if x != nil { + return x.CreationContext + } + return "" +} + +func (x *MaintenanceTaskData) GetAssignmentHistory() []*TaskAssignmentRecord { + if x != nil { + return x.AssignmentHistory + } + return nil +} + +func (x *MaintenanceTaskData) GetDetailedReason() string { + if x != nil { + return x.DetailedReason + } + return "" +} + +func (x *MaintenanceTaskData) GetTags() map[string]string { + if x != nil { + return x.Tags + } + return nil +} + +func (x *MaintenanceTaskData) GetCreationMetrics() *TaskCreationMetrics { + if x != nil { + return x.CreationMetrics + } + return nil +} + +// TaskAssignmentRecord tracks worker assignments for a task +type TaskAssignmentRecord struct { + state protoimpl.MessageState `protogen:"open.v1"` + WorkerId string `protobuf:"bytes,1,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"` + WorkerAddress string `protobuf:"bytes,2,opt,name=worker_address,json=workerAddress,proto3" json:"worker_address,omitempty"` + AssignedAt int64 `protobuf:"varint,3,opt,name=assigned_at,json=assignedAt,proto3" json:"assigned_at,omitempty"` + UnassignedAt int64 `protobuf:"varint,4,opt,name=unassigned_at,json=unassignedAt,proto3" json:"unassigned_at,omitempty"` // Optional: when worker was unassigned + Reason string `protobuf:"bytes,5,opt,name=reason,proto3" json:"reason,omitempty"` // Reason for assignment/unassignment + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *TaskAssignmentRecord) Reset() { + *x = TaskAssignmentRecord{} + mi := &file_worker_proto_msgTypes[32] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *TaskAssignmentRecord) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TaskAssignmentRecord) ProtoMessage() {} + +func (x *TaskAssignmentRecord) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[32] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TaskAssignmentRecord.ProtoReflect.Descriptor instead. +func (*TaskAssignmentRecord) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{32} +} + +func (x *TaskAssignmentRecord) GetWorkerId() string { + if x != nil { + return x.WorkerId + } + return "" +} + +func (x *TaskAssignmentRecord) GetWorkerAddress() string { + if x != nil { + return x.WorkerAddress + } + return "" +} + +func (x *TaskAssignmentRecord) GetAssignedAt() int64 { + if x != nil { + return x.AssignedAt + } + return 0 +} + +func (x *TaskAssignmentRecord) GetUnassignedAt() int64 { + if x != nil { + return x.UnassignedAt + } + return 0 +} + +func (x *TaskAssignmentRecord) GetReason() string { + if x != nil { + return x.Reason + } + return "" +} + +// TaskCreationMetrics tracks why and how a task was created +type TaskCreationMetrics struct { + state protoimpl.MessageState `protogen:"open.v1"` + TriggerMetric string `protobuf:"bytes,1,opt,name=trigger_metric,json=triggerMetric,proto3" json:"trigger_metric,omitempty"` // Name of metric that triggered creation + MetricValue float64 `protobuf:"fixed64,2,opt,name=metric_value,json=metricValue,proto3" json:"metric_value,omitempty"` // Value that triggered creation + Threshold float64 `protobuf:"fixed64,3,opt,name=threshold,proto3" json:"threshold,omitempty"` // Threshold that was exceeded + VolumeMetrics *VolumeHealthMetrics `protobuf:"bytes,4,opt,name=volume_metrics,json=volumeMetrics,proto3" json:"volume_metrics,omitempty"` // Volume health at creation time + AdditionalData map[string]string `protobuf:"bytes,5,rep,name=additional_data,json=additionalData,proto3" json:"additional_data,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // Additional context data + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *TaskCreationMetrics) Reset() { + *x = TaskCreationMetrics{} + mi := &file_worker_proto_msgTypes[33] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *TaskCreationMetrics) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TaskCreationMetrics) ProtoMessage() {} + +func (x *TaskCreationMetrics) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[33] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TaskCreationMetrics.ProtoReflect.Descriptor instead. +func (*TaskCreationMetrics) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{33} +} + +func (x *TaskCreationMetrics) GetTriggerMetric() string { + if x != nil { + return x.TriggerMetric + } + return "" +} + +func (x *TaskCreationMetrics) GetMetricValue() float64 { + if x != nil { + return x.MetricValue + } + return 0 +} + +func (x *TaskCreationMetrics) GetThreshold() float64 { + if x != nil { + return x.Threshold + } + return 0 +} + +func (x *TaskCreationMetrics) GetVolumeMetrics() *VolumeHealthMetrics { + if x != nil { + return x.VolumeMetrics + } + return nil +} + +func (x *TaskCreationMetrics) GetAdditionalData() map[string]string { + if x != nil { + return x.AdditionalData + } + return nil +} + +// VolumeHealthMetrics captures volume state at task creation +type VolumeHealthMetrics struct { + state protoimpl.MessageState `protogen:"open.v1"` + TotalSize uint64 `protobuf:"varint,1,opt,name=total_size,json=totalSize,proto3" json:"total_size,omitempty"` + UsedSize uint64 `protobuf:"varint,2,opt,name=used_size,json=usedSize,proto3" json:"used_size,omitempty"` + GarbageSize uint64 `protobuf:"varint,3,opt,name=garbage_size,json=garbageSize,proto3" json:"garbage_size,omitempty"` + GarbageRatio float64 `protobuf:"fixed64,4,opt,name=garbage_ratio,json=garbageRatio,proto3" json:"garbage_ratio,omitempty"` + FileCount int32 `protobuf:"varint,5,opt,name=file_count,json=fileCount,proto3" json:"file_count,omitempty"` + DeletedFileCount int32 `protobuf:"varint,6,opt,name=deleted_file_count,json=deletedFileCount,proto3" json:"deleted_file_count,omitempty"` + LastModified int64 `protobuf:"varint,7,opt,name=last_modified,json=lastModified,proto3" json:"last_modified,omitempty"` + ReplicaCount int32 `protobuf:"varint,8,opt,name=replica_count,json=replicaCount,proto3" json:"replica_count,omitempty"` + IsEcVolume bool `protobuf:"varint,9,opt,name=is_ec_volume,json=isEcVolume,proto3" json:"is_ec_volume,omitempty"` + Collection string `protobuf:"bytes,10,opt,name=collection,proto3" json:"collection,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *VolumeHealthMetrics) Reset() { + *x = VolumeHealthMetrics{} + mi := &file_worker_proto_msgTypes[34] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *VolumeHealthMetrics) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*VolumeHealthMetrics) ProtoMessage() {} + +func (x *VolumeHealthMetrics) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[34] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use VolumeHealthMetrics.ProtoReflect.Descriptor instead. +func (*VolumeHealthMetrics) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{34} +} + +func (x *VolumeHealthMetrics) GetTotalSize() uint64 { + if x != nil { + return x.TotalSize + } + return 0 +} + +func (x *VolumeHealthMetrics) GetUsedSize() uint64 { + if x != nil { + return x.UsedSize + } + return 0 +} + +func (x *VolumeHealthMetrics) GetGarbageSize() uint64 { + if x != nil { + return x.GarbageSize + } + return 0 +} + +func (x *VolumeHealthMetrics) GetGarbageRatio() float64 { + if x != nil { + return x.GarbageRatio + } + return 0 +} + +func (x *VolumeHealthMetrics) GetFileCount() int32 { + if x != nil { + return x.FileCount + } + return 0 +} + +func (x *VolumeHealthMetrics) GetDeletedFileCount() int32 { + if x != nil { + return x.DeletedFileCount + } + return 0 +} + +func (x *VolumeHealthMetrics) GetLastModified() int64 { + if x != nil { + return x.LastModified + } + return 0 +} + +func (x *VolumeHealthMetrics) GetReplicaCount() int32 { + if x != nil { + return x.ReplicaCount + } + return 0 +} + +func (x *VolumeHealthMetrics) GetIsEcVolume() bool { + if x != nil { + return x.IsEcVolume + } + return false +} + +func (x *VolumeHealthMetrics) GetCollection() string { + if x != nil { + return x.Collection + } + return "" +} + +// TaskStateFile wraps task data with metadata for persistence +type TaskStateFile struct { + state protoimpl.MessageState `protogen:"open.v1"` + Task *MaintenanceTaskData `protobuf:"bytes,1,opt,name=task,proto3" json:"task,omitempty"` + LastUpdated int64 `protobuf:"varint,2,opt,name=last_updated,json=lastUpdated,proto3" json:"last_updated,omitempty"` + AdminVersion string `protobuf:"bytes,3,opt,name=admin_version,json=adminVersion,proto3" json:"admin_version,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *TaskStateFile) Reset() { + *x = TaskStateFile{} + mi := &file_worker_proto_msgTypes[35] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *TaskStateFile) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TaskStateFile) ProtoMessage() {} + +func (x *TaskStateFile) ProtoReflect() protoreflect.Message { + mi := &file_worker_proto_msgTypes[35] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TaskStateFile.ProtoReflect.Descriptor instead. +func (*TaskStateFile) Descriptor() ([]byte, []int) { + return file_worker_proto_rawDescGZIP(), []int{35} +} + +func (x *TaskStateFile) GetTask() *MaintenanceTaskData { + if x != nil { + return x.Task + } + return nil +} + +func (x *TaskStateFile) GetLastUpdated() int64 { + if x != nil { + return x.LastUpdated + } + return 0 +} + +func (x *TaskStateFile) GetAdminVersion() string { + if x != nil { + return x.AdminVersion + } + return "" +} + +var File_worker_proto protoreflect.FileDescriptor + +const file_worker_proto_rawDesc = "" + + "\n" + + "\fworker.proto\x12\tworker_pb\"\x90\x04\n" + + "\rWorkerMessage\x12\x1b\n" + + "\tworker_id\x18\x01 \x01(\tR\bworkerId\x12\x1c\n" + + "\ttimestamp\x18\x02 \x01(\x03R\ttimestamp\x12C\n" + + "\fregistration\x18\x03 \x01(\v2\x1d.worker_pb.WorkerRegistrationH\x00R\fregistration\x12:\n" + + "\theartbeat\x18\x04 \x01(\v2\x1a.worker_pb.WorkerHeartbeatH\x00R\theartbeat\x12;\n" + + "\ftask_request\x18\x05 \x01(\v2\x16.worker_pb.TaskRequestH\x00R\vtaskRequest\x128\n" + + "\vtask_update\x18\x06 \x01(\v2\x15.worker_pb.TaskUpdateH\x00R\n" + + "taskUpdate\x12>\n" + + "\rtask_complete\x18\a \x01(\v2\x17.worker_pb.TaskCompleteH\x00R\ftaskComplete\x127\n" + + "\bshutdown\x18\b \x01(\v2\x19.worker_pb.WorkerShutdownH\x00R\bshutdown\x12H\n" + + "\x11task_log_response\x18\t \x01(\v2\x1a.worker_pb.TaskLogResponseH\x00R\x0ftaskLogResponseB\t\n" + + "\amessage\"\x95\x04\n" + + "\fAdminMessage\x12\x19\n" + + "\badmin_id\x18\x01 \x01(\tR\aadminId\x12\x1c\n" + + "\ttimestamp\x18\x02 \x01(\x03R\ttimestamp\x12V\n" + + "\x15registration_response\x18\x03 \x01(\v2\x1f.worker_pb.RegistrationResponseH\x00R\x14registrationResponse\x12M\n" + + "\x12heartbeat_response\x18\x04 \x01(\v2\x1c.worker_pb.HeartbeatResponseH\x00R\x11heartbeatResponse\x12D\n" + + "\x0ftask_assignment\x18\x05 \x01(\v2\x19.worker_pb.TaskAssignmentH\x00R\x0etaskAssignment\x12J\n" + + "\x11task_cancellation\x18\x06 \x01(\v2\x1b.worker_pb.TaskCancellationH\x00R\x10taskCancellation\x12A\n" + + "\x0eadmin_shutdown\x18\a \x01(\v2\x18.worker_pb.AdminShutdownH\x00R\radminShutdown\x12E\n" + + "\x10task_log_request\x18\b \x01(\v2\x19.worker_pb.TaskLogRequestH\x00R\x0etaskLogRequestB\t\n" + + "\amessage\"\x9c\x02\n" + + "\x12WorkerRegistration\x12\x1b\n" + + "\tworker_id\x18\x01 \x01(\tR\bworkerId\x12\x18\n" + + "\aaddress\x18\x02 \x01(\tR\aaddress\x12\"\n" + + "\fcapabilities\x18\x03 \x03(\tR\fcapabilities\x12%\n" + + "\x0emax_concurrent\x18\x04 \x01(\x05R\rmaxConcurrent\x12G\n" + + "\bmetadata\x18\x05 \x03(\v2+.worker_pb.WorkerRegistration.MetadataEntryR\bmetadata\x1a;\n" + + "\rMetadataEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"x\n" + + "\x14RegistrationResponse\x12\x18\n" + + "\asuccess\x18\x01 \x01(\bR\asuccess\x12\x18\n" + + "\amessage\x18\x02 \x01(\tR\amessage\x12,\n" + + "\x12assigned_worker_id\x18\x03 \x01(\tR\x10assignedWorkerId\"\xad\x02\n" + + "\x0fWorkerHeartbeat\x12\x1b\n" + + "\tworker_id\x18\x01 \x01(\tR\bworkerId\x12\x16\n" + + "\x06status\x18\x02 \x01(\tR\x06status\x12!\n" + + "\fcurrent_load\x18\x03 \x01(\x05R\vcurrentLoad\x12%\n" + + "\x0emax_concurrent\x18\x04 \x01(\x05R\rmaxConcurrent\x12(\n" + + "\x10current_task_ids\x18\x05 \x03(\tR\x0ecurrentTaskIds\x12'\n" + + "\x0ftasks_completed\x18\x06 \x01(\x05R\x0etasksCompleted\x12!\n" + + "\ftasks_failed\x18\a \x01(\x05R\vtasksFailed\x12%\n" + + "\x0euptime_seconds\x18\b \x01(\x03R\ruptimeSeconds\"G\n" + + "\x11HeartbeatResponse\x12\x18\n" + + "\asuccess\x18\x01 \x01(\bR\asuccess\x12\x18\n" + + "\amessage\x18\x02 \x01(\tR\amessage\"w\n" + + "\vTaskRequest\x12\x1b\n" + + "\tworker_id\x18\x01 \x01(\tR\bworkerId\x12\"\n" + + "\fcapabilities\x18\x02 \x03(\tR\fcapabilities\x12'\n" + + "\x0favailable_slots\x18\x03 \x01(\x05R\x0eavailableSlots\"\xb6\x02\n" + + "\x0eTaskAssignment\x12\x17\n" + + "\atask_id\x18\x01 \x01(\tR\x06taskId\x12\x1b\n" + + "\ttask_type\x18\x02 \x01(\tR\btaskType\x12-\n" + + "\x06params\x18\x03 \x01(\v2\x15.worker_pb.TaskParamsR\x06params\x12\x1a\n" + + "\bpriority\x18\x04 \x01(\x05R\bpriority\x12!\n" + + "\fcreated_time\x18\x05 \x01(\x03R\vcreatedTime\x12C\n" + + "\bmetadata\x18\x06 \x03(\v2'.worker_pb.TaskAssignment.MetadataEntryR\bmetadata\x1a;\n" + + "\rMetadataEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"\xe1\x04\n" + + "\n" + + "TaskParams\x12\x17\n" + + "\atask_id\x18\x01 \x01(\tR\x06taskId\x12\x1b\n" + + "\tvolume_id\x18\x02 \x01(\rR\bvolumeId\x12\x1e\n" + + "\n" + + "collection\x18\x03 \x01(\tR\n" + + "collection\x12\x1f\n" + + "\vdata_center\x18\x04 \x01(\tR\n" + + "dataCenter\x12\x12\n" + + "\x04rack\x18\x05 \x01(\tR\x04rack\x12\x1f\n" + + "\vvolume_size\x18\x06 \x01(\x04R\n" + + "volumeSize\x12/\n" + + "\asources\x18\a \x03(\v2\x15.worker_pb.TaskSourceR\asources\x12/\n" + + "\atargets\x18\b \x03(\v2\x15.worker_pb.TaskTargetR\atargets\x12B\n" + + "\rvacuum_params\x18\t \x01(\v2\x1b.worker_pb.VacuumTaskParamsH\x00R\fvacuumParams\x12X\n" + + "\x15erasure_coding_params\x18\n" + + " \x01(\v2\".worker_pb.ErasureCodingTaskParamsH\x00R\x13erasureCodingParams\x12E\n" + + "\x0ebalance_params\x18\v \x01(\v2\x1c.worker_pb.BalanceTaskParamsH\x00R\rbalanceParams\x12Q\n" + + "\x12replication_params\x18\f \x01(\v2 .worker_pb.ReplicationTaskParamsH\x00R\x11replicationParamsB\r\n" + + "\vtask_params\"\xcb\x01\n" + + "\x10VacuumTaskParams\x12+\n" + + "\x11garbage_threshold\x18\x01 \x01(\x01R\x10garbageThreshold\x12!\n" + + "\fforce_vacuum\x18\x02 \x01(\bR\vforceVacuum\x12\x1d\n" + + "\n" + + "batch_size\x18\x03 \x01(\x05R\tbatchSize\x12\x1f\n" + + "\vworking_dir\x18\x04 \x01(\tR\n" + + "workingDir\x12'\n" + + "\x0fverify_checksum\x18\x05 \x01(\bR\x0everifyChecksum\"\xfe\x01\n" + + "\x17ErasureCodingTaskParams\x120\n" + + "\x14estimated_shard_size\x18\x01 \x01(\x04R\x12estimatedShardSize\x12\x1f\n" + + "\vdata_shards\x18\x02 \x01(\x05R\n" + + "dataShards\x12#\n" + + "\rparity_shards\x18\x03 \x01(\x05R\fparityShards\x12\x1f\n" + + "\vworking_dir\x18\x04 \x01(\tR\n" + + "workingDir\x12#\n" + + "\rmaster_client\x18\x05 \x01(\tR\fmasterClient\x12%\n" + + "\x0ecleanup_source\x18\x06 \x01(\bR\rcleanupSource\"\xcf\x01\n" + + "\n" + + "TaskSource\x12\x12\n" + + "\x04node\x18\x01 \x01(\tR\x04node\x12\x17\n" + + "\adisk_id\x18\x02 \x01(\rR\x06diskId\x12\x12\n" + + "\x04rack\x18\x03 \x01(\tR\x04rack\x12\x1f\n" + + "\vdata_center\x18\x04 \x01(\tR\n" + + "dataCenter\x12\x1b\n" + + "\tvolume_id\x18\x05 \x01(\rR\bvolumeId\x12\x1b\n" + + "\tshard_ids\x18\x06 \x03(\rR\bshardIds\x12%\n" + + "\x0eestimated_size\x18\a \x01(\x04R\restimatedSize\"\xcf\x01\n" + + "\n" + + "TaskTarget\x12\x12\n" + + "\x04node\x18\x01 \x01(\tR\x04node\x12\x17\n" + + "\adisk_id\x18\x02 \x01(\rR\x06diskId\x12\x12\n" + + "\x04rack\x18\x03 \x01(\tR\x04rack\x12\x1f\n" + + "\vdata_center\x18\x04 \x01(\tR\n" + + "dataCenter\x12\x1b\n" + + "\tvolume_id\x18\x05 \x01(\rR\bvolumeId\x12\x1b\n" + + "\tshard_ids\x18\x06 \x03(\rR\bshardIds\x12%\n" + + "\x0eestimated_size\x18\a \x01(\x04R\restimatedSize\"[\n" + + "\x11BalanceTaskParams\x12\x1d\n" + + "\n" + + "force_move\x18\x01 \x01(\bR\tforceMove\x12'\n" + + "\x0ftimeout_seconds\x18\x02 \x01(\x05R\x0etimeoutSeconds\"k\n" + + "\x15ReplicationTaskParams\x12#\n" + + "\rreplica_count\x18\x01 \x01(\x05R\freplicaCount\x12-\n" + + "\x12verify_consistency\x18\x02 \x01(\bR\x11verifyConsistency\"\x8e\x02\n" + + "\n" + + "TaskUpdate\x12\x17\n" + + "\atask_id\x18\x01 \x01(\tR\x06taskId\x12\x1b\n" + + "\tworker_id\x18\x02 \x01(\tR\bworkerId\x12\x16\n" + + "\x06status\x18\x03 \x01(\tR\x06status\x12\x1a\n" + + "\bprogress\x18\x04 \x01(\x02R\bprogress\x12\x18\n" + + "\amessage\x18\x05 \x01(\tR\amessage\x12?\n" + + "\bmetadata\x18\x06 \x03(\v2#.worker_pb.TaskUpdate.MetadataEntryR\bmetadata\x1a;\n" + + "\rMetadataEntry\x12\x10\n" + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"\xc5\x02\n" + "\fTaskComplete\x12\x17\n" + @@ -3076,7 +3569,80 @@ const file_worker_proto_rawDesc = "" + "\x13imbalance_threshold\x18\x01 \x01(\x01R\x12imbalanceThreshold\x12(\n" + "\x10min_server_count\x18\x02 \x01(\x05R\x0eminServerCount\"I\n" + "\x15ReplicationTaskConfig\x120\n" + - "\x14target_replica_count\x18\x01 \x01(\x05R\x12targetReplicaCount2V\n" + + "\x14target_replica_count\x18\x01 \x01(\x05R\x12targetReplicaCount\"\xae\a\n" + + "\x13MaintenanceTaskData\x12\x0e\n" + + "\x02id\x18\x01 \x01(\tR\x02id\x12\x12\n" + + "\x04type\x18\x02 \x01(\tR\x04type\x12\x1a\n" + + "\bpriority\x18\x03 \x01(\tR\bpriority\x12\x16\n" + + "\x06status\x18\x04 \x01(\tR\x06status\x12\x1b\n" + + "\tvolume_id\x18\x05 \x01(\rR\bvolumeId\x12\x16\n" + + "\x06server\x18\x06 \x01(\tR\x06server\x12\x1e\n" + + "\n" + + "collection\x18\a \x01(\tR\n" + + "collection\x128\n" + + "\ftyped_params\x18\b \x01(\v2\x15.worker_pb.TaskParamsR\vtypedParams\x12\x16\n" + + "\x06reason\x18\t \x01(\tR\x06reason\x12\x1d\n" + + "\n" + + "created_at\x18\n" + + " \x01(\x03R\tcreatedAt\x12!\n" + + "\fscheduled_at\x18\v \x01(\x03R\vscheduledAt\x12\x1d\n" + + "\n" + + "started_at\x18\f \x01(\x03R\tstartedAt\x12!\n" + + "\fcompleted_at\x18\r \x01(\x03R\vcompletedAt\x12\x1b\n" + + "\tworker_id\x18\x0e \x01(\tR\bworkerId\x12\x14\n" + + "\x05error\x18\x0f \x01(\tR\x05error\x12\x1a\n" + + "\bprogress\x18\x10 \x01(\x01R\bprogress\x12\x1f\n" + + "\vretry_count\x18\x11 \x01(\x05R\n" + + "retryCount\x12\x1f\n" + + "\vmax_retries\x18\x12 \x01(\x05R\n" + + "maxRetries\x12\x1d\n" + + "\n" + + "created_by\x18\x13 \x01(\tR\tcreatedBy\x12)\n" + + "\x10creation_context\x18\x14 \x01(\tR\x0fcreationContext\x12N\n" + + "\x12assignment_history\x18\x15 \x03(\v2\x1f.worker_pb.TaskAssignmentRecordR\x11assignmentHistory\x12'\n" + + "\x0fdetailed_reason\x18\x16 \x01(\tR\x0edetailedReason\x12<\n" + + "\x04tags\x18\x17 \x03(\v2(.worker_pb.MaintenanceTaskData.TagsEntryR\x04tags\x12I\n" + + "\x10creation_metrics\x18\x18 \x01(\v2\x1e.worker_pb.TaskCreationMetricsR\x0fcreationMetrics\x1a7\n" + + "\tTagsEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"\xb8\x01\n" + + "\x14TaskAssignmentRecord\x12\x1b\n" + + "\tworker_id\x18\x01 \x01(\tR\bworkerId\x12%\n" + + "\x0eworker_address\x18\x02 \x01(\tR\rworkerAddress\x12\x1f\n" + + "\vassigned_at\x18\x03 \x01(\x03R\n" + + "assignedAt\x12#\n" + + "\runassigned_at\x18\x04 \x01(\x03R\funassignedAt\x12\x16\n" + + "\x06reason\x18\x05 \x01(\tR\x06reason\"\xe4\x02\n" + + "\x13TaskCreationMetrics\x12%\n" + + "\x0etrigger_metric\x18\x01 \x01(\tR\rtriggerMetric\x12!\n" + + "\fmetric_value\x18\x02 \x01(\x01R\vmetricValue\x12\x1c\n" + + "\tthreshold\x18\x03 \x01(\x01R\tthreshold\x12E\n" + + "\x0evolume_metrics\x18\x04 \x01(\v2\x1e.worker_pb.VolumeHealthMetricsR\rvolumeMetrics\x12[\n" + + "\x0fadditional_data\x18\x05 \x03(\v22.worker_pb.TaskCreationMetrics.AdditionalDataEntryR\x0eadditionalData\x1aA\n" + + "\x13AdditionalDataEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"\xf2\x02\n" + + "\x13VolumeHealthMetrics\x12\x1d\n" + + "\n" + + "total_size\x18\x01 \x01(\x04R\ttotalSize\x12\x1b\n" + + "\tused_size\x18\x02 \x01(\x04R\busedSize\x12!\n" + + "\fgarbage_size\x18\x03 \x01(\x04R\vgarbageSize\x12#\n" + + "\rgarbage_ratio\x18\x04 \x01(\x01R\fgarbageRatio\x12\x1d\n" + + "\n" + + "file_count\x18\x05 \x01(\x05R\tfileCount\x12,\n" + + "\x12deleted_file_count\x18\x06 \x01(\x05R\x10deletedFileCount\x12#\n" + + "\rlast_modified\x18\a \x01(\x03R\flastModified\x12#\n" + + "\rreplica_count\x18\b \x01(\x05R\freplicaCount\x12 \n" + + "\fis_ec_volume\x18\t \x01(\bR\n" + + "isEcVolume\x12\x1e\n" + + "\n" + + "collection\x18\n" + + " \x01(\tR\n" + + "collection\"\x8b\x01\n" + + "\rTaskStateFile\x122\n" + + "\x04task\x18\x01 \x01(\v2\x1e.worker_pb.MaintenanceTaskDataR\x04task\x12!\n" + + "\flast_updated\x18\x02 \x01(\x03R\vlastUpdated\x12#\n" + + "\radmin_version\x18\x03 \x01(\tR\fadminVersion2V\n" + "\rWorkerService\x12E\n" + "\fWorkerStream\x12\x18.worker_pb.WorkerMessage\x1a\x17.worker_pb.AdminMessage(\x010\x01B2Z0github.com/seaweedfs/seaweedfs/weed/pb/worker_pbb\x06proto3" @@ -3092,7 +3658,7 @@ func file_worker_proto_rawDescGZIP() []byte { return file_worker_proto_rawDescData } -var file_worker_proto_msgTypes = make([]protoimpl.MessageInfo, 38) +var file_worker_proto_msgTypes = make([]protoimpl.MessageInfo, 45) var file_worker_proto_goTypes = []any{ (*WorkerMessage)(nil), // 0: worker_pb.WorkerMessage (*AdminMessage)(nil), // 1: worker_pb.AdminMessage @@ -3105,8 +3671,8 @@ var file_worker_proto_goTypes = []any{ (*TaskParams)(nil), // 8: worker_pb.TaskParams (*VacuumTaskParams)(nil), // 9: worker_pb.VacuumTaskParams (*ErasureCodingTaskParams)(nil), // 10: worker_pb.ErasureCodingTaskParams - (*ECDestination)(nil), // 11: worker_pb.ECDestination - (*ExistingECShardLocation)(nil), // 12: worker_pb.ExistingECShardLocation + (*TaskSource)(nil), // 11: worker_pb.TaskSource + (*TaskTarget)(nil), // 12: worker_pb.TaskTarget (*BalanceTaskParams)(nil), // 13: worker_pb.BalanceTaskParams (*ReplicationTaskParams)(nil), // 14: worker_pb.ReplicationTaskParams (*TaskUpdate)(nil), // 15: worker_pb.TaskUpdate @@ -3125,13 +3691,20 @@ var file_worker_proto_goTypes = []any{ (*ErasureCodingTaskConfig)(nil), // 28: worker_pb.ErasureCodingTaskConfig (*BalanceTaskConfig)(nil), // 29: worker_pb.BalanceTaskConfig (*ReplicationTaskConfig)(nil), // 30: worker_pb.ReplicationTaskConfig - nil, // 31: worker_pb.WorkerRegistration.MetadataEntry - nil, // 32: worker_pb.TaskAssignment.MetadataEntry - nil, // 33: worker_pb.TaskUpdate.MetadataEntry - nil, // 34: worker_pb.TaskComplete.ResultMetadataEntry - nil, // 35: worker_pb.TaskLogMetadata.CustomDataEntry - nil, // 36: worker_pb.TaskLogEntry.FieldsEntry - nil, // 37: worker_pb.MaintenancePolicy.TaskPoliciesEntry + (*MaintenanceTaskData)(nil), // 31: worker_pb.MaintenanceTaskData + (*TaskAssignmentRecord)(nil), // 32: worker_pb.TaskAssignmentRecord + (*TaskCreationMetrics)(nil), // 33: worker_pb.TaskCreationMetrics + (*VolumeHealthMetrics)(nil), // 34: worker_pb.VolumeHealthMetrics + (*TaskStateFile)(nil), // 35: worker_pb.TaskStateFile + nil, // 36: worker_pb.WorkerRegistration.MetadataEntry + nil, // 37: worker_pb.TaskAssignment.MetadataEntry + nil, // 38: worker_pb.TaskUpdate.MetadataEntry + nil, // 39: worker_pb.TaskComplete.ResultMetadataEntry + nil, // 40: worker_pb.TaskLogMetadata.CustomDataEntry + nil, // 41: worker_pb.TaskLogEntry.FieldsEntry + nil, // 42: worker_pb.MaintenancePolicy.TaskPoliciesEntry + nil, // 43: worker_pb.MaintenanceTaskData.TagsEntry + nil, // 44: worker_pb.TaskCreationMetrics.AdditionalDataEntry } var file_worker_proto_depIdxs = []int32{ 2, // 0: worker_pb.WorkerMessage.registration:type_name -> worker_pb.WorkerRegistration @@ -3147,35 +3720,42 @@ var file_worker_proto_depIdxs = []int32{ 17, // 10: worker_pb.AdminMessage.task_cancellation:type_name -> worker_pb.TaskCancellation 19, // 11: worker_pb.AdminMessage.admin_shutdown:type_name -> worker_pb.AdminShutdown 20, // 12: worker_pb.AdminMessage.task_log_request:type_name -> worker_pb.TaskLogRequest - 31, // 13: worker_pb.WorkerRegistration.metadata:type_name -> worker_pb.WorkerRegistration.MetadataEntry + 36, // 13: worker_pb.WorkerRegistration.metadata:type_name -> worker_pb.WorkerRegistration.MetadataEntry 8, // 14: worker_pb.TaskAssignment.params:type_name -> worker_pb.TaskParams - 32, // 15: worker_pb.TaskAssignment.metadata:type_name -> worker_pb.TaskAssignment.MetadataEntry - 9, // 16: worker_pb.TaskParams.vacuum_params:type_name -> worker_pb.VacuumTaskParams - 10, // 17: worker_pb.TaskParams.erasure_coding_params:type_name -> worker_pb.ErasureCodingTaskParams - 13, // 18: worker_pb.TaskParams.balance_params:type_name -> worker_pb.BalanceTaskParams - 14, // 19: worker_pb.TaskParams.replication_params:type_name -> worker_pb.ReplicationTaskParams - 11, // 20: worker_pb.ErasureCodingTaskParams.destinations:type_name -> worker_pb.ECDestination - 12, // 21: worker_pb.ErasureCodingTaskParams.existing_shard_locations:type_name -> worker_pb.ExistingECShardLocation - 33, // 22: worker_pb.TaskUpdate.metadata:type_name -> worker_pb.TaskUpdate.MetadataEntry - 34, // 23: worker_pb.TaskComplete.result_metadata:type_name -> worker_pb.TaskComplete.ResultMetadataEntry + 37, // 15: worker_pb.TaskAssignment.metadata:type_name -> worker_pb.TaskAssignment.MetadataEntry + 11, // 16: worker_pb.TaskParams.sources:type_name -> worker_pb.TaskSource + 12, // 17: worker_pb.TaskParams.targets:type_name -> worker_pb.TaskTarget + 9, // 18: worker_pb.TaskParams.vacuum_params:type_name -> worker_pb.VacuumTaskParams + 10, // 19: worker_pb.TaskParams.erasure_coding_params:type_name -> worker_pb.ErasureCodingTaskParams + 13, // 20: worker_pb.TaskParams.balance_params:type_name -> worker_pb.BalanceTaskParams + 14, // 21: worker_pb.TaskParams.replication_params:type_name -> worker_pb.ReplicationTaskParams + 38, // 22: worker_pb.TaskUpdate.metadata:type_name -> worker_pb.TaskUpdate.MetadataEntry + 39, // 23: worker_pb.TaskComplete.result_metadata:type_name -> worker_pb.TaskComplete.ResultMetadataEntry 22, // 24: worker_pb.TaskLogResponse.metadata:type_name -> worker_pb.TaskLogMetadata 23, // 25: worker_pb.TaskLogResponse.log_entries:type_name -> worker_pb.TaskLogEntry - 35, // 26: worker_pb.TaskLogMetadata.custom_data:type_name -> worker_pb.TaskLogMetadata.CustomDataEntry - 36, // 27: worker_pb.TaskLogEntry.fields:type_name -> worker_pb.TaskLogEntry.FieldsEntry + 40, // 26: worker_pb.TaskLogMetadata.custom_data:type_name -> worker_pb.TaskLogMetadata.CustomDataEntry + 41, // 27: worker_pb.TaskLogEntry.fields:type_name -> worker_pb.TaskLogEntry.FieldsEntry 25, // 28: worker_pb.MaintenanceConfig.policy:type_name -> worker_pb.MaintenancePolicy - 37, // 29: worker_pb.MaintenancePolicy.task_policies:type_name -> worker_pb.MaintenancePolicy.TaskPoliciesEntry + 42, // 29: worker_pb.MaintenancePolicy.task_policies:type_name -> worker_pb.MaintenancePolicy.TaskPoliciesEntry 27, // 30: worker_pb.TaskPolicy.vacuum_config:type_name -> worker_pb.VacuumTaskConfig 28, // 31: worker_pb.TaskPolicy.erasure_coding_config:type_name -> worker_pb.ErasureCodingTaskConfig 29, // 32: worker_pb.TaskPolicy.balance_config:type_name -> worker_pb.BalanceTaskConfig 30, // 33: worker_pb.TaskPolicy.replication_config:type_name -> worker_pb.ReplicationTaskConfig - 26, // 34: worker_pb.MaintenancePolicy.TaskPoliciesEntry.value:type_name -> worker_pb.TaskPolicy - 0, // 35: worker_pb.WorkerService.WorkerStream:input_type -> worker_pb.WorkerMessage - 1, // 36: worker_pb.WorkerService.WorkerStream:output_type -> worker_pb.AdminMessage - 36, // [36:37] is the sub-list for method output_type - 35, // [35:36] is the sub-list for method input_type - 35, // [35:35] is the sub-list for extension type_name - 35, // [35:35] is the sub-list for extension extendee - 0, // [0:35] is the sub-list for field type_name + 8, // 34: worker_pb.MaintenanceTaskData.typed_params:type_name -> worker_pb.TaskParams + 32, // 35: worker_pb.MaintenanceTaskData.assignment_history:type_name -> worker_pb.TaskAssignmentRecord + 43, // 36: worker_pb.MaintenanceTaskData.tags:type_name -> worker_pb.MaintenanceTaskData.TagsEntry + 33, // 37: worker_pb.MaintenanceTaskData.creation_metrics:type_name -> worker_pb.TaskCreationMetrics + 34, // 38: worker_pb.TaskCreationMetrics.volume_metrics:type_name -> worker_pb.VolumeHealthMetrics + 44, // 39: worker_pb.TaskCreationMetrics.additional_data:type_name -> worker_pb.TaskCreationMetrics.AdditionalDataEntry + 31, // 40: worker_pb.TaskStateFile.task:type_name -> worker_pb.MaintenanceTaskData + 26, // 41: worker_pb.MaintenancePolicy.TaskPoliciesEntry.value:type_name -> worker_pb.TaskPolicy + 0, // 42: worker_pb.WorkerService.WorkerStream:input_type -> worker_pb.WorkerMessage + 1, // 43: worker_pb.WorkerService.WorkerStream:output_type -> worker_pb.AdminMessage + 43, // [43:44] is the sub-list for method output_type + 42, // [42:43] is the sub-list for method input_type + 42, // [42:42] is the sub-list for extension type_name + 42, // [42:42] is the sub-list for extension extendee + 0, // [0:42] is the sub-list for field type_name } func init() { file_worker_proto_init() } @@ -3218,7 +3798,7 @@ func file_worker_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: unsafe.Slice(unsafe.StringData(file_worker_proto_rawDesc), len(file_worker_proto_rawDesc)), NumEnums: 0, - NumMessages: 38, + NumMessages: 45, NumExtensions: 0, NumServices: 1, }, diff --git a/weed/worker/client.go b/weed/worker/client.go index ef7e431c0..b9042f18c 100644 --- a/weed/worker/client.go +++ b/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 "" +} diff --git a/weed/worker/log_adapter.go b/weed/worker/log_adapter.go new file mode 100644 index 000000000..7a8f7578f --- /dev/null +++ b/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 +} diff --git a/weed/worker/tasks/balance/balance_task.go b/weed/worker/tasks/balance/balance_task.go index 439a406a4..8daafde97 100644 --- a/weed/worker/tasks/balance/balance_task.go +++ b/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 diff --git a/weed/worker/tasks/balance/detection.go b/weed/worker/tasks/balance/detection.go index be03fb92f..6d433c719 100644 --- a/weed/worker/tasks/balance/detection.go +++ b/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 -} diff --git a/weed/worker/tasks/balance/execution.go b/weed/worker/tasks/balance/execution.go index 91cd912f0..0acd2b662 100644 --- a/weed/worker/tasks/balance/execution.go +++ b/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 diff --git a/weed/worker/tasks/balance/register.go b/weed/worker/tasks/balance/register.go index adf30c11c..76d56c7c5 100644 --- a/weed/worker/tasks/balance/register.go +++ b/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 diff --git a/weed/worker/tasks/base/typed_task.go b/weed/worker/tasks/base/typed_task.go index 9d2839607..1530f6314 100644 --- a/weed/worker/tasks/base/typed_task.go +++ b/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 } diff --git a/weed/worker/tasks/erasure_coding/detection.go b/weed/worker/tasks/erasure_coding/detection.go index ec632436f..cd74bed33 100644 --- a/weed/worker/tasks/erasure_coding/detection.go +++ b/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 { diff --git a/weed/worker/tasks/erasure_coding/ec_task.go b/weed/worker/tasks/erasure_coding/ec_task.go index 97332f63f..18f192bc9 100644 --- a/weed/worker/tasks/erasure_coding/ec_task.go +++ b/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 } diff --git a/weed/worker/tasks/erasure_coding/register.go b/weed/worker/tasks/erasure_coding/register.go index 883aaf965..e574e0033 100644 --- a/weed/worker/tasks/erasure_coding/register.go +++ b/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 diff --git a/weed/worker/tasks/task.go b/weed/worker/tasks/task.go index 9813ae97f..f3eed8b2d 100644 --- a/weed/worker/tasks/task.go +++ b/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 "" +} diff --git a/weed/worker/tasks/task_log_handler.go b/weed/worker/tasks/task_log_handler.go index be5f00f12..fee62325e 100644 --- a/weed/worker/tasks/task_log_handler.go +++ b/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 diff --git a/weed/worker/tasks/task_logger.go b/weed/worker/tasks/task_logger.go index e9c06c35c..430513184 100644 --- a/weed/worker/tasks/task_logger.go +++ b/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, }) diff --git a/weed/worker/tasks/vacuum/detection.go b/weed/worker/tasks/vacuum/detection.go index 0c14bb956..bd86a2742 100644 --- a/weed/worker/tasks/vacuum/detection.go +++ b/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, diff --git a/weed/worker/tasks/vacuum/register.go b/weed/worker/tasks/vacuum/register.go index 66d94d28e..2c1360b5b 100644 --- a/weed/worker/tasks/vacuum/register.go +++ b/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 diff --git a/weed/worker/tasks/vacuum/vacuum_task.go b/weed/worker/tasks/vacuum/vacuum_task.go index 005f5a681..ebb41564f 100644 --- a/weed/worker/tasks/vacuum/vacuum_task.go +++ b/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 { diff --git a/weed/worker/types/base/task.go b/weed/worker/types/base/task.go index 5403f8ae9..243df5630 100644 --- a/weed/worker/types/base/task.go +++ b/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 diff --git a/weed/worker/types/data_types.go b/weed/worker/types/data_types.go index 203cbfadb..c8a67edc7 100644 --- a/weed/worker/types/data_types.go +++ b/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 diff --git a/weed/worker/types/task.go b/weed/worker/types/task.go index 2c9ed7f8a..9106a63e3 100644 --- a/weed/worker/types/task.go +++ b/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 diff --git a/weed/worker/types/task_types.go b/weed/worker/types/task_types.go index d5dbc4008..c4cafd07f 100644 --- a/weed/worker/types/task_types.go +++ b/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"` diff --git a/weed/worker/types/typed_task_interface.go b/weed/worker/types/typed_task_interface.go index 3dffe510c..d04cea3d3 100644 --- a/weed/worker/types/typed_task_interface.go +++ b/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) diff --git a/weed/worker/worker.go b/weed/worker/worker.go index 2bc0e1e11..49d1ea57f 100644 --- a/weed/worker/worker.go +++ b/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),