aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorChris Lu <chrislusf@users.noreply.github.com>2025-08-09 21:47:29 -0700
committerGitHub <noreply@github.com>2025-08-09 21:47:29 -0700
commit25bbf4c3d44b1c8a9aa4980e37ed399ec249f771 (patch)
treeaabb2ce3c6f55e4cf1e26ce2b6989086c17830bc
parent3ac2a2e22d863753a6b568596fbe9d76d03023b5 (diff)
downloadseaweedfs-25bbf4c3d44b1c8a9aa4980e37ed399ec249f771.tar.xz
seaweedfs-25bbf4c3d44b1c8a9aa4980e37ed399ec249f771.zip
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>
-rw-r--r--weed/admin/dash/admin_server.go265
-rw-r--r--weed/admin/dash/config_persistence.go545
-rw-r--r--weed/admin/dash/ec_shard_management.go13
-rw-r--r--weed/admin/dash/volume_management.go18
-rw-r--r--weed/admin/dash/worker_grpc_server.go164
-rw-r--r--weed/admin/handlers/admin_handlers.go6
-rw-r--r--weed/admin/handlers/cluster_handlers.go8
-rw-r--r--weed/admin/handlers/maintenance_handlers.go91
-rw-r--r--weed/admin/maintenance/maintenance_queue.go236
-rw-r--r--weed/admin/maintenance/maintenance_scanner.go4
-rw-r--r--weed/admin/maintenance/maintenance_types.go67
-rw-r--r--weed/admin/topology/structs.go19
-rw-r--r--weed/admin/topology/task_management.go9
-rw-r--r--weed/admin/topology/topology_management.go12
-rw-r--r--weed/admin/view/app/admin.templ2
-rw-r--r--weed/admin/view/app/admin_templ.go2
-rw-r--r--weed/admin/view/app/cluster_ec_volumes.templ13
-rw-r--r--weed/admin/view/app/cluster_ec_volumes_templ.go501
-rw-r--r--weed/admin/view/app/cluster_volume_servers.templ4
-rw-r--r--weed/admin/view/app/cluster_volume_servers_templ.go217
-rw-r--r--weed/admin/view/app/collection_details.templ10
-rw-r--r--weed/admin/view/app/collection_details_templ.go99
-rw-r--r--weed/admin/view/app/maintenance_queue.templ221
-rw-r--r--weed/admin/view/app/maintenance_queue_templ.go718
-rw-r--r--weed/admin/view/app/task_config_schema.templ3
-rw-r--r--weed/admin/view/app/task_config_schema_templ.go425
-rw-r--r--weed/admin/view/app/task_detail.templ1118
-rw-r--r--weed/admin/view/app/task_detail_templ.go1628
-rw-r--r--weed/pb/worker.proto167
-rw-r--r--weed/pb/worker_pb/worker.pb.go1140
-rw-r--r--weed/worker/client.go10
-rw-r--r--weed/worker/log_adapter.go85
-rw-r--r--weed/worker/tasks/balance/balance_task.go31
-rw-r--r--weed/worker/tasks/balance/detection.go66
-rw-r--r--weed/worker/tasks/balance/execution.go72
-rw-r--r--weed/worker/tasks/balance/register.go5
-rw-r--r--weed/worker/tasks/base/typed_task.go39
-rw-r--r--weed/worker/tasks/erasure_coding/detection.go194
-rw-r--r--weed/worker/tasks/erasure_coding/ec_task.go368
-rw-r--r--weed/worker/tasks/erasure_coding/register.go5
-rw-r--r--weed/worker/tasks/task.go66
-rw-r--r--weed/worker/tasks/task_log_handler.go36
-rw-r--r--weed/worker/tasks/task_logger.go4
-rw-r--r--weed/worker/tasks/vacuum/detection.go22
-rw-r--r--weed/worker/tasks/vacuum/register.go5
-rw-r--r--weed/worker/tasks/vacuum/vacuum_task.go12
-rw-r--r--weed/worker/types/base/task.go25
-rw-r--r--weed/worker/types/data_types.go2
-rw-r--r--weed/worker/types/task.go27
-rw-r--r--weed/worker/types/task_types.go1
-rw-r--r--weed/worker/types/typed_task_interface.go2
-rw-r--r--weed/worker/worker.go95
52 files changed, 7100 insertions, 1797 deletions
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) {
</h1>
<div class="btn-toolbar mb-2 mb-md-0">
<div class="btn-group me-2">
- <a href="/s3/buckets" class="btn btn-sm btn-primary">
+ <a href="/object-store/buckets" class="btn btn-sm btn-primary">
<i class="fas fa-cube me-1"></i>Object Store Buckets
</a>
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, "<div class=\"d-flex justify-content-between flex-wrap flex-md-nowrap align-items-center pt-3 pb-2 mb-3 border-bottom\"><h1 class=\"h2\"><i class=\"fas fa-tachometer-alt me-2\"></i>Dashboard</h1><div class=\"btn-toolbar mb-2 mb-md-0\"><div class=\"btn-group me-2\"><a href=\"/s3/buckets\" class=\"btn btn-sm btn-primary\"><i class=\"fas fa-cube me-1\"></i>Object Store Buckets</a></div></div></div><div id=\"dashboard-content\"><!-- Summary Cards --><div class=\"row mb-4\"><div class=\"col-xl-3 col-md-6 mb-4\"><div class=\"card border-left-success shadow h-100 py-2\"><div class=\"card-body\"><div class=\"row no-gutters align-items-center\"><div class=\"col mr-2\"><div class=\"text-xs font-weight-bold text-success text-uppercase mb-1\">Total Volumes</div><div class=\"h5 mb-0 font-weight-bold text-gray-800\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 1, "<div class=\"d-flex justify-content-between flex-wrap flex-md-nowrap align-items-center pt-3 pb-2 mb-3 border-bottom\"><h1 class=\"h2\"><i class=\"fas fa-tachometer-alt me-2\"></i>Dashboard</h1><div class=\"btn-toolbar mb-2 mb-md-0\"><div class=\"btn-group me-2\"><a href=\"/object-store/buckets\" class=\"btn btn-sm btn-primary\"><i class=\"fas fa-cube me-1\"></i>Object Store Buckets</a></div></div></div><div id=\"dashboard-content\"><!-- Summary Cards --><div class=\"row mb-4\"><div class=\"col-xl-3 col-md-6 mb-4\"><div class=\"card border-left-success shadow h-100 py-2\"><div class=\"card-body\"><div class=\"row no-gutters align-items-center\"><div class=\"col mr-2\"><div class=\"text-xs font-weight-bold text-success text-uppercase mb-1\">Total Volumes</div><div class=\"h5 mb-0 font-weight-bold text-gray-800\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
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) {
<div class="alert alert-info mb-4" role="alert">
<i class="fas fa-info-circle me-2"></i>
<strong>EC Storage Note:</strong>
- EC volumes use erasure coding (10+4) which stores data across 14 shards with redundancy.
- Physical storage is approximately 1.4x the original logical data size due to 4 parity shards.
+ EC volumes use erasure coding ({ fmt.Sprintf("%d+%d", erasure_coding.DataShardsCount, erasure_coding.ParityShardsCount) }) which stores data across { fmt.Sprintf("%d", erasure_coding.TotalShardsCount) } shards with redundancy.
+ Physical storage is approximately { fmt.Sprintf("%.1fx", float64(erasure_coding.TotalShardsCount)/float64(erasure_coding.DataShardsCount)) } the original logical data size due to { fmt.Sprintf("%d", erasure_coding.ParityShardsCount) } parity shards.
</div>
<!-- Volumes Table -->
@@ -688,7 +689,7 @@ func formatIndividualShardSizes(shardSizes map[int]int64) string {
}
var idRanges []string
- if len(shardIds) <= 4 {
+ if len(shardIds) <= erasure_coding.ParityShardsCount {
// Show individual IDs if few shards
for _, id := range shardIds {
idRanges = append(idRanges, fmt.Sprintf("%d", id))
@@ -719,11 +720,11 @@ templ displayEcVolumeStatus(volume dash.EcVolumeWithShards) {
if volume.IsComplete {
<span class="badge bg-success"><i class="fas fa-check me-1"></i>Complete</span>
} else {
- if len(volume.MissingShards) > 10 {
+ if len(volume.MissingShards) > erasure_coding.DataShardsCount {
<span class="badge bg-danger"><i class="fas fa-skull me-1"></i>Critical ({fmt.Sprintf("%d", len(volume.MissingShards))} missing)</span>
- } else if len(volume.MissingShards) > 6 {
+ } else if len(volume.MissingShards) > (erasure_coding.DataShardsCount/2) {
<span class="badge bg-warning"><i class="fas fa-exclamation-triangle me-1"></i>Degraded ({fmt.Sprintf("%d", len(volume.MissingShards))} missing)</span>
- } else if len(volume.MissingShards) > 2 {
+ } else if len(volume.MissingShards) > (erasure_coding.ParityShardsCount/2) {
<span class="badge bg-warning"><i class="fas fa-info-circle me-1"></i>Incomplete ({fmt.Sprintf("%d", len(volume.MissingShards))} missing)</span>
} else {
<span class="badge bg-info"><i class="fas fa-info-circle me-1"></i>Minor Issues ({fmt.Sprintf("%d", len(volume.MissingShards))} missing)</span>
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, "</h4><small>Missing shards</small></div><div class=\"align-self-center\"><i class=\"fas fa-exclamation-triangle fa-2x\"></i></div></div></div></div></div></div><!-- EC Storage Information Note --><div class=\"alert alert-info mb-4\" role=\"alert\"><i class=\"fas fa-info-circle me-2\"></i> <strong>EC Storage Note:</strong> EC volumes use erasure coding (10+4) which stores data across 14 shards with redundancy. Physical storage is approximately 1.4x the original logical data size due to 4 parity shards.</div><!-- Volumes Table --><div class=\"d-flex justify-content-between align-items-center mb-3\"><div class=\"d-flex align-items-center\"><span class=\"me-3\">Showing ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 6, "</h4><small>Missing shards</small></div><div class=\"align-self-center\"><i class=\"fas fa-exclamation-triangle fa-2x\"></i></div></div></div></div></div></div><!-- EC Storage Information Note --><div class=\"alert alert-info mb-4\" role=\"alert\"><i class=\"fas fa-info-circle me-2\"></i> <strong>EC Storage Note:</strong> EC volumes use erasure coding (")
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.</div><!-- Volumes Table --><div class=\"d-flex justify-content-between align-items-center mb-3\"><div class=\"d-flex align-items-center\"><span class=\"me-3\">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</span><div class=\"d-flex align-items-center\"><label for=\"pageSize\" class=\"form-label me-2 mb-0\">Show:</label> <select id=\"pageSize\" class=\"form-select form-select-sm\" style=\"width: auto;\" onchange=\"changePageSize(this.value)\"><option value=\"5\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, " volumes</span><div class=\"d-flex align-items-center\"><label for=\"pageSize\" class=\"form-label me-2 mb-0\">Show:</label> <select id=\"pageSize\" class=\"form-select form-select-sm\" style=\"width: auto;\" onchange=\"changePageSize(this.value)\"><option value=\"5\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if data.PageSize == 5 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 10, " selected")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 14, " selected")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 11, ">5</option> <option value=\"10\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, ">5</option> <option value=\"10\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if data.PageSize == 10 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 12, " selected")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, " selected")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, ">10</option> <option value=\"25\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, ">10</option> <option value=\"25\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if data.PageSize == 25 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 14, " selected")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 18, " selected")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, ">25</option> <option value=\"50\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, ">25</option> <option value=\"50\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if data.PageSize == 50 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, " selected")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, " selected")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, ">50</option> <option value=\"100\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, ">50</option> <option value=\"100\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if data.PageSize == 100 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 18, " selected")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 22, " selected")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, ">100</option></select> <span class=\"ms-2\">per page</span></div></div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 23, ">100</option></select> <span class=\"ms-2\">per page</span></div></div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if data.Collection != "" {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, "<div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, "<div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if data.Collection == "default" {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, "<span class=\"badge bg-secondary text-white\">Collection: default</span> ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 25, "<span class=\"badge bg-secondary text-white\">Collection: default</span> ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 22, "<span class=\"badge bg-info text-white\">Collection: ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, "<span class=\"badge bg-info text-white\">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, "</span> ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, "</span> ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, "<a href=\"/cluster/ec-shards\" class=\"btn btn-sm btn-outline-secondary ms-2\">Clear Filter</a></div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, "<a href=\"/cluster/ec-shards\" class=\"btn btn-sm btn-outline-secondary ms-2\">Clear Filter</a></div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 25, "</div><div class=\"table-responsive\"><table class=\"table table-striped table-hover\" id=\"ecVolumesTable\"><thead><tr><th><a href=\"#\" onclick=\"sortBy('volume_id')\" class=\"text-dark text-decoration-none\">Volume ID ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, "</div><div class=\"table-responsive\"><table class=\"table table-striped table-hover\" id=\"ecVolumesTable\"><thead><tr><th><a href=\"#\" onclick=\"sortBy('volume_id')\" class=\"text-dark text-decoration-none\">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, "<i class=\"fas fa-sort-up ms-1\"></i>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 30, "<i class=\"fas fa-sort-up ms-1\"></i>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, "<i class=\"fas fa-sort-down ms-1\"></i>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 31, "<i class=\"fas fa-sort-down ms-1\"></i>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, "<i class=\"fas fa-sort ms-1 text-muted\"></i>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 32, "<i class=\"fas fa-sort ms-1 text-muted\"></i>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, "</a></th>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 33, "</a></th>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if data.ShowCollectionColumn {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 30, "<th><a href=\"#\" onclick=\"sortBy('collection')\" class=\"text-dark text-decoration-none\">Collection ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 34, "<th><a href=\"#\" onclick=\"sortBy('collection')\" class=\"text-dark text-decoration-none\">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, "<i class=\"fas fa-sort-up ms-1\"></i>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 35, "<i class=\"fas fa-sort-up ms-1\"></i>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 32, "<i class=\"fas fa-sort-down ms-1\"></i>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 36, "<i class=\"fas fa-sort-down ms-1\"></i>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 33, "<i class=\"fas fa-sort ms-1 text-muted\"></i>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 37, "<i class=\"fas fa-sort ms-1 text-muted\"></i>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 34, "</a></th>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 38, "</a></th>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 35, "<th><a href=\"#\" onclick=\"sortBy('total_shards')\" class=\"text-dark text-decoration-none\">Shard Count ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 39, "<th><a href=\"#\" onclick=\"sortBy('total_shards')\" class=\"text-dark text-decoration-none\">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, "<i class=\"fas fa-sort-up ms-1\"></i>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 40, "<i class=\"fas fa-sort-up ms-1\"></i>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 37, "<i class=\"fas fa-sort-down ms-1\"></i>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 41, "<i class=\"fas fa-sort-down ms-1\"></i>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 38, "<i class=\"fas fa-sort ms-1 text-muted\"></i>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 42, "<i class=\"fas fa-sort ms-1 text-muted\"></i>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 39, "</a></th><th class=\"text-dark\">Shard Size</th><th class=\"text-dark\">Shard Locations</th><th><a href=\"#\" onclick=\"sortBy('completeness')\" class=\"text-dark text-decoration-none\">Status ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 43, "</a></th><th class=\"text-dark\">Shard Size</th><th class=\"text-dark\">Shard Locations</th><th><a href=\"#\" onclick=\"sortBy('completeness')\" class=\"text-dark text-decoration-none\">Status ")
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, "<i class=\"fas fa-sort-up ms-1\"></i>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 44, "<i class=\"fas fa-sort-up ms-1\"></i>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 41, "<i class=\"fas fa-sort-down ms-1\"></i>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 45, "<i class=\"fas fa-sort-down ms-1\"></i>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 42, "<i class=\"fas fa-sort ms-1 text-muted\"></i>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 46, "<i class=\"fas fa-sort ms-1 text-muted\"></i>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 43, "</a></th>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 47, "</a></th>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if data.ShowDataCenterColumn {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 44, "<th class=\"text-dark\">Data Centers</th>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 48, "<th class=\"text-dark\">Data Centers</th>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 45, "<th class=\"text-dark\">Actions</th></tr></thead> <tbody>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 49, "<th class=\"text-dark\">Actions</th></tr></thead> <tbody>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
for _, volume := range data.EcVolumes {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 46, "<tr><td><strong>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 50, "<tr><td><strong>")
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, "</strong></td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 51, "</strong></td>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if data.ShowCollectionColumn {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 48, "<td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 52, "<td>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if volume.Collection != "" {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 49, "<a href=\"/cluster/ec-shards?collection={volume.Collection}\" class=\"text-decoration-none\"><span class=\"badge bg-info text-white\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 53, "<a href=\"/cluster/ec-shards?collection={volume.Collection}\" class=\"text-decoration-none\"><span class=\"badge bg-info text-white\">")
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, "</span></a>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 54, "</span></a>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 51, "<a href=\"/cluster/ec-shards?collection=default\" class=\"text-decoration-none\"><span class=\"badge bg-secondary text-white\">default</span></a>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 55, "<a href=\"/cluster/ec-shards?collection=default\" class=\"text-decoration-none\"><span class=\"badge bg-secondary text-white\">default</span></a>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 52, "</td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 56, "</td>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 53, "<td><span class=\"badge bg-primary\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 57, "<td><span class=\"badge bg-primary\">")
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, "</span></td><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 58, "</span></td><td>")
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, "</td><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 59, "</td><td>")
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, "</td><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 60, "</td><td>")
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, "</td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 61, "</td>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if data.ShowDataCenterColumn {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 58, "<td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 62, "<td>")
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, "<span>, </span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 63, "<span>, </span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 60, " <span class=\"badge bg-primary text-white\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 64, " <span class=\"badge bg-primary text-white\">")
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, "</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 65, "</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 62, "</td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 66, "</td>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 63, "<td><div class=\"btn-group\" role=\"group\"><button type=\"button\" class=\"btn btn-sm btn-outline-primary\" onclick=\"showVolumeDetails(event)\" data-volume-id=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 67, "<td><div class=\"btn-group\" role=\"group\"><button type=\"button\" class=\"btn btn-sm btn-outline-primary\" onclick=\"showVolumeDetails(event)\" data-volume-id=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var15 string
- templ_7745c5c3_Var15, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", volume.VolumeID))
+ var templ_7745c5c3_Var19 string
+ templ_7745c5c3_Var19, 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: 259, Col: 95}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 260, Col: 95}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var15))
+ _, 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, 64, "\" title=\"View EC volume details\"><i class=\"fas fa-info-circle\"></i></button> ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 68, "\" title=\"View EC volume details\"><i class=\"fas fa-info-circle\"></i></button> ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if !volume.IsComplete {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 65, "<button type=\"button\" class=\"btn btn-sm btn-outline-warning\" onclick=\"repairVolume(event)\" data-volume-id=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 69, "<button type=\"button\" class=\"btn btn-sm btn-outline-warning\" onclick=\"repairVolume(event)\" data-volume-id=\"")
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", volume.VolumeID))
+ var templ_7745c5c3_Var20 string
+ templ_7745c5c3_Var20, 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: 266, Col: 99}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 267, Col: 99}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var16))
+ _, 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, 66, "\" title=\"Repair missing shards\"><i class=\"fas fa-wrench\"></i></button>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 70, "\" title=\"Repair missing shards\"><i class=\"fas fa-wrench\"></i></button>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 67, "</div></td></tr>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 71, "</div></td></tr>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 68, "</tbody></table></div><!-- Pagination -->")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 72, "</tbody></table></div><!-- Pagination -->")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if data.TotalPages > 1 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 69, "<nav aria-label=\"EC Volumes pagination\"><ul class=\"pagination justify-content-center\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 73, "<nav aria-label=\"EC Volumes pagination\"><ul class=\"pagination justify-content-center\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if data.Page > 1 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 70, "<li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"1\">First</a></li><li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 74, "<li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"1\">First</a></li><li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var17 string
- templ_7745c5c3_Var17, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.Page-1))
+ var templ_7745c5c3_Var21 string
+ templ_7745c5c3_Var21, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.Page-1))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 288, Col: 126}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 289, Col: 126}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var17))
+ _, 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, 71, "\">Previous</a></li>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 75, "\">Previous</a></li>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
for i := 1; i <= data.TotalPages; i++ {
if i == data.Page {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 72, "<li class=\"page-item active\"><span class=\"page-link\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 76, "<li class=\"page-item active\"><span class=\"page-link\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var18 string
- templ_7745c5c3_Var18, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", i))
+ var templ_7745c5c3_Var22 string
+ templ_7745c5c3_Var22, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", i))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 295, Col: 77}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 296, Col: 77}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var18))
+ _, 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, 73, "</span></li>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 77, "</span></li>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else if i <= 3 || i > data.TotalPages-3 || (i >= data.Page-2 && i <= data.Page+2) {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 74, "<li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 78, "<li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var19 string
- templ_7745c5c3_Var19, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", i))
+ var templ_7745c5c3_Var23 string
+ templ_7745c5c3_Var23, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", i))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 299, Col: 120}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 300, Col: 120}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var19))
+ _, 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, "\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 79, "\">")
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))
+ var templ_7745c5c3_Var24 string
+ templ_7745c5c3_Var24, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", i))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 299, Col: 144}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 300, Col: 144}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var20))
+ _, 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, 76, "</a></li>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 80, "</a></li>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else if i == 4 && data.Page > 6 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 77, "<li class=\"page-item disabled\"><span class=\"page-link\">...</span></li>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 81, "<li class=\"page-item disabled\"><span class=\"page-link\">...</span></li>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else if i == data.TotalPages-3 && data.Page < data.TotalPages-5 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 78, "<li class=\"page-item disabled\"><span class=\"page-link\">...</span></li>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 82, "<li class=\"page-item disabled\"><span class=\"page-link\">...</span></li>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
}
if data.Page < data.TotalPages {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 79, "<li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 83, "<li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var21 string
- templ_7745c5c3_Var21, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.Page+1))
+ var templ_7745c5c3_Var25 string
+ templ_7745c5c3_Var25, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.Page+1))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 314, Col: 126}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 315, Col: 126}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var21))
+ _, 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, 80, "\">Next</a></li><li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 84, "\">Next</a></li><li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var22 string
- templ_7745c5c3_Var22, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.TotalPages))
+ var templ_7745c5c3_Var26 string
+ templ_7745c5c3_Var26, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.TotalPages))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 317, Col: 130}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 318, Col: 130}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var22))
+ _, 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, 81, "\">Last</a></li>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 85, "\">Last</a></li>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 82, "</ul></nav>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 86, "</ul></nav>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 83, "</div><script src=\"https://cdn.jsdelivr.net/npm/bootstrap@5.3.0/dist/js/bootstrap.bundle.min.js\"></script><script>\n // Sorting functionality\n function sortBy(field) {\n const currentSort = new URLSearchParams(window.location.search).get('sort_by');\n const currentOrder = new URLSearchParams(window.location.search).get('sort_order') || 'asc';\n \n let newOrder = 'asc';\n if (currentSort === field && currentOrder === 'asc') {\n newOrder = 'desc';\n }\n \n const url = new URL(window.location);\n url.searchParams.set('sort_by', field);\n url.searchParams.set('sort_order', newOrder);\n url.searchParams.set('page', '1'); // Reset to first page\n window.location.href = url.toString();\n }\n\n // Pagination functionality\n function goToPage(event) {\n event.preventDefault();\n const page = event.target.closest('a').getAttribute('data-page');\n const url = new URL(window.location);\n url.searchParams.set('page', page);\n window.location.href = url.toString();\n }\n\n // Page size functionality\n function changePageSize(newPageSize) {\n const url = new URL(window.location);\n url.searchParams.set('page_size', newPageSize);\n url.searchParams.set('page', '1'); // Reset to first page when changing page size\n window.location.href = url.toString();\n }\n\n // Volume details\n function showVolumeDetails(event) {\n const volumeId = event.target.closest('button').getAttribute('data-volume-id');\n window.location.href = `/cluster/ec-volumes/${volumeId}`;\n }\n\n // Repair volume\n function repairVolume(event) {\n const volumeId = event.target.closest('button').getAttribute('data-volume-id');\n if (confirm(`Are you sure you want to repair missing shards for volume ${volumeId}?`)) {\n // TODO: Implement repair functionality\n alert('Repair functionality will be implemented soon.');\n }\n }\n </script></body></html>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 87, "</div><script src=\"https://cdn.jsdelivr.net/npm/bootstrap@5.3.0/dist/js/bootstrap.bundle.min.js\"></script><script>\n // Sorting functionality\n function sortBy(field) {\n const currentSort = new URLSearchParams(window.location.search).get('sort_by');\n const currentOrder = new URLSearchParams(window.location.search).get('sort_order') || 'asc';\n \n let newOrder = 'asc';\n if (currentSort === field && currentOrder === 'asc') {\n newOrder = 'desc';\n }\n \n const url = new URL(window.location);\n url.searchParams.set('sort_by', field);\n url.searchParams.set('sort_order', newOrder);\n url.searchParams.set('page', '1'); // Reset to first page\n window.location.href = url.toString();\n }\n\n // Pagination functionality\n function goToPage(event) {\n event.preventDefault();\n const page = event.target.closest('a').getAttribute('data-page');\n const url = new URL(window.location);\n url.searchParams.set('page', page);\n window.location.href = url.toString();\n }\n\n // Page size functionality\n function changePageSize(newPageSize) {\n const url = new URL(window.location);\n url.searchParams.set('page_size', newPageSize);\n url.searchParams.set('page', '1'); // Reset to first page when changing page size\n window.location.href = url.toString();\n }\n\n // Volume details\n function showVolumeDetails(event) {\n const volumeId = event.target.closest('button').getAttribute('data-volume-id');\n window.location.href = `/cluster/ec-volumes/${volumeId}`;\n }\n\n // Repair volume\n function repairVolume(event) {\n const volumeId = event.target.closest('button').getAttribute('data-volume-id');\n if (confirm(`Are you sure you want to repair missing shards for volume ${volumeId}?`)) {\n // TODO: Implement repair functionality\n alert('Repair functionality will be implemented soon.');\n }\n }\n </script></body></html>")
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, "<span class=\"text-muted\">No shards</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 88, "<span class=\"text-muted\">No shards</span>")
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, "<br>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 89, "<br>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 86, " <strong><a href=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 90, " <strong><a href=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var24 templ.SafeURL
- templ_7745c5c3_Var24, templ_7745c5c3_Err = templ.JoinURLErrs(templ.URL("/cluster/volume-servers/" + serverInfo.Server))
+ var templ_7745c5c3_Var28 templ.SafeURL
+ templ_7745c5c3_Var28, templ_7745c5c3_Err = templ.JoinURLErrs(templ.URL("/cluster/volume-servers/" + serverInfo.Server))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 390, Col: 71}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_ec_volumes.templ`, Line: 391, Col: 71}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var24))
+ _, 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, 87, "\" class=\"text-primary text-decoration-none\">")
+ 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, "</a>:</strong> ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 92, "</a>:</strong> ")
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, "<span class=\"text-muted\">-</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 93, "<span class=\"text-muted\">-</span>")
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, " <span class=\"text-success\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 94, " <span class=\"text-success\">")
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, "</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 95, "</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 92, " <div class=\"shard-sizes\" style=\"max-width: 300px;\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 96, " <div class=\"shard-sizes\" style=\"max-width: 300px;\">")
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, "</div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 97, "</div>")
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, "<div class=\"small\"><i class=\"fas fa-sitemap me-1\"></i> ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 98, "<div class=\"small\"><i class=\"fas fa-sitemap me-1\"></i> ")
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, "</div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 99, "</div>")
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, "<span class=\"badge bg-success\"><i class=\"fas fa-check me-1\"></i>Complete</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 100, "<span class=\"badge bg-success\"><i class=\"fas fa-check me-1\"></i>Complete</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
- if len(volume.MissingShards) > 10 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 97, "<span class=\"badge bg-danger\"><i class=\"fas fa-skull me-1\"></i>Critical (")
+ if len(volume.MissingShards) > erasure_coding.DataShardsCount {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 101, "<span class=\"badge bg-danger\"><i class=\"fas fa-skull me-1\"></i>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)</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 102, " missing)</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- } else if len(volume.MissingShards) > 6 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 99, "<span class=\"badge bg-warning\"><i class=\"fas fa-exclamation-triangle me-1\"></i>Degraded (")
+ } else if len(volume.MissingShards) > (erasure_coding.DataShardsCount / 2) {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 103, "<span class=\"badge bg-warning\"><i class=\"fas fa-exclamation-triangle me-1\"></i>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)</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 104, " missing)</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- } else if len(volume.MissingShards) > 2 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 101, "<span class=\"badge bg-warning\"><i class=\"fas fa-info-circle me-1\"></i>Incomplete (")
+ } else if len(volume.MissingShards) > (erasure_coding.ParityShardsCount / 2) {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 105, "<span class=\"badge bg-warning\"><i class=\"fas fa-info-circle me-1\"></i>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)</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 106, " missing)</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 103, "<span class=\"badge bg-info\"><i class=\"fas fa-info-circle me-1\"></i>Minor Issues (")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 107, "<span class=\"badge bg-info\"><i class=\"fas fa-info-circle me-1\"></i>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)</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 108, " missing)</span>")
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) {
<table class="table table-hover" id="hostsTable">
<thead>
<tr>
- <th>Server ID</th>
<th>Address</th>
<th>Data Center</th>
<th>Rack</th>
@@ -114,9 +113,6 @@ templ ClusterVolumeServers(data dash.ClusterVolumeServersData) {
for _, host := range data.VolumeServers {
<tr>
<td>
- <code>{host.ID}</code>
- </td>
- <td>
<a href={templ.SafeURL(fmt.Sprintf("http://%s/ui/index.html", host.PublicURL))} target="_blank" class="text-decoration-none">
{host.Address}
<i class="fas fa-external-link-alt ms-1 text-muted"></i>
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, "<div class=\"table-responsive\"><table class=\"table table-hover\" id=\"hostsTable\"><thead><tr><th>Server ID</th><th>Address</th><th>Data Center</th><th>Rack</th><th>Volumes</th><th>Max Volumes</th><th>EC Shards</th><th>Capacity</th><th>Usage</th><th>Actions</th></tr></thead> <tbody>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 5, "<div class=\"table-responsive\"><table class=\"table table-hover\" id=\"hostsTable\"><thead><tr><th>Address</th><th>Data Center</th><th>Rack</th><th>Volumes</th><th>Max Volumes</th><th>EC Shards</th><th>Capacity</th><th>Usage</th><th>Actions</th></tr></thead> <tbody>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
for _, host := range data.VolumeServers {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 6, "<tr><td><code>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 6, "<tr><td><a href=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var5 string
- templ_7745c5c3_Var5, templ_7745c5c3_Err = templ.JoinStringErrs(host.ID)
+ var templ_7745c5c3_Var5 templ.SafeURL
+ templ_7745c5c3_Var5, templ_7745c5c3_Err = templ.JoinURLErrs(templ.SafeURL(fmt.Sprintf("http://%s/ui/index.html", host.PublicURL)))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 117, Col: 58}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 116, Col: 122}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var5))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 7, "</code></td><td><a href=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 7, "\" target=\"_blank\" class=\"text-decoration-none\">")
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, " <i class=\"fas fa-external-link-alt ms-1 text-muted\"></i></a></td><td><span class=\"badge bg-light text-dark\">")
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, " <i class=\"fas fa-external-link-alt ms-1 text-muted\"></i></a></td><td><span class=\"badge bg-light text-dark\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 9, "</span></td><td><span class=\"badge bg-light text-dark\">")
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, "</span></td><td><span class=\"badge bg-light text-dark\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 10, "</span></td><td><div class=\"d-flex align-items-center\"><div class=\"progress me-2\" style=\"width: 60px; height: 16px;\"><div class=\"progress-bar\" role=\"progressbar\" style=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var9 string
- templ_7745c5c3_Var9, templ_7745c5c3_Err = templ.JoinStringErrs(host.Rack)
+ templ_7745c5c3_Var9, templ_7745c5c3_Err = templruntime.SanitizeStyleAttributeValues(fmt.Sprintf("width: %d%%", calculatePercent(host.Volumes, host.MaxVolumes)))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 129, Col: 93}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 131, Col: 139}
}
_, 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, 11, "</span></td><td><div class=\"d-flex align-items-center\"><div class=\"progress me-2\" style=\"width: 60px; height: 16px;\"><div class=\"progress-bar\" role=\"progressbar\" style=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 11, "\"></div></div><span class=\"badge bg-primary\">")
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, "\"></div></div><span class=\"badge bg-primary\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 12, "</span></div></td><td><span class=\"badge bg-secondary\">")
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, "</span></div></td><td><span class=\"badge bg-secondary\">")
- 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, "</span></td><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, "</span></td><td>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if host.EcShards > 0 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "<div class=\"d-flex align-items-center\"><i class=\"fas fa-layer-group me-1 text-info\"></i> <span class=\"badge bg-info text-white me-1\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 14, "<div class=\"d-flex align-items-center\"><i class=\"fas fa-layer-group me-1 text-info\"></i> <span class=\"badge bg-info text-white me-1\">")
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, "</span> <small class=\"text-muted\">shards</small></div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "</span> <small class=\"text-muted\">shards</small></div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if host.EcVolumes > 0 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, "<div class=\"mt-1\"><small class=\"text-muted\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, "<div class=\"mt-1\"><small class=\"text-muted\">")
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, "</small></div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, "</small></div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, "<span class=\"text-muted\">-</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 18, "<span class=\"text-muted\">-</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, "</td><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, "</td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var14 string
+ templ_7745c5c3_Var14, templ_7745c5c3_Err = 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, "</td><td><div class=\"d-flex align-items-center\"><div class=\"progress me-2\" style=\"width: 60px; height: 16px;\"><div class=\"progress-bar\" role=\"progressbar\" style=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var15 string
- templ_7745c5c3_Var15, templ_7745c5c3_Err = templ.JoinStringErrs(formatBytes(host.DiskCapacity))
+ templ_7745c5c3_Var15, templ_7745c5c3_Err = templruntime.SanitizeStyleAttributeValues(fmt.Sprintf("width: %d%%", calculatePercent(int(host.DiskUsage), int(host.DiskCapacity))))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 160, Col: 75}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 161, Col: 153}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var15))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, "</td><td><div class=\"d-flex align-items-center\"><div class=\"progress me-2\" style=\"width: 60px; height: 16px;\"><div class=\"progress-bar\" role=\"progressbar\" style=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, "\"></div></div><small>")
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, "\"></div></div><small>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 22, "</small></div></td><td><button type=\"button\" class=\"btn btn-outline-primary btn-sm\" title=\"View Details\" data-action=\"view-details\" data-id=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var17 string
- templ_7745c5c3_Var17, templ_7745c5c3_Err = templ.JoinStringErrs(formatBytes(host.DiskUsage))
+ templ_7745c5c3_Var17, templ_7745c5c3_Err = templ.JoinStringErrs(host.ID)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 168, Col: 83}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 172, Col: 68}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var17))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 23, "</small></div></td><td><button type=\"button\" class=\"btn btn-outline-primary btn-sm\" title=\"View Details\" data-action=\"view-details\" data-id=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 23, "\" data-address=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var18 string
- templ_7745c5c3_Var18, templ_7745c5c3_Err = templ.JoinStringErrs(host.ID)
+ templ_7745c5c3_Var18, 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: 176, Col: 68}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 173, Col: 78}
}
_, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var18))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, "\" data-address=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, "\" data-public-url=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var19 string
- templ_7745c5c3_Var19, templ_7745c5c3_Err = templ.JoinStringErrs(host.Address)
+ templ_7745c5c3_Var19, templ_7745c5c3_Err = templ.JoinStringErrs(host.PublicURL)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 177, Col: 78}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 174, Col: 83}
}
_, 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, 25, "\" data-public-url=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 25, "\" data-datacenter=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var20 string
- templ_7745c5c3_Var20, templ_7745c5c3_Err = templ.JoinStringErrs(host.PublicURL)
+ templ_7745c5c3_Var20, 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: 178, Col: 83}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 175, Col: 84}
}
_, 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, 26, "\" data-datacenter=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, "\" data-rack=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var21 string
- templ_7745c5c3_Var21, templ_7745c5c3_Err = templ.JoinStringErrs(host.DataCenter)
+ templ_7745c5c3_Var21, 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: 179, Col: 84}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 176, Col: 72}
}
_, 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, 27, "\" data-rack=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, "\" data-volumes=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var22 string
- templ_7745c5c3_Var22, templ_7745c5c3_Err = templ.JoinStringErrs(host.Rack)
+ templ_7745c5c3_Var22, 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: 180, Col: 72}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 177, Col: 97}
}
_, 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, 28, "\" data-volumes=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, "\" data-max-volumes=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var23 string
- templ_7745c5c3_Var23, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", host.Volumes))
+ templ_7745c5c3_Var23, 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: 181, Col: 97}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 178, Col: 104}
}
_, 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, 29, "\" data-max-volumes=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, "\" data-disk-usage=\"")
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", host.MaxVolumes))
+ templ_7745c5c3_Var24, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", host.DiskUsage))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 182, Col: 104}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 179, Col: 102}
}
_, 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, 30, "\" data-disk-usage=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 30, "\" data-disk-capacity=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var25 string
- templ_7745c5c3_Var25, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", host.DiskUsage))
+ templ_7745c5c3_Var25, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", host.DiskCapacity))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 183, Col: 102}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 180, Col: 108}
}
_, 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, 31, "\" data-disk-capacity=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 31, "\" data-ec-volumes=\"")
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", host.DiskCapacity))
+ templ_7745c5c3_Var26, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", host.EcVolumes))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 184, Col: 108}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 181, Col: 102}
}
_, 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, 32, "\" data-ec-volumes=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 32, "\" data-ec-shards=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var27 string
- templ_7745c5c3_Var27, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", host.EcVolumes))
+ templ_7745c5c3_Var27, 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: 185, Col: 102}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 182, Col: 100}
}
_, 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, 33, "\" data-ec-shards=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 33, "\" data-last-heartbeat=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
var templ_7745c5c3_Var28 string
- templ_7745c5c3_Var28, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", host.EcShards))
+ templ_7745c5c3_Var28, templ_7745c5c3_Err = templ.JoinStringErrs(host.LastHeartbeat.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: 186, Col: 100}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/cluster_volume_servers.templ`, Line: 183, Col: 121}
}
_, 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, 34, "\" data-last-heartbeat=\"")
- if templ_7745c5c3_Err != nil {
- return templ_7745c5c3_Err
- }
- var templ_7745c5c3_Var29 string
- templ_7745c5c3_Var29, templ_7745c5c3_Err = templ.JoinStringErrs(host.LastHeartbeat.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: 187, Col: 121}
- }
- _, 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, 35, "\"><i class=\"fas fa-eye\"></i></button></td></tr>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 34, "\"><i class=\"fas fa-eye\"></i></button></td></tr>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 36, "</tbody></table></div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 35, "</tbody></table></div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 37, "<div class=\"text-center py-5\"><i class=\"fas fa-server fa-3x text-muted mb-3\"></i><h5 class=\"text-muted\">No Volume Servers Found</h5><p class=\"text-muted\">No volume servers are currently available in the cluster.</p></div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 36, "<div class=\"text-center py-5\"><i class=\"fas fa-server fa-3x text-muted mb-3\"></i><h5 class=\"text-muted\">No Volume Servers Found</h5><p class=\"text-muted\">No volume servers are currently available in the cluster.</p></div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 38, "</div></div><!-- Last Updated --><div class=\"row\"><div class=\"col-12\"><small class=\"text-muted\"><i class=\"fas fa-clock me-1\"></i> Last updated: ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 37, "</div></div><!-- Last Updated --><div class=\"row\"><div class=\"col-12\"><small class=\"text-muted\"><i class=\"fas fa-clock me-1\"></i> Last updated: ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_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, "</small></div></div></div><!-- JavaScript for cluster volume servers functionality --><script>\n document.addEventListener('DOMContentLoaded', function() {\n // Handle volume server action buttons\n document.addEventListener('click', function(e) {\n const button = e.target.closest('[data-action]');\n if (!button) return;\n \n const action = button.getAttribute('data-action');\n \n switch(action) {\n case 'view-details':\n const serverData = {\n id: button.getAttribute('data-id'),\n address: button.getAttribute('data-address'),\n publicUrl: button.getAttribute('data-public-url'),\n datacenter: button.getAttribute('data-datacenter'),\n rack: button.getAttribute('data-rack'),\n volumes: parseInt(button.getAttribute('data-volumes')),\n maxVolumes: parseInt(button.getAttribute('data-max-volumes')),\n diskUsage: parseInt(button.getAttribute('data-disk-usage')),\n diskCapacity: parseInt(button.getAttribute('data-disk-capacity')),\n ecVolumes: parseInt(button.getAttribute('data-ec-volumes')),\n ecShards: parseInt(button.getAttribute('data-ec-shards')),\n lastHeartbeat: button.getAttribute('data-last-heartbeat')\n };\n showVolumeServerDetails(serverData);\n break;\n }\n });\n });\n \n function showVolumeServerDetails(server) {\n const volumePercent = server.maxVolumes > 0 ? Math.round((server.volumes / server.maxVolumes) * 100) : 0;\n const diskPercent = server.diskCapacity > 0 ? Math.round((server.diskUsage / server.diskCapacity) * 100) : 0;\n \n const modalHtml = '<div class=\"modal fade\" id=\"volumeServerDetailsModal\" tabindex=\"-1\">' +\n '<div class=\"modal-dialog modal-lg\">' +\n '<div class=\"modal-content\">' +\n '<div class=\"modal-header\">' +\n '<h5 class=\"modal-title\"><i class=\"fas fa-server me-2\"></i>Volume Server Details: ' + server.address + '</h5>' +\n '<button type=\"button\" class=\"btn-close\" data-bs-dismiss=\"modal\"></button>' +\n '</div>' +\n '<div class=\"modal-body\">' +\n '<div class=\"row\">' +\n '<div class=\"col-md-6\">' +\n '<h6 class=\"text-primary\"><i class=\"fas fa-info-circle me-1\"></i>Basic Information</h6>' +\n '<table class=\"table table-sm\">' +\n '<tr><td><strong>Server ID:</strong></td><td><code>' + server.id + '</code></td></tr>' +\n '<tr><td><strong>Address:</strong></td><td>' + server.address + '</td></tr>' +\n '<tr><td><strong>Public URL:</strong></td><td>' + server.publicUrl + '</td></tr>' +\n '<tr><td><strong>Data Center:</strong></td><td><span class=\"badge bg-light text-dark\">' + server.datacenter + '</span></td></tr>' +\n '<tr><td><strong>Rack:</strong></td><td><span class=\"badge bg-light text-dark\">' + server.rack + '</span></td></tr>' +\n '<tr><td><strong>Last Heartbeat:</strong></td><td>' + server.lastHeartbeat + '</td></tr>' +\n '</table>' +\n '</div>' +\n '<div class=\"col-md-6\">' +\n '<h6 class=\"text-primary\"><i class=\"fas fa-chart-bar me-1\"></i>Usage Statistics</h6>' +\n '<table class=\"table table-sm\">' +\n '<tr><td><strong>Volumes:</strong></td><td>' +\n '<div class=\"d-flex align-items-center\">' +\n '<div class=\"progress me-2\" style=\"width: 100px; height: 20px;\">' +\n '<div class=\"progress-bar\" role=\"progressbar\" style=\"width: ' + volumePercent + '%\"></div>' +\n '</div>' +\n '<span>' + server.volumes + '/' + server.maxVolumes + ' (' + volumePercent + '%)</span>' +\n '</div>' +\n '</td></tr>' +\n '<tr><td><strong>Disk Usage:</strong></td><td>' +\n '<div class=\"d-flex align-items-center\">' +\n '<div class=\"progress me-2\" style=\"width: 100px; height: 20px;\">' +\n '<div class=\"progress-bar\" role=\"progressbar\" style=\"width: ' + diskPercent + '%\"></div>' +\n '</div>' +\n '<span>' + formatBytes(server.diskUsage) + '/' + formatBytes(server.diskCapacity) + ' (' + diskPercent + '%)</span>' +\n '</div>' +\n '</td></tr>' +\n '<tr><td><strong>Available Space:</strong></td><td>' + formatBytes(server.diskCapacity - server.diskUsage) + '</td></tr>' +\n '</table>' +\n '</div>' +\n '</div>' +\n \n // Add EC Shard information if available\n (server.ecShards > 0 ? \n '<div class=\"row mt-3\">' +\n '<div class=\"col-12\">' +\n '<h6 class=\"text-primary\"><i class=\"fas fa-layer-group me-1\"></i>Erasure Coding Information</h6>' +\n '<table class=\"table table-sm\">' +\n '<tr><td><strong>EC Volumes:</strong></td><td><span class=\"badge bg-info text-white\">' + server.ecVolumes + '</span></td></tr>' +\n '<tr><td><strong>EC Shards:</strong></td><td><span class=\"badge bg-info text-white\">' + server.ecShards + '</span></td></tr>' +\n '</table>' +\n '</div>' +\n '</div>' : '') +\n \n '<div class=\"row mt-3\">' +\n '<div class=\"col-12\">' +\n '<h6 class=\"text-primary\"><i class=\"fas fa-link me-1\"></i>Quick Actions</h6>' +\n '<div class=\"d-grid gap-2 d-md-flex\">' +\n '<a href=\"http://' + server.publicUrl + '/ui/index.html\" target=\"_blank\" class=\"btn btn-outline-primary\">' +\n '<i class=\"fas fa-external-link-alt me-1\"></i>Open Volume Server UI' +\n '</a>' +\n '<a href=\"/cluster/volumes?server=' + encodeURIComponent(server.address) + '\" class=\"btn btn-outline-info\">' +\n '<i class=\"fas fa-database me-1\"></i>View Volumes' +\n '</a>' +\n '</div>' +\n '</div>' +\n '</div>' +\n '</div>' +\n '<div class=\"modal-footer\">' +\n '<button type=\"button\" class=\"btn btn-secondary\" data-bs-dismiss=\"modal\">Close</button>' +\n '</div>' +\n '</div>' +\n '</div>' +\n '</div>';\n \n // Remove existing modal if present\n const existingModal = document.getElementById('volumeServerDetailsModal');\n if (existingModal) {\n existingModal.remove();\n }\n \n // Add modal to body and show\n document.body.insertAdjacentHTML('beforeend', modalHtml);\n const modal = new bootstrap.Modal(document.getElementById('volumeServerDetailsModal'));\n modal.show();\n \n // Remove modal when hidden\n document.getElementById('volumeServerDetailsModal').addEventListener('hidden.bs.modal', function() {\n this.remove();\n });\n }\n \n function formatBytes(bytes) {\n if (bytes === 0) return '0 Bytes';\n const k = 1024;\n const sizes = ['Bytes', 'KB', 'MB', 'GB', 'TB'];\n const i = Math.floor(Math.log(bytes) / Math.log(k));\n return parseFloat((bytes / Math.pow(k, i)).toFixed(2)) + ' ' + sizes[i];\n }\n \n function exportVolumeServers() {\n // Simple CSV export of volume servers list\n const rows = Array.from(document.querySelectorAll('#hostsTable tbody tr')).map(row => {\n const cells = row.querySelectorAll('td');\n if (cells.length > 1) {\n return {\n id: cells[0].textContent.trim(),\n address: cells[1].textContent.trim(),\n datacenter: cells[2].textContent.trim(),\n rack: cells[3].textContent.trim(),\n volumes: cells[4].textContent.trim(),\n capacity: cells[5].textContent.trim(),\n usage: cells[6].textContent.trim()\n };\n }\n return null;\n }).filter(row => row !== null);\n \n const csvContent = \"data:text/csv;charset=utf-8,\" + \n \"Server ID,Address,Data Center,Rack,Volumes,Capacity,Usage\\n\" +\n rows.map(r => '\"' + r.id + '\",\"' + r.address + '\",\"' + r.datacenter + '\",\"' + r.rack + '\",\"' + r.volumes + '\",\"' + r.capacity + '\",\"' + r.usage + '\"').join(\"\\n\");\n \n const encodedUri = encodeURI(csvContent);\n const link = document.createElement(\"a\");\n link.setAttribute(\"href\", encodedUri);\n link.setAttribute(\"download\", \"volume_servers.csv\");\n document.body.appendChild(link);\n link.click();\n document.body.removeChild(link);\n }\n </script>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 38, "</small></div></div></div><!-- JavaScript for cluster volume servers functionality --><script>\n document.addEventListener('DOMContentLoaded', function() {\n // Handle volume server action buttons\n document.addEventListener('click', function(e) {\n const button = e.target.closest('[data-action]');\n if (!button) return;\n \n const action = button.getAttribute('data-action');\n \n switch(action) {\n case 'view-details':\n const serverData = {\n id: button.getAttribute('data-id'),\n address: button.getAttribute('data-address'),\n publicUrl: button.getAttribute('data-public-url'),\n datacenter: button.getAttribute('data-datacenter'),\n rack: button.getAttribute('data-rack'),\n volumes: parseInt(button.getAttribute('data-volumes')),\n maxVolumes: parseInt(button.getAttribute('data-max-volumes')),\n diskUsage: parseInt(button.getAttribute('data-disk-usage')),\n diskCapacity: parseInt(button.getAttribute('data-disk-capacity')),\n ecVolumes: parseInt(button.getAttribute('data-ec-volumes')),\n ecShards: parseInt(button.getAttribute('data-ec-shards')),\n lastHeartbeat: button.getAttribute('data-last-heartbeat')\n };\n showVolumeServerDetails(serverData);\n break;\n }\n });\n });\n \n function showVolumeServerDetails(server) {\n const volumePercent = server.maxVolumes > 0 ? Math.round((server.volumes / server.maxVolumes) * 100) : 0;\n const diskPercent = server.diskCapacity > 0 ? Math.round((server.diskUsage / server.diskCapacity) * 100) : 0;\n \n const modalHtml = '<div class=\"modal fade\" id=\"volumeServerDetailsModal\" tabindex=\"-1\">' +\n '<div class=\"modal-dialog modal-lg\">' +\n '<div class=\"modal-content\">' +\n '<div class=\"modal-header\">' +\n '<h5 class=\"modal-title\"><i class=\"fas fa-server me-2\"></i>Volume Server Details: ' + server.address + '</h5>' +\n '<button type=\"button\" class=\"btn-close\" data-bs-dismiss=\"modal\"></button>' +\n '</div>' +\n '<div class=\"modal-body\">' +\n '<div class=\"row\">' +\n '<div class=\"col-md-6\">' +\n '<h6 class=\"text-primary\"><i class=\"fas fa-info-circle me-1\"></i>Basic Information</h6>' +\n '<table class=\"table table-sm\">' +\n '<tr><td><strong>Server ID:</strong></td><td><code>' + server.id + '</code></td></tr>' +\n '<tr><td><strong>Address:</strong></td><td>' + server.address + '</td></tr>' +\n '<tr><td><strong>Public URL:</strong></td><td>' + server.publicUrl + '</td></tr>' +\n '<tr><td><strong>Data Center:</strong></td><td><span class=\"badge bg-light text-dark\">' + server.datacenter + '</span></td></tr>' +\n '<tr><td><strong>Rack:</strong></td><td><span class=\"badge bg-light text-dark\">' + server.rack + '</span></td></tr>' +\n '<tr><td><strong>Last Heartbeat:</strong></td><td>' + server.lastHeartbeat + '</td></tr>' +\n '</table>' +\n '</div>' +\n '<div class=\"col-md-6\">' +\n '<h6 class=\"text-primary\"><i class=\"fas fa-chart-bar me-1\"></i>Usage Statistics</h6>' +\n '<table class=\"table table-sm\">' +\n '<tr><td><strong>Volumes:</strong></td><td>' +\n '<div class=\"d-flex align-items-center\">' +\n '<div class=\"progress me-2\" style=\"width: 100px; height: 20px;\">' +\n '<div class=\"progress-bar\" role=\"progressbar\" style=\"width: ' + volumePercent + '%\"></div>' +\n '</div>' +\n '<span>' + server.volumes + '/' + server.maxVolumes + ' (' + volumePercent + '%)</span>' +\n '</div>' +\n '</td></tr>' +\n '<tr><td><strong>Disk Usage:</strong></td><td>' +\n '<div class=\"d-flex align-items-center\">' +\n '<div class=\"progress me-2\" style=\"width: 100px; height: 20px;\">' +\n '<div class=\"progress-bar\" role=\"progressbar\" style=\"width: ' + diskPercent + '%\"></div>' +\n '</div>' +\n '<span>' + formatBytes(server.diskUsage) + '/' + formatBytes(server.diskCapacity) + ' (' + diskPercent + '%)</span>' +\n '</div>' +\n '</td></tr>' +\n '<tr><td><strong>Available Space:</strong></td><td>' + formatBytes(server.diskCapacity - server.diskUsage) + '</td></tr>' +\n '</table>' +\n '</div>' +\n '</div>' +\n \n // Add EC Shard information if available\n (server.ecShards > 0 ? \n '<div class=\"row mt-3\">' +\n '<div class=\"col-12\">' +\n '<h6 class=\"text-primary\"><i class=\"fas fa-layer-group me-1\"></i>Erasure Coding Information</h6>' +\n '<table class=\"table table-sm\">' +\n '<tr><td><strong>EC Volumes:</strong></td><td><span class=\"badge bg-info text-white\">' + server.ecVolumes + '</span></td></tr>' +\n '<tr><td><strong>EC Shards:</strong></td><td><span class=\"badge bg-info text-white\">' + server.ecShards + '</span></td></tr>' +\n '</table>' +\n '</div>' +\n '</div>' : '') +\n \n '<div class=\"row mt-3\">' +\n '<div class=\"col-12\">' +\n '<h6 class=\"text-primary\"><i class=\"fas fa-link me-1\"></i>Quick Actions</h6>' +\n '<div class=\"d-grid gap-2 d-md-flex\">' +\n '<a href=\"http://' + server.publicUrl + '/ui/index.html\" target=\"_blank\" class=\"btn btn-outline-primary\">' +\n '<i class=\"fas fa-external-link-alt me-1\"></i>Open Volume Server UI' +\n '</a>' +\n '<a href=\"/cluster/volumes?server=' + encodeURIComponent(server.address) + '\" class=\"btn btn-outline-info\">' +\n '<i class=\"fas fa-database me-1\"></i>View Volumes' +\n '</a>' +\n '</div>' +\n '</div>' +\n '</div>' +\n '</div>' +\n '<div class=\"modal-footer\">' +\n '<button type=\"button\" class=\"btn btn-secondary\" data-bs-dismiss=\"modal\">Close</button>' +\n '</div>' +\n '</div>' +\n '</div>' +\n '</div>';\n \n // Remove existing modal if present\n const existingModal = document.getElementById('volumeServerDetailsModal');\n if (existingModal) {\n existingModal.remove();\n }\n \n // Add modal to body and show\n document.body.insertAdjacentHTML('beforeend', modalHtml);\n const modal = new bootstrap.Modal(document.getElementById('volumeServerDetailsModal'));\n modal.show();\n \n // Remove modal when hidden\n document.getElementById('volumeServerDetailsModal').addEventListener('hidden.bs.modal', function() {\n this.remove();\n });\n }\n \n function formatBytes(bytes) {\n if (bytes === 0) return '0 Bytes';\n const k = 1024;\n const sizes = ['Bytes', 'KB', 'MB', 'GB', 'TB'];\n const i = Math.floor(Math.log(bytes) / Math.log(k));\n return parseFloat((bytes / Math.pow(k, i)).toFixed(2)) + ' ' + sizes[i];\n }\n \n function exportVolumeServers() {\n // Simple CSV export of volume servers list\n const rows = Array.from(document.querySelectorAll('#hostsTable tbody tr')).map(row => {\n const cells = row.querySelectorAll('td');\n if (cells.length > 1) {\n return {\n id: cells[0].textContent.trim(),\n address: cells[1].textContent.trim(),\n datacenter: cells[2].textContent.trim(),\n rack: cells[3].textContent.trim(),\n volumes: cells[4].textContent.trim(),\n capacity: cells[5].textContent.trim(),\n usage: cells[6].textContent.trim()\n };\n }\n return null;\n }).filter(row => row !== null);\n \n const csvContent = \"data:text/csv;charset=utf-8,\" + \n \"Server ID,Address,Data Center,Rack,Volumes,Capacity,Usage\\n\" +\n rows.map(r => '\"' + r.id + '\",\"' + r.address + '\",\"' + r.datacenter + '\",\"' + r.rack + '\",\"' + r.volumes + '\",\"' + r.capacity + '\",\"' + r.usage + '\"').join(\"\\n\");\n \n const encodedUri = encodeURI(csvContent);\n const link = document.createElement(\"a\");\n link.setAttribute(\"href\", encodedUri);\n link.setAttribute(\"download\", \"volume_servers.csv\");\n document.body.appendChild(link);\n link.click();\n document.body.removeChild(link);\n }\n </script>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
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) {
</td>
</tr>
}
+
+ // Show message when no volumes found
+ if len(data.RegularVolumes) == 0 && len(data.EcVolumes) == 0 {
+ <tr>
+ <td colspan="6" class="text-center text-muted py-4">
+ <i class="fas fa-info-circle me-2"></i>
+ No volumes found for collection "{data.CollectionName}"
+ </td>
+ </tr>
+ }
</tbody>
</table>
</div>
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, "</tbody></table></div><!-- Pagination -->")
+ if len(data.RegularVolumes) == 0 && len(data.EcVolumes) == 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 47, "<tr><td colspan=\"6\" class=\"text-center text-muted py-4\"><i class=\"fas fa-info-circle me-2\"></i> No volumes found for collection \"")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var21 string
+ templ_7745c5c3_Var21, templ_7745c5c3_Err = templ.JoinStringErrs(data.CollectionName)
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 271, Col: 60}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var21))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 48, "\"</td></tr>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 49, "</tbody></table></div><!-- Pagination -->")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if data.TotalPages > 1 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 48, "<nav aria-label=\"Collection volumes pagination\"><ul class=\"pagination justify-content-center\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 50, "<nav aria-label=\"Collection volumes pagination\"><ul class=\"pagination justify-content-center\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if data.Page > 1 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 49, "<li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"1\">First</a></li><li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 51, "<li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"1\">First</a></li><li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var21 string
- templ_7745c5c3_Var21, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.Page-1))
+ var templ_7745c5c3_Var22 string
+ templ_7745c5c3_Var22, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.Page-1))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 278, Col: 104}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 288, Col: 104}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var21))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var22))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 50, "\">Previous</a></li>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 52, "\">Previous</a></li>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
for i := 1; i <= data.TotalPages; i++ {
if i == data.Page {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 51, "<li class=\"page-item active\"><span class=\"page-link\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 53, "<li class=\"page-item active\"><span class=\"page-link\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var22 string
- templ_7745c5c3_Var22, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", i))
+ var templ_7745c5c3_Var23 string
+ templ_7745c5c3_Var23, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", i))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 285, Col: 52}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 295, Col: 52}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var22))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var23))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 52, "</span></li>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 54, "</span></li>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else if i <= 3 || i > data.TotalPages-3 || (i >= data.Page-2 && i <= data.Page+2) {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 53, "<li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 55, "<li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var23 string
- templ_7745c5c3_Var23, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", i))
+ var templ_7745c5c3_Var24 string
+ templ_7745c5c3_Var24, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", i))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 289, Col: 95}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 299, Col: 95}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var23))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var24))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 54, "\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 56, "\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var24 string
- templ_7745c5c3_Var24, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", i))
+ var templ_7745c5c3_Var25 string
+ templ_7745c5c3_Var25, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", i))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 289, Col: 119}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 299, Col: 119}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var24))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var25))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 55, "</a></li>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 57, "</a></li>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else if i == 4 && data.Page > 6 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 56, "<li class=\"page-item disabled\"><span class=\"page-link\">...</span></li>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 58, "<li class=\"page-item disabled\"><span class=\"page-link\">...</span></li>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else if i == data.TotalPages-3 && data.Page < data.TotalPages-5 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 57, "<li class=\"page-item disabled\"><span class=\"page-link\">...</span></li>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 59, "<li class=\"page-item disabled\"><span class=\"page-link\">...</span></li>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
}
if data.Page < data.TotalPages {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 58, "<li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 60, "<li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var25 string
- templ_7745c5c3_Var25, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.Page+1))
+ var templ_7745c5c3_Var26 string
+ templ_7745c5c3_Var26, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.Page+1))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 304, Col: 104}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 314, Col: 104}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var25))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var26))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 59, "\">Next</a></li><li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 61, "\">Next</a></li><li class=\"page-item\"><a class=\"page-link\" href=\"#\" onclick=\"goToPage(event)\" data-page=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var26 string
- templ_7745c5c3_Var26, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.TotalPages))
+ var templ_7745c5c3_Var27 string
+ templ_7745c5c3_Var27, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", data.TotalPages))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 307, Col: 108}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/collection_details.templ`, Line: 317, Col: 108}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var26))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var27))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 60, "\">Last</a></li>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 62, "\">Last</a></li>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 61, "</ul></nav>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 63, "</ul></nav>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 62, "<script>\n\t\t// Sorting functionality\n\t\tfunction sortBy(field) {\n\t\t\tconst currentSort = new URLSearchParams(window.location.search).get('sort_by');\n\t\t\tconst currentOrder = new URLSearchParams(window.location.search).get('sort_order') || 'asc';\n\t\t\t\n\t\t\tlet newOrder = 'asc';\n\t\t\tif (currentSort === field && currentOrder === 'asc') {\n\t\t\t\tnewOrder = 'desc';\n\t\t\t}\n\t\t\t\n\t\t\tconst url = new URL(window.location);\n\t\t\turl.searchParams.set('sort_by', field);\n\t\t\turl.searchParams.set('sort_order', newOrder);\n\t\t\turl.searchParams.set('page', '1'); // Reset to first page\n\t\t\twindow.location.href = url.toString();\n\t\t}\n\n\t\t// Pagination functionality\n\t\tfunction goToPage(event) {\n\t\t\tevent.preventDefault();\n\t\t\tconst page = event.target.closest('a').getAttribute('data-page');\n\t\t\tconst url = new URL(window.location);\n\t\t\turl.searchParams.set('page', page);\n\t\t\twindow.location.href = url.toString();\n\t\t}\n\n\t\t// Page size functionality\n\t\tfunction changePageSize(newPageSize) {\n\t\t\tconst url = new URL(window.location);\n\t\t\turl.searchParams.set('page_size', newPageSize);\n\t\t\turl.searchParams.set('page', '1'); // Reset to first page when changing page size\n\t\t\twindow.location.href = url.toString();\n\t\t}\n\n\t\t// Volume details\n\t\tfunction showVolumeDetails(event) {\n\t\t\tconst volumeId = event.target.closest('button').getAttribute('data-volume-id');\n\t\t\tconst server = event.target.closest('button').getAttribute('data-server');\n\t\t\twindow.location.href = `/cluster/volumes/${volumeId}/${server}`;\n\t\t}\n\n\t\t// EC Volume details\n\t\tfunction showEcVolumeDetails(event) {\n\t\t\tconst volumeId = event.target.closest('button').getAttribute('data-volume-id');\n\t\t\twindow.location.href = `/cluster/ec-volumes/${volumeId}`;\n\t\t}\n\n\t\t// Repair EC Volume\n\t\tfunction repairEcVolume(event) {\n\t\t\tconst volumeId = event.target.closest('button').getAttribute('data-volume-id');\n\t\t\tif (confirm(`Are you sure you want to repair missing shards for EC volume ${volumeId}?`)) {\n\t\t\t\t// TODO: Implement repair functionality\n\t\t\t\talert('Repair functionality will be implemented soon.');\n\t\t\t}\n\t\t}\n\t</script>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 64, "<script>\n\t\t// Sorting functionality\n\t\tfunction sortBy(field) {\n\t\t\tconst currentSort = new URLSearchParams(window.location.search).get('sort_by');\n\t\t\tconst currentOrder = new URLSearchParams(window.location.search).get('sort_order') || 'asc';\n\t\t\t\n\t\t\tlet newOrder = 'asc';\n\t\t\tif (currentSort === field && currentOrder === 'asc') {\n\t\t\t\tnewOrder = 'desc';\n\t\t\t}\n\t\t\t\n\t\t\tconst url = new URL(window.location);\n\t\t\turl.searchParams.set('sort_by', field);\n\t\t\turl.searchParams.set('sort_order', newOrder);\n\t\t\turl.searchParams.set('page', '1'); // Reset to first page\n\t\t\twindow.location.href = url.toString();\n\t\t}\n\n\t\t// Pagination functionality\n\t\tfunction goToPage(event) {\n\t\t\tevent.preventDefault();\n\t\t\tconst page = event.target.closest('a').getAttribute('data-page');\n\t\t\tconst url = new URL(window.location);\n\t\t\turl.searchParams.set('page', page);\n\t\t\twindow.location.href = url.toString();\n\t\t}\n\n\t\t// Page size functionality\n\t\tfunction changePageSize(newPageSize) {\n\t\t\tconst url = new URL(window.location);\n\t\t\turl.searchParams.set('page_size', newPageSize);\n\t\t\turl.searchParams.set('page', '1'); // Reset to first page when changing page size\n\t\t\twindow.location.href = url.toString();\n\t\t}\n\n\t\t// Volume details\n\t\tfunction showVolumeDetails(event) {\n\t\t\tconst volumeId = event.target.closest('button').getAttribute('data-volume-id');\n\t\t\tconst server = event.target.closest('button').getAttribute('data-server');\n\t\t\twindow.location.href = `/cluster/volumes/${volumeId}/${server}`;\n\t\t}\n\n\t\t// EC Volume details\n\t\tfunction showEcVolumeDetails(event) {\n\t\t\tconst volumeId = event.target.closest('button').getAttribute('data-volume-id');\n\t\t\twindow.location.href = `/cluster/ec-volumes/${volumeId}`;\n\t\t}\n\n\t\t// Repair EC Volume\n\t\tfunction repairEcVolume(event) {\n\t\t\tconst volumeId = event.target.closest('button').getAttribute('data-volume-id');\n\t\t\tif (confirm(`Are you sure you want to repair missing shards for EC volume ${volumeId}?`)) {\n\t\t\t\t// TODO: Implement repair functionality\n\t\t\t\talert('Repair functionality will be implemented soon.');\n\t\t\t}\n\t\t}\n\t</script>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
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) {
</div>
</div>
+ <!-- Completed Tasks -->
+ <div class="row mb-4">
+ <div class="col-12">
+ <div class="card">
+ <div class="card-header bg-success text-white">
+ <h5 class="mb-0">
+ <i class="fas fa-check-circle me-2"></i>
+ Completed Tasks
+ </h5>
+ </div>
+ <div class="card-body">
+ if data.Stats.CompletedToday == 0 && data.Stats.FailedToday == 0 {
+ <div class="text-center text-muted py-4">
+ <i class="fas fa-check-circle fa-3x mb-3"></i>
+ <p>No completed maintenance tasks today</p>
+ <small>Completed tasks will appear here after workers finish processing them</small>
+ </div>
+ } else {
+ <div class="table-responsive">
+ <table class="table table-hover">
+ <thead>
+ <tr>
+ <th>Type</th>
+ <th>Status</th>
+ <th>Volume</th>
+ <th>Worker</th>
+ <th>Duration</th>
+ <th>Completed</th>
+ </tr>
+ </thead>
+ <tbody>
+ for _, task := range data.Tasks {
+ if string(task.Status) == "completed" || string(task.Status) == "failed" || string(task.Status) == "cancelled" {
+ if string(task.Status) == "failed" {
+ <tr class="table-danger clickable-row" data-task-id={task.ID} onclick="navigateToTask(this)" style="cursor: pointer;">
+ <td>
+ @TaskTypeIcon(task.Type)
+ {string(task.Type)}
+ </td>
+ <td>@StatusBadge(task.Status)</td>
+ <td>{fmt.Sprintf("%d", task.VolumeID)}</td>
+ <td>
+ if task.WorkerID != "" {
+ <small>{task.WorkerID}</small>
+ } else {
+ <span class="text-muted">-</span>
+ }
+ </td>
+ <td>
+ if task.StartedAt != nil && task.CompletedAt != nil {
+ {formatDuration(task.CompletedAt.Sub(*task.StartedAt))}
+ } else {
+ <span class="text-muted">-</span>
+ }
+ </td>
+ <td>
+ if task.CompletedAt != nil {
+ {task.CompletedAt.Format("2006-01-02 15:04")}
+ } else {
+ <span class="text-muted">-</span>
+ }
+ </td>
+ </tr>
+ } else {
+ <tr class="clickable-row" data-task-id={task.ID} onclick="navigateToTask(this)" style="cursor: pointer;">
+ <td>
+ @TaskTypeIcon(task.Type)
+ {string(task.Type)}
+ </td>
+ <td>@StatusBadge(task.Status)</td>
+ <td>{fmt.Sprintf("%d", task.VolumeID)}</td>
+ <td>
+ if task.WorkerID != "" {
+ <small>{task.WorkerID}</small>
+ } else {
+ <span class="text-muted">-</span>
+ }
+ </td>
+ <td>
+ if task.StartedAt != nil && task.CompletedAt != nil {
+ {formatDuration(task.CompletedAt.Sub(*task.StartedAt))}
+ } else {
+ <span class="text-muted">-</span>
+ }
+ </td>
+ <td>
+ if task.CompletedAt != nil {
+ {task.CompletedAt.Format("2006-01-02 15:04")}
+ } else {
+ <span class="text-muted">-</span>
+ }
+ </td>
+ </tr>
+ }
+ }
+ }
+ </tbody>
+ </table>
+ </div>
+ }
+ </div>
+ </div>
+ </div>
+ </div>
+
<!-- Pending Tasks -->
<div class="row mb-4">
<div class="col-12">
@@ -103,7 +208,7 @@ templ MaintenanceQueue(data *maintenance.MaintenanceQueueData) {
<tbody>
for _, task := range data.Tasks {
if string(task.Status) == "pending" {
- <tr>
+ <tr class="clickable-row" data-task-id={task.ID} onclick="navigateToTask(this)" style="cursor: pointer;">
<td>
@TaskTypeIcon(task.Type)
{string(task.Type)}
@@ -158,7 +263,7 @@ templ MaintenanceQueue(data *maintenance.MaintenanceQueueData) {
<tbody>
for _, task := range data.Tasks {
if string(task.Status) == "assigned" || string(task.Status) == "in_progress" {
- <tr>
+ <tr class="clickable-row" data-task-id={task.ID} onclick="navigateToTask(this)" style="cursor: pointer;">
<td>
@TaskTypeIcon(task.Type)
{string(task.Type)}
@@ -191,111 +296,6 @@ templ MaintenanceQueue(data *maintenance.MaintenanceQueueData) {
</div>
</div>
</div>
-
- <!-- Completed Tasks -->
- <div class="row mb-4">
- <div class="col-12">
- <div class="card">
- <div class="card-header bg-success text-white">
- <h5 class="mb-0">
- <i class="fas fa-check-circle me-2"></i>
- Completed Tasks
- </h5>
- </div>
- <div class="card-body">
- if data.Stats.CompletedToday == 0 && data.Stats.FailedToday == 0 {
- <div class="text-center text-muted py-4">
- <i class="fas fa-check-circle fa-3x mb-3"></i>
- <p>No completed maintenance tasks today</p>
- <small>Completed tasks will appear here after workers finish processing them</small>
- </div>
- } else {
- <div class="table-responsive">
- <table class="table table-hover">
- <thead>
- <tr>
- <th>Type</th>
- <th>Status</th>
- <th>Volume</th>
- <th>Worker</th>
- <th>Duration</th>
- <th>Completed</th>
- </tr>
- </thead>
- <tbody>
- for _, task := range data.Tasks {
- if string(task.Status) == "completed" || string(task.Status) == "failed" || string(task.Status) == "cancelled" {
- if string(task.Status) == "failed" {
- <tr class="table-danger">
- <td>
- @TaskTypeIcon(task.Type)
- {string(task.Type)}
- </td>
- <td>@StatusBadge(task.Status)</td>
- <td>{fmt.Sprintf("%d", task.VolumeID)}</td>
- <td>
- if task.WorkerID != "" {
- <small>{task.WorkerID}</small>
- } else {
- <span class="text-muted">-</span>
- }
- </td>
- <td>
- if task.StartedAt != nil && task.CompletedAt != nil {
- {formatDuration(task.CompletedAt.Sub(*task.StartedAt))}
- } else {
- <span class="text-muted">-</span>
- }
- </td>
- <td>
- if task.CompletedAt != nil {
- {task.CompletedAt.Format("2006-01-02 15:04")}
- } else {
- <span class="text-muted">-</span>
- }
- </td>
- </tr>
- } else {
- <tr>
- <td>
- @TaskTypeIcon(task.Type)
- {string(task.Type)}
- </td>
- <td>@StatusBadge(task.Status)</td>
- <td>{fmt.Sprintf("%d", task.VolumeID)}</td>
- <td>
- if task.WorkerID != "" {
- <small>{task.WorkerID}</small>
- } else {
- <span class="text-muted">-</span>
- }
- </td>
- <td>
- if task.StartedAt != nil && task.CompletedAt != nil {
- {formatDuration(task.CompletedAt.Sub(*task.StartedAt))}
- } else {
- <span class="text-muted">-</span>
- }
- </td>
- <td>
- if task.CompletedAt != nil {
- {task.CompletedAt.Format("2006-01-02 15:04")}
- } else {
- <span class="text-muted">-</span>
- }
- </td>
- </tr>
- }
- }
- }
- </tbody>
- </table>
- </div>
- }
- </div>
- </div>
- </div>
- </div>
</div>
<script>
@@ -335,6 +335,13 @@ templ MaintenanceQueue(data *maintenance.MaintenanceQueueData) {
console.log("refreshPage called");
window.location.reload();
};
+
+ window.navigateToTask = function(element) {
+ const taskId = element.getAttribute('data-task-id');
+ if (taskId) {
+ window.location.href = '/maintenance/tasks/' + taskId;
+ }
+ };
</script>
}
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, "</h4><p class=\"text-muted mb-0\">Failed Today</p></div></div></div></div><!-- Pending Tasks --><div class=\"row mb-4\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header bg-primary text-white\"><h5 class=\"mb-0\"><i class=\"fas fa-clock me-2\"></i> Pending Tasks</h5></div><div class=\"card-body\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 5, "</h4><p class=\"text-muted mb-0\">Failed Today</p></div></div></div></div><!-- Completed Tasks --><div class=\"row mb-4\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header bg-success text-white\"><h5 class=\"mb-0\"><i class=\"fas fa-check-circle me-2\"></i> Completed Tasks</h5></div><div class=\"card-body\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- if data.Stats.PendingTasks == 0 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 6, "<div class=\"text-center text-muted py-4\"><i class=\"fas fa-clipboard-list fa-3x mb-3\"></i><p>No pending maintenance tasks</p><small>Pending tasks will appear here when the system detects maintenance needs</small></div>")
- if templ_7745c5c3_Err != nil {
- return templ_7745c5c3_Err
- }
- } else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 7, "<div class=\"table-responsive\"><table class=\"table table-hover\"><thead><tr><th>Type</th><th>Priority</th><th>Volume</th><th>Server</th><th>Reason</th><th>Created</th></tr></thead> <tbody>")
- 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, "<tr><td>")
- 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, "</td><td>")
- 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, "</td><td>")
- 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, "</td><td><small>")
- 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, "</small></td><td><small>")
- 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, "</small></td><td>")
- if templ_7745c5c3_Err != nil {
- return templ_7745c5c3_Err
- }
- var templ_7745c5c3_Var10 string
- templ_7745c5c3_Var10, templ_7745c5c3_Err = templ.JoinStringErrs(task.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, "</td></tr>")
- if templ_7745c5c3_Err != nil {
- return templ_7745c5c3_Err
- }
- }
- }
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "</tbody></table></div>")
- if templ_7745c5c3_Err != nil {
- return templ_7745c5c3_Err
- }
- }
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, "</div></div></div></div><!-- Active Tasks --><div class=\"row mb-4\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header bg-warning text-dark\"><h5 class=\"mb-0\"><i class=\"fas fa-running me-2\"></i> Active Tasks</h5></div><div class=\"card-body\">")
- if templ_7745c5c3_Err != nil {
- return templ_7745c5c3_Err
- }
- if data.Stats.RunningTasks == 0 {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, "<div class=\"text-center text-muted py-4\"><i class=\"fas fa-tasks fa-3x mb-3\"></i><p>No active maintenance tasks</p><small>Active tasks will appear here when workers start processing them</small></div>")
+ if data.Stats.CompletedToday == 0 && data.Stats.FailedToday == 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 6, "<div class=\"text-center text-muted py-4\"><i class=\"fas fa-check-circle fa-3x mb-3\"></i><p>No completed maintenance tasks today</p><small>Completed tasks will appear here after workers finish processing them</small></div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 18, "<div class=\"table-responsive\"><table class=\"table table-hover\"><thead><tr><th>Type</th><th>Status</th><th>Progress</th><th>Volume</th><th>Worker</th><th>Started</th></tr></thead> <tbody>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 7, "<div class=\"table-responsive\"><table class=\"table table-hover\"><thead><tr><th>Type</th><th>Status</th><th>Volume</th><th>Worker</th><th>Duration</th><th>Completed</th></tr></thead> <tbody>")
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, "<tr><td>")
- 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, "</td><td>")
- 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, "</td><td>")
- 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, "</td><td>")
- 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, "</td><td>")
- if templ_7745c5c3_Err != nil {
- return templ_7745c5c3_Err
- }
- if task.WorkerID != "" {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, "<small>")
- 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, "</small>")
- if templ_7745c5c3_Err != nil {
- return templ_7745c5c3_Err
- }
- } else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, "<span class=\"text-muted\">-</span>")
+ 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, "<tr class=\"table-danger clickable-row\" data-task-id=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- }
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, "</td><td>")
- 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, "<span class=\"text-muted\">-</span>")
- if templ_7745c5c3_Err != nil {
- return templ_7745c5c3_Err
- }
- }
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, "</td></tr>")
- if templ_7745c5c3_Err != nil {
- return templ_7745c5c3_Err
- }
- }
- }
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 30, "</tbody></table></div>")
- if templ_7745c5c3_Err != nil {
- return templ_7745c5c3_Err
- }
- }
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 31, "</div></div></div></div><!-- Completed Tasks --><div class=\"row mb-4\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header bg-success text-white\"><h5 class=\"mb-0\"><i class=\"fas fa-check-circle me-2\"></i> Completed Tasks</h5></div><div class=\"card-body\">")
- if 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, "<div class=\"text-center text-muted py-4\"><i class=\"fas fa-check-circle fa-3x mb-3\"></i><p>No completed maintenance tasks today</p><small>Completed tasks will appear here after workers finish processing them</small></div>")
- if templ_7745c5c3_Err != nil {
- return templ_7745c5c3_Err
- }
- } else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 33, "<div class=\"table-responsive\"><table class=\"table table-hover\"><thead><tr><th>Type</th><th>Status</th><th>Volume</th><th>Worker</th><th>Duration</th><th>Completed</th></tr></thead> <tbody>")
- 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, "<tr class=\"table-danger\"><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 9, "\" onclick=\"navigateToTask(this)\" style=\"cursor: pointer;\"><td>")
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, "</td><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 10, "</td><td>")
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, "</td><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 11, "</td><td>")
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, "</td><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 12, "</td><td>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if task.WorkerID != "" {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 38, "<small>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, "<small>")
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, "</small>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 14, "</small>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 40, "<span class=\"text-muted\">-</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "<span class=\"text-muted\">-</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 41, "</td><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, "</td><td>")
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, "<span class=\"text-muted\">-</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, "<span class=\"text-muted\">-</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 43, "</td><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 18, "</td><td>")
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, "<span class=\"text-muted\">-</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, "<span class=\"text-muted\">-</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 45, "</td></tr>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, "</td></tr>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 46, "<tr><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, "<tr class=\"clickable-row\" data-task-id=\"")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var12 string
+ templ_7745c5c3_Var12, 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: 137, Col: 99}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var12))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 22, "\" onclick=\"navigateToTask(this)\" style=\"cursor: pointer;\"><td>")
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, "</td><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 23, "</td><td>")
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, "</td><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, "</td><td>")
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, "</td><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 25, "</td><td>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if task.WorkerID != "" {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 50, "<small>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, "<small>")
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, "</small>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, "</small>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 52, "<span class=\"text-muted\">-</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, "<span class=\"text-muted\">-</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 53, "</td><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, "</td><td>")
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, "<span class=\"text-muted\">-</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 30, "<span class=\"text-muted\">-</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 55, "</td><td>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 31, "</td><td>")
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, "<span class=\"text-muted\">-</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 32, "<span class=\"text-muted\">-</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 57, "</td></tr>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 33, "</td></tr>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 34, "</tbody></table></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 35, "</div></div></div></div><!-- Pending Tasks --><div class=\"row mb-4\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header bg-primary text-white\"><h5 class=\"mb-0\"><i class=\"fas fa-clock me-2\"></i> Pending Tasks</h5></div><div class=\"card-body\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if data.Stats.PendingTasks == 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 36, "<div class=\"text-center text-muted py-4\"><i class=\"fas fa-clipboard-list fa-3x mb-3\"></i><p>No pending maintenance tasks</p><small>Pending tasks will appear here when the system detects maintenance needs</small></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 37, "<div class=\"table-responsive\"><table class=\"table table-hover\"><thead><tr><th>Type</th><th>Priority</th><th>Volume</th><th>Server</th><th>Reason</th><th>Created</th></tr></thead> <tbody>")
+ 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, "<tr class=\"clickable-row\" data-task-id=\"")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var18 string
+ templ_7745c5c3_Var18, 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: 211, Col: 95}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var18))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 39, "\" onclick=\"navigateToTask(this)\" style=\"cursor: pointer;\"><td>")
+ 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, "</td><td>")
+ 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, "</td><td>")
+ 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, "</td><td><small>")
+ 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, "</small></td><td><small>")
+ 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, "</small></td><td>")
+ 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, "</td></tr>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 46, "</tbody></table></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 47, "</div></div></div></div><!-- Active Tasks --><div class=\"row mb-4\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header bg-warning text-dark\"><h5 class=\"mb-0\"><i class=\"fas fa-running me-2\"></i> Active Tasks</h5></div><div class=\"card-body\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if data.Stats.RunningTasks == 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 48, "<div class=\"text-center text-muted py-4\"><i class=\"fas fa-tasks fa-3x mb-3\"></i><p>No active maintenance tasks</p><small>Active tasks will appear here when workers start processing them</small></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 49, "<div class=\"table-responsive\"><table class=\"table table-hover\"><thead><tr><th>Type</th><th>Status</th><th>Progress</th><th>Volume</th><th>Worker</th><th>Started</th></tr></thead> <tbody>")
+ 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, "<tr class=\"clickable-row\" data-task-id=\"")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var24 string
+ templ_7745c5c3_Var24, 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: 266, Col: 95}
+ }
+ _, 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, 51, "\" onclick=\"navigateToTask(this)\" style=\"cursor: pointer;\"><td>")
+ 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, "</td><td>")
+ 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, "</td><td>")
+ 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, "</td><td>")
+ 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, "</td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if task.WorkerID != "" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 56, "<small>")
+ 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, "</small>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 58, "<span class=\"text-muted\">-</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 59, "</td><td>")
+ 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, "<span class=\"text-muted\">-</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 61, "</td></tr>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 58, "</tbody></table></div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 62, "</tbody></table></div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 59, "</div></div></div></div></div><script>\n // Debug output to browser console\n console.log(\"DEBUG: Maintenance Queue Template loaded\");\n \n // Auto-refresh every 10 seconds\n setInterval(function() {\n if (!document.hidden) {\n window.location.reload();\n }\n }, 10000);\n\n window.triggerScan = function() {\n console.log(\"triggerScan called\");\n fetch('/api/maintenance/scan', {\n method: 'POST',\n headers: {\n 'Content-Type': 'application/json',\n }\n })\n .then(response => response.json())\n .then(data => {\n if (data.success) {\n alert('Maintenance scan triggered successfully');\n setTimeout(() => window.location.reload(), 2000);\n } else {\n alert('Failed to trigger scan: ' + (data.error || 'Unknown error'));\n }\n })\n .catch(error => {\n alert('Error: ' + error.message);\n });\n };\n\n window.refreshPage = function() {\n console.log(\"refreshPage called\");\n window.location.reload();\n };\n </script>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 63, "</div></div></div></div></div><script>\n // Debug output to browser console\n console.log(\"DEBUG: Maintenance Queue Template loaded\");\n \n // Auto-refresh every 10 seconds\n setInterval(function() {\n if (!document.hidden) {\n window.location.reload();\n }\n }, 10000);\n\n window.triggerScan = function() {\n console.log(\"triggerScan called\");\n fetch('/api/maintenance/scan', {\n method: 'POST',\n headers: {\n 'Content-Type': 'application/json',\n }\n })\n .then(response => response.json())\n .then(data => {\n if (data.success) {\n alert('Maintenance scan triggered successfully');\n setTimeout(() => window.location.reload(), 2000);\n } else {\n alert('Failed to trigger scan: ' + (data.error || 'Unknown error'));\n }\n })\n .catch(error => {\n alert('Error: ' + error.message);\n });\n };\n\n window.refreshPage = function() {\n console.log(\"refreshPage called\");\n window.location.reload();\n };\n\n window.navigateToTask = function(element) {\n const taskId = element.getAttribute('data-task-id');\n if (taskId) {\n window.location.href = '/maintenance/tasks/' + taskId;\n }\n };\n </script>")
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, "<i class=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 64, "<i class=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var27 string
- templ_7745c5c3_Var27, templ_7745c5c3_Err = templ.JoinStringErrs(templ.CSSClasses(templ_7745c5c3_Var26).String())
+ var templ_7745c5c3_Var31 string
+ templ_7745c5c3_Var31, templ_7745c5c3_Err = templ.JoinStringErrs(templ.CSSClasses(templ_7745c5c3_Var30).String())
if templ_7745c5c3_Err != nil {
return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 1, Col: 0}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var27))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var31))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 61, "\"></i>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 65, "\"></i>")
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, "<span class=\"badge bg-danger\">Critical</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 66, "<span class=\"badge bg-danger\">Critical</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
case maintenance.PriorityHigh:
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 63, "<span class=\"badge bg-warning\">High</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 67, "<span class=\"badge bg-warning\">High</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
case maintenance.PriorityNormal:
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 64, "<span class=\"badge bg-primary\">Normal</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 68, "<span class=\"badge bg-primary\">Normal</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
case maintenance.PriorityLow:
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 65, "<span class=\"badge bg-secondary\">Low</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 69, "<span class=\"badge bg-secondary\">Low</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
default:
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 66, "<span class=\"badge bg-light text-dark\">Unknown</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 70, "<span class=\"badge bg-light text-dark\">Unknown</span>")
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, "<span class=\"badge bg-secondary\">Pending</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 71, "<span class=\"badge bg-secondary\">Pending</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
case maintenance.TaskStatusAssigned:
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 68, "<span class=\"badge bg-info\">Assigned</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 72, "<span class=\"badge bg-info\">Assigned</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
case maintenance.TaskStatusInProgress:
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 69, "<span class=\"badge bg-warning\">Running</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 73, "<span class=\"badge bg-warning\">Running</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
case maintenance.TaskStatusCompleted:
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 70, "<span class=\"badge bg-success\">Completed</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 74, "<span class=\"badge bg-success\">Completed</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
case maintenance.TaskStatusFailed:
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 71, "<span class=\"badge bg-danger\">Failed</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 75, "<span class=\"badge bg-danger\">Failed</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
case maintenance.TaskStatusCancelled:
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 72, "<span class=\"badge bg-light text-dark\">Cancelled</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 76, "<span class=\"badge bg-light text-dark\">Cancelled</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
default:
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 73, "<span class=\"badge bg-light text-dark\">Unknown</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 77, "<span class=\"badge bg-light text-dark\">Unknown</span>")
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, "<div class=\"progress\" style=\"height: 8px; min-width: 100px;\"><div class=\"progress-bar\" role=\"progressbar\" style=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 78, "<div class=\"progress\" style=\"height: 8px; min-width: 100px;\"><div class=\"progress-bar\" role=\"progressbar\" style=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var31 string
- templ_7745c5c3_Var31, templ_7745c5c3_Err = templruntime.SanitizeStyleAttributeValues(fmt.Sprintf("width: %.1f%%", progress))
+ var templ_7745c5c3_Var35 string
+ templ_7745c5c3_Var35, templ_7745c5c3_Err = templruntime.SanitizeStyleAttributeValues(fmt.Sprintf("width: %.1f%%", progress))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 383, Col: 102}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/maintenance_queue.templ`, Line: 390, Col: 102}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var31))
+ _, 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, 75, "\"></div></div><small class=\"text-muted\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 79, "\"></div></div><small class=\"text-muted\">")
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, "</small>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 80, "</small>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else if status == maintenance.TaskStatusCompleted {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 77, "<div class=\"progress\" style=\"height: 8px; min-width: 100px;\"><div class=\"progress-bar bg-success\" role=\"progressbar\" style=\"width: 100%\"></div></div><small class=\"text-success\">100%</small>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 81, "<div class=\"progress\" style=\"height: 8px; min-width: 100px;\"><div class=\"progress-bar bg-success\" role=\"progressbar\" style=\"width: 100%\"></div></div><small class=\"text-success\">100%</small>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 78, "<span class=\"text-muted\">-</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 82, "<span class=\"text-muted\">-</span>")
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" {
<h6 class="alert-heading">Erasure Coding Operations:</h6>
<p class="mb-2"><strong>Performance:</strong> Erasure coding is CPU and I/O intensive. Consider running during off-peak hours.</p>
- <p class="mb-2"><strong>Durability:</strong> With 10+4 configuration, can tolerate up to 4 shard failures.</p>
+ <p class="mb-2"><strong>Durability:</strong> With { fmt.Sprintf("%d+%d", erasure_coding.DataShardsCount, erasure_coding.ParityShardsCount) } configuration, can tolerate up to { fmt.Sprintf("%d", erasure_coding.ParityShardsCount) } shard failures.</p>
<p class="mb-0"><strong>Configuration:</strong> Fullness ratio should be between 0.5 and 1.0 (e.g., 0.90 for 90%).</p>
}
</div>
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, "<h6 class=\"alert-heading\">Erasure Coding Operations:</h6><p class=\"mb-2\"><strong>Performance:</strong> Erasure coding is CPU and I/O intensive. Consider running during off-peak hours.</p><p class=\"mb-2\"><strong>Durability:</strong> With 10+4 configuration, can tolerate up to 4 shard failures.</p><p class=\"mb-0\"><strong>Configuration:</strong> Fullness ratio should be between 0.5 and 1.0 (e.g., 0.90 for 90%).</p>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 9, "<h6 class=\"alert-heading\">Erasure Coding Operations:</h6><p class=\"mb-2\"><strong>Performance:</strong> Erasure coding is CPU and I/O intensive. Consider running during off-peak hours.</p><p class=\"mb-2\"><strong>Durability:</strong> With ")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var6 string
+ templ_7745c5c3_Var6, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d+%d", erasure_coding.DataShardsCount, erasure_coding.ParityShardsCount))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 118, Col: 170}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var6))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 10, " configuration, can tolerate up to ")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var7 string
+ templ_7745c5c3_Var7, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%d", erasure_coding.ParityShardsCount))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 118, Col: 260}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var7))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 11, " shard failures.</p><p class=\"mb-0\"><strong>Configuration:</strong> Fullness ratio should be between 0.5 and 1.0 (e.g., 0.90 for 90%).</p>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 10, "</div></div></div></div></div></div><script>\n function resetToDefaults() {\n if (confirm('Are you sure you want to reset to default configuration? This will overwrite your current settings.')) {\n // Reset form fields to their default values\n const form = document.getElementById('taskConfigForm');\n const schemaFields = window.taskConfigSchema ? window.taskConfigSchema.fields : {};\n \n Object.keys(schemaFields).forEach(fieldName => {\n const field = schemaFields[fieldName];\n const element = document.getElementById(fieldName);\n \n if (element && field.default_value !== undefined) {\n if (field.input_type === 'checkbox') {\n element.checked = field.default_value;\n } else if (field.input_type === 'interval') {\n // Handle interval fields with value and unit\n const valueElement = document.getElementById(fieldName + '_value');\n const unitElement = document.getElementById(fieldName + '_unit');\n if (valueElement && unitElement && field.default_value) {\n const defaultSeconds = field.default_value;\n const { value, unit } = convertSecondsToTaskIntervalValueUnit(defaultSeconds);\n valueElement.value = value;\n unitElement.value = unit;\n }\n } else {\n element.value = field.default_value;\n }\n }\n });\n }\n }\n\n function convertSecondsToTaskIntervalValueUnit(totalSeconds) {\n if (totalSeconds === 0) {\n return { value: 0, unit: 'minutes' };\n }\n\n // Check if it's evenly divisible by days\n if (totalSeconds % (24 * 3600) === 0) {\n return { value: totalSeconds / (24 * 3600), unit: 'days' };\n }\n\n // Check if it's evenly divisible by hours\n if (totalSeconds % 3600 === 0) {\n return { value: totalSeconds / 3600, unit: 'hours' };\n }\n\n // Default to minutes\n return { value: totalSeconds / 60, unit: 'minutes' };\n }\n\n // Store schema data for JavaScript access (moved to after div is created)\n </script><!-- Hidden element to store schema data --><div data-task-schema=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 12, "</div></div></div></div></div></div><script>\n function resetToDefaults() {\n if (confirm('Are you sure you want to reset to default configuration? This will overwrite your current settings.')) {\n // Reset form fields to their default values\n const form = document.getElementById('taskConfigForm');\n const schemaFields = window.taskConfigSchema ? window.taskConfigSchema.fields : {};\n \n Object.keys(schemaFields).forEach(fieldName => {\n const field = schemaFields[fieldName];\n const element = document.getElementById(fieldName);\n \n if (element && field.default_value !== undefined) {\n if (field.input_type === 'checkbox') {\n element.checked = field.default_value;\n } else if (field.input_type === 'interval') {\n // Handle interval fields with value and unit\n const valueElement = document.getElementById(fieldName + '_value');\n const unitElement = document.getElementById(fieldName + '_unit');\n if (valueElement && unitElement && field.default_value) {\n const defaultSeconds = field.default_value;\n const { value, unit } = convertSecondsToTaskIntervalValueUnit(defaultSeconds);\n valueElement.value = value;\n unitElement.value = unit;\n }\n } else {\n element.value = field.default_value;\n }\n }\n });\n }\n }\n\n function convertSecondsToTaskIntervalValueUnit(totalSeconds) {\n if (totalSeconds === 0) {\n return { value: 0, unit: 'minutes' };\n }\n\n // Check if it's evenly divisible by days\n if (totalSeconds % (24 * 3600) === 0) {\n return { value: totalSeconds / (24 * 3600), unit: 'days' };\n }\n\n // Check if it's evenly divisible by hours\n if (totalSeconds % 3600 === 0) {\n return { value: totalSeconds / 3600, unit: 'hours' };\n }\n\n // Default to minutes\n return { value: totalSeconds / 60, unit: 'minutes' };\n }\n\n // Store schema data for JavaScript access (moved to after div is created)\n </script><!-- Hidden element to store schema data --><div data-task-schema=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var6 string
- templ_7745c5c3_Var6, templ_7745c5c3_Err = templ.JoinStringErrs(taskSchemaToBase64JSON(schema))
+ var templ_7745c5c3_Var8 string
+ templ_7745c5c3_Var8, templ_7745c5c3_Err = templ.JoinStringErrs(taskSchemaToBase64JSON(schema))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 182, Col: 58}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 183, Col: 58}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var6))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var8))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 11, "\" style=\"display: none;\"></div><script>\n // Load schema data now that the div exists\n const base64Data = document.querySelector('[data-task-schema]').getAttribute('data-task-schema');\n const jsonStr = atob(base64Data);\n window.taskConfigSchema = JSON.parse(jsonStr);\n </script>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, "\" style=\"display: none;\"></div><script>\n // Load schema data now that the div exists\n const base64Data = document.querySelector('[data-task-schema]').getAttribute('data-task-schema');\n const jsonStr = atob(base64Data);\n window.taskConfigSchema = JSON.parse(jsonStr);\n </script>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
@@ -181,584 +208,584 @@ func TaskConfigField(field *config.Field, config interface{}) templ.Component {
}()
}
ctx = templ.InitializeContext(ctx)
- templ_7745c5c3_Var7 := templ.GetChildren(ctx)
- if templ_7745c5c3_Var7 == nil {
- templ_7745c5c3_Var7 = templ.NopComponent
+ templ_7745c5c3_Var9 := templ.GetChildren(ctx)
+ if templ_7745c5c3_Var9 == nil {
+ templ_7745c5c3_Var9 = templ.NopComponent
}
ctx = templ.ClearChildren(ctx)
if field.InputType == "interval" {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 12, "<!-- Interval field with number input + unit dropdown --> <div class=\"mb-3\"><label for=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 14, "<!-- Interval field with number input + unit dropdown --> <div class=\"mb-3\"><label for=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var8 string
- templ_7745c5c3_Var8, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
+ var templ_7745c5c3_Var10 string
+ templ_7745c5c3_Var10, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 197, Col: 39}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 198, Col: 39}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var8))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var10))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 13, "\" class=\"form-label\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "\" class=\"form-label\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var9 string
- templ_7745c5c3_Var9, templ_7745c5c3_Err = templ.JoinStringErrs(field.DisplayName)
+ var templ_7745c5c3_Var11 string
+ templ_7745c5c3_Var11, templ_7745c5c3_Err = templ.JoinStringErrs(field.DisplayName)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 198, Col: 35}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 199, Col: 35}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var9))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var11))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 14, " ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, " ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Required {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "<span class=\"text-danger\">*</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, "<span class=\"text-danger\">*</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, "</label><div class=\"input-group\"><input type=\"number\" class=\"form-control\" id=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 18, "</label><div class=\"input-group\"><input type=\"number\" class=\"form-control\" id=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var10 string
- templ_7745c5c3_Var10, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName + "_value")
+ var templ_7745c5c3_Var12 string
+ templ_7745c5c3_Var12, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName + "_value")
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 207, Col: 50}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 208, Col: 50}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var10))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var12))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, "\" name=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, "\" name=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var11 string
- templ_7745c5c3_Var11, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName + "_value")
+ var templ_7745c5c3_Var13 string
+ templ_7745c5c3_Var13, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName + "_value")
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 208, Col: 52}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 209, Col: 52}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var11))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var13))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 18, "\" value=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, "\" value=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var12 string
- templ_7745c5c3_Var12, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%.0f", components.ConvertInt32SecondsToDisplayValue(getTaskConfigInt32Field(config, field.JSONName))))
+ var templ_7745c5c3_Var14 string
+ templ_7745c5c3_Var14, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%.0f", components.ConvertInt32SecondsToDisplayValue(getTaskConfigInt32Field(config, field.JSONName))))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 209, Col: 142}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 210, Col: 142}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var12))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var14))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, "\" step=\"1\" min=\"1\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, "\" step=\"1\" min=\"1\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Required {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 20, " required")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 22, " required")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, "> <select class=\"form-select\" id=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 23, "> <select class=\"form-select\" id=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var13 string
- templ_7745c5c3_Var13, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName + "_unit")
+ var templ_7745c5c3_Var15 string
+ templ_7745c5c3_Var15, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName + "_unit")
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 218, Col: 49}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 219, Col: 49}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var13))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var15))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 22, "\" name=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, "\" name=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var14 string
- templ_7745c5c3_Var14, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName + "_unit")
+ var templ_7745c5c3_Var16 string
+ templ_7745c5c3_Var16, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName + "_unit")
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 219, Col: 51}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 220, Col: 51}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var14))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var16))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 23, "\" style=\"max-width: 120px;\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 25, "\" style=\"max-width: 120px;\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Required {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 24, " required")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, " required")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 25, "><option value=\"minutes\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, "><option value=\"minutes\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if components.GetInt32DisplayUnit(getTaskConfigInt32Field(config, field.JSONName)) == "minutes" {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, " selected")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, " selected")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, ">Minutes</option> <option value=\"hours\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, ">Minutes</option> <option value=\"hours\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if components.GetInt32DisplayUnit(getTaskConfigInt32Field(config, field.JSONName)) == "hours" {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, " selected")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 30, " selected")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, ">Hours</option> <option value=\"days\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 31, ">Hours</option> <option value=\"days\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if components.GetInt32DisplayUnit(getTaskConfigInt32Field(config, field.JSONName)) == "days" {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 30, " selected")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 32, " selected")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 31, ">Days</option></select></div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 33, ">Days</option></select></div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Description != "" {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 32, "<div class=\"form-text text-muted\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 34, "<div class=\"form-text text-muted\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var15 string
- templ_7745c5c3_Var15, templ_7745c5c3_Err = templ.JoinStringErrs(field.Description)
+ var templ_7745c5c3_Var17 string
+ templ_7745c5c3_Var17, templ_7745c5c3_Err = templ.JoinStringErrs(field.Description)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 252, Col: 69}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 253, Col: 69}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var15))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var17))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 33, "</div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 35, "</div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 34, "</div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 36, "</div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else if field.InputType == "checkbox" {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 35, "<!-- Checkbox field --> <div class=\"mb-3\"><div class=\"form-check form-switch\"><input class=\"form-check-input\" type=\"checkbox\" id=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 37, "<!-- Checkbox field --> <div class=\"mb-3\"><div class=\"form-check form-switch\"><input class=\"form-check-input\" type=\"checkbox\" id=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var16 string
- templ_7745c5c3_Var16, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
+ var templ_7745c5c3_Var18 string
+ templ_7745c5c3_Var18, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 262, Col: 39}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 263, Col: 39}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var16))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var18))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 36, "\" name=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 38, "\" name=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var17 string
- templ_7745c5c3_Var17, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
+ var templ_7745c5c3_Var19 string
+ templ_7745c5c3_Var19, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 263, Col: 41}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 264, Col: 41}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var17))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var19))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 37, "\" value=\"on\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 39, "\" value=\"on\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if getTaskConfigBoolField(config, field.JSONName) {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 38, " checked")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 40, " checked")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 39, "> <label class=\"form-check-label\" for=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 41, "> <label class=\"form-check-label\" for=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var18 string
- templ_7745c5c3_Var18, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
+ var templ_7745c5c3_Var20 string
+ templ_7745c5c3_Var20, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 269, Col: 68}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 270, Col: 68}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var18))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var20))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 40, "\"><strong>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 42, "\"><strong>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var19 string
- templ_7745c5c3_Var19, templ_7745c5c3_Err = templ.JoinStringErrs(field.DisplayName)
+ var templ_7745c5c3_Var21 string
+ templ_7745c5c3_Var21, templ_7745c5c3_Err = templ.JoinStringErrs(field.DisplayName)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 270, Col: 47}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 271, Col: 47}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var19))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var21))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 41, "</strong></label></div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 43, "</strong></label></div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Description != "" {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 42, "<div class=\"form-text text-muted\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 44, "<div class=\"form-text text-muted\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var20 string
- templ_7745c5c3_Var20, templ_7745c5c3_Err = templ.JoinStringErrs(field.Description)
+ var templ_7745c5c3_Var22 string
+ templ_7745c5c3_Var22, templ_7745c5c3_Err = templ.JoinStringErrs(field.Description)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 274, Col: 69}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 275, Col: 69}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var20))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var22))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 43, "</div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 45, "</div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 44, "</div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 46, "</div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else if field.InputType == "text" {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 45, "<!-- Text field --> <div class=\"mb-3\"><label for=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 47, "<!-- Text field --> <div class=\"mb-3\"><label for=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var21 string
- templ_7745c5c3_Var21, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
+ var templ_7745c5c3_Var23 string
+ templ_7745c5c3_Var23, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 280, Col: 39}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 281, Col: 39}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var21))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var23))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 46, "\" class=\"form-label\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 48, "\" class=\"form-label\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var22 string
- templ_7745c5c3_Var22, templ_7745c5c3_Err = templ.JoinStringErrs(field.DisplayName)
+ var templ_7745c5c3_Var24 string
+ templ_7745c5c3_Var24, templ_7745c5c3_Err = templ.JoinStringErrs(field.DisplayName)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 281, Col: 35}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 282, Col: 35}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var22))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var24))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 47, " ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 49, " ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Required {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 48, "<span class=\"text-danger\">*</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 50, "<span class=\"text-danger\">*</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 49, "</label> <input type=\"text\" class=\"form-control\" id=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 51, "</label> <input type=\"text\" class=\"form-control\" id=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var23 string
- templ_7745c5c3_Var23, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
+ var templ_7745c5c3_Var25 string
+ templ_7745c5c3_Var25, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 289, Col: 35}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 290, Col: 35}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var23))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var25))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 50, "\" name=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 52, "\" name=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var24 string
- templ_7745c5c3_Var24, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
+ var templ_7745c5c3_Var26 string
+ templ_7745c5c3_Var26, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 290, Col: 37}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 291, Col: 37}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var24))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var26))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 51, "\" value=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 53, "\" value=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var25 string
- templ_7745c5c3_Var25, templ_7745c5c3_Err = templ.JoinStringErrs(getTaskConfigStringField(config, field.JSONName))
+ var templ_7745c5c3_Var27 string
+ templ_7745c5c3_Var27, templ_7745c5c3_Err = templ.JoinStringErrs(getTaskConfigStringField(config, field.JSONName))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 291, Col: 72}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 292, Col: 72}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var25))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var27))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 52, "\" placeholder=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 54, "\" placeholder=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var26 string
- templ_7745c5c3_Var26, templ_7745c5c3_Err = templ.JoinStringErrs(field.Placeholder)
+ var templ_7745c5c3_Var28 string
+ templ_7745c5c3_Var28, templ_7745c5c3_Err = templ.JoinStringErrs(field.Placeholder)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 292, Col: 47}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 293, Col: 47}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var26))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var28))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 53, "\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 55, "\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Required {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 54, " required")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 56, " required")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 55, "> ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 57, "> ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Description != "" {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 56, "<div class=\"form-text text-muted\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 58, "<div class=\"form-text text-muted\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var27 string
- templ_7745c5c3_Var27, templ_7745c5c3_Err = templ.JoinStringErrs(field.Description)
+ var templ_7745c5c3_Var29 string
+ templ_7745c5c3_Var29, templ_7745c5c3_Err = templ.JoinStringErrs(field.Description)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 298, Col: 69}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 299, Col: 69}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var27))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var29))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 57, "</div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 59, "</div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 58, "</div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 60, "</div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
} else {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 59, "<!-- Number field --> <div class=\"mb-3\"><label for=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 61, "<!-- Number field --> <div class=\"mb-3\"><label for=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var28 string
- templ_7745c5c3_Var28, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
+ var templ_7745c5c3_Var30 string
+ templ_7745c5c3_Var30, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 304, Col: 39}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 305, Col: 39}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var28))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var30))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 60, "\" class=\"form-label\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 62, "\" class=\"form-label\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var29 string
- templ_7745c5c3_Var29, templ_7745c5c3_Err = templ.JoinStringErrs(field.DisplayName)
+ var templ_7745c5c3_Var31 string
+ templ_7745c5c3_Var31, templ_7745c5c3_Err = templ.JoinStringErrs(field.DisplayName)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 305, Col: 35}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 306, Col: 35}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var29))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var31))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 61, " ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 63, " ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Required {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 62, "<span class=\"text-danger\">*</span>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 64, "<span class=\"text-danger\">*</span>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 63, "</label> <input type=\"number\" class=\"form-control\" id=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 65, "</label> <input type=\"number\" class=\"form-control\" id=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var30 string
- templ_7745c5c3_Var30, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
+ var templ_7745c5c3_Var32 string
+ templ_7745c5c3_Var32, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 313, Col: 35}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 314, Col: 35}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var30))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var32))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 64, "\" name=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 66, "\" name=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var31 string
- templ_7745c5c3_Var31, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
+ var templ_7745c5c3_Var33 string
+ templ_7745c5c3_Var33, templ_7745c5c3_Err = templ.JoinStringErrs(field.JSONName)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 314, Col: 37}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 315, Col: 37}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var31))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var33))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 65, "\" value=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 67, "\" value=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var32 string
- templ_7745c5c3_Var32, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%.6g", getTaskConfigFloatField(config, field.JSONName)))
+ var templ_7745c5c3_Var34 string
+ templ_7745c5c3_Var34, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%.6g", getTaskConfigFloatField(config, field.JSONName)))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 315, Col: 92}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 316, Col: 92}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var32))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var34))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 66, "\" placeholder=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 68, "\" placeholder=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var33 string
- templ_7745c5c3_Var33, templ_7745c5c3_Err = templ.JoinStringErrs(field.Placeholder)
+ var templ_7745c5c3_Var35 string
+ templ_7745c5c3_Var35, templ_7745c5c3_Err = templ.JoinStringErrs(field.Placeholder)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 316, Col: 47}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 317, Col: 47}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var33))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var35))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 67, "\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 69, "\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.MinValue != nil {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 68, " min=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 70, " min=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var34 string
- templ_7745c5c3_Var34, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%v", field.MinValue))
+ var templ_7745c5c3_Var36 string
+ templ_7745c5c3_Var36, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%v", field.MinValue))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 318, Col: 59}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 319, Col: 59}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var34))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var36))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 69, "\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 71, "\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
if field.MaxValue != nil {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 70, " max=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 72, " max=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var35 string
- templ_7745c5c3_Var35, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%v", field.MaxValue))
+ var templ_7745c5c3_Var37 string
+ templ_7745c5c3_Var37, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("%v", field.MaxValue))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 321, Col: 59}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 322, Col: 59}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var35))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var37))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 71, "\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 73, "\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 72, " step=\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 74, " step=\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var36 string
- templ_7745c5c3_Var36, templ_7745c5c3_Err = templ.JoinStringErrs(getTaskNumberStep(field))
+ var templ_7745c5c3_Var38 string
+ templ_7745c5c3_Var38, templ_7745c5c3_Err = templ.JoinStringErrs(getTaskNumberStep(field))
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 323, Col: 47}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 324, Col: 47}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var36))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var38))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 73, "\"")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 75, "\"")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Required {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 74, " required")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 76, " required")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 75, "> ")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 77, "> ")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
if field.Description != "" {
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 76, "<div class=\"form-text text-muted\">")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 78, "<div class=\"form-text text-muted\">")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- var templ_7745c5c3_Var37 string
- templ_7745c5c3_Var37, templ_7745c5c3_Err = templ.JoinStringErrs(field.Description)
+ var templ_7745c5c3_Var39 string
+ templ_7745c5c3_Var39, templ_7745c5c3_Err = templ.JoinStringErrs(field.Description)
if templ_7745c5c3_Err != nil {
- return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 329, Col: 69}
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_config_schema.templ`, Line: 330, Col: 69}
}
- _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var37))
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var39))
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 77, "</div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 79, "</div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
}
- templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 78, "</div>")
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 80, "</div>")
if templ_7745c5c3_Err != nil {
return templ_7745c5c3_Err
}
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) {
+ <div class="container-fluid">
+ <!-- Header -->
+ <div class="row mb-4">
+ <div class="col-12">
+ <div class="d-flex justify-content-between align-items-center">
+ <div>
+ <nav aria-label="breadcrumb">
+ <ol class="breadcrumb mb-1">
+ <li class="breadcrumb-item"><a href="/maintenance">Maintenance</a></li>
+ <li class="breadcrumb-item active" aria-current="page">Task Detail</li>
+ </ol>
+ </nav>
+ <h2 class="mb-0">
+ <i class="fas fa-tasks me-2"></i>
+ Task Detail: {data.Task.ID}
+ </h2>
+ </div>
+ <div class="btn-group">
+ <button type="button" class="btn btn-secondary" onclick="history.back()">
+ <i class="fas fa-arrow-left me-1"></i>
+ Back
+ </button>
+ <button type="button" class="btn btn-secondary" onclick="refreshPage()">
+ <i class="fas fa-sync-alt me-1"></i>
+ Refresh
+ </button>
+ </div>
+ </div>
+ </div>
+ </div>
+
+ <!-- Task Overview Card -->
+ <div class="row mb-4">
+ <div class="col-12">
+ <div class="card">
+ <div class="card-header">
+ <h5 class="mb-0">
+ <i class="fas fa-info-circle me-2"></i>
+ Task Overview
+ </h5>
+ </div>
+ <div class="card-body">
+ <div class="row">
+ <div class="col-md-6">
+ <dl class="row">
+ <dt class="col-sm-4">Task ID:</dt>
+ <dd class="col-sm-8"><code>{data.Task.ID}</code></dd>
+
+ <dt class="col-sm-4">Type:</dt>
+ <dd class="col-sm-8">
+ <span class="badge bg-info">{string(data.Task.Type)}</span>
+ </dd>
+
+ <dt class="col-sm-4">Status:</dt>
+ <dd class="col-sm-8">
+ if data.Task.Status == maintenance.TaskStatusPending {
+ <span class="badge bg-secondary">Pending</span>
+ } else if data.Task.Status == maintenance.TaskStatusAssigned {
+ <span class="badge bg-info">Assigned</span>
+ } else if data.Task.Status == maintenance.TaskStatusInProgress {
+ <span class="badge bg-warning">In Progress</span>
+ } else if data.Task.Status == maintenance.TaskStatusCompleted {
+ <span class="badge bg-success">Completed</span>
+ } else if data.Task.Status == maintenance.TaskStatusFailed {
+ <span class="badge bg-danger">Failed</span>
+ } else if data.Task.Status == maintenance.TaskStatusCancelled {
+ <span class="badge bg-dark">Cancelled</span>
+ }
+ </dd>
+
+ <dt class="col-sm-4">Priority:</dt>
+ <dd class="col-sm-8">
+ if data.Task.Priority == maintenance.PriorityHigh {
+ <span class="badge bg-danger">High</span>
+ } else if data.Task.Priority == maintenance.PriorityCritical {
+ <span class="badge bg-danger">Critical</span>
+ } else if data.Task.Priority == maintenance.PriorityNormal {
+ <span class="badge bg-warning">Normal</span>
+ } else {
+ <span class="badge bg-secondary">Low</span>
+ }
+ </dd>
+
+ if data.Task.Reason != "" {
+ <dt class="col-sm-4">Reason:</dt>
+ <dd class="col-sm-8">
+ <span class="text-muted">{data.Task.Reason}</span>
+ </dd>
+ }
+ </dl>
+ </div>
+ <div class="col-md-6">
+ <!-- Task Timeline -->
+ <div class="mb-3">
+ <h6 class="text-primary mb-3">
+ <i class="fas fa-clock me-1"></i>Task Timeline
+ </h6>
+ <div class="timeline-container">
+ <div class="timeline-progress">
+ <div class="timeline-step" data-step="created">
+ <div class="timeline-circle completed">
+ <i class="fas fa-plus"></i>
+ </div>
+ <div class="timeline-connector completed"></div>
+ <div class="timeline-label">
+ <strong>Created</strong>
+ <small class="d-block text-muted">{data.Task.CreatedAt.Format("01-02 15:04:05")}</small>
+ </div>
+ </div>
+
+ <div class="timeline-step" data-step="scheduled">
+ <div class="timeline-circle completed">
+ <i class="fas fa-calendar"></i>
+ </div>
+ if data.Task.StartedAt != nil {
+ <div class="timeline-connector completed"></div>
+ } else {
+ <div class="timeline-connector"></div>
+ }
+ <div class="timeline-label">
+ <strong>Scheduled</strong>
+ <small class="d-block text-muted">{data.Task.ScheduledAt.Format("01-02 15:04:05")}</small>
+ </div>
+ </div>
+
+ <div class="timeline-step" data-step="started">
+ if data.Task.StartedAt != nil {
+ <div class="timeline-circle completed">
+ <i class="fas fa-play"></i>
+ </div>
+ } else {
+ <div class="timeline-circle pending">
+ <i class="fas fa-clock"></i>
+ </div>
+ }
+ if data.Task.CompletedAt != nil {
+ <div class="timeline-connector completed"></div>
+ } else {
+ <div class="timeline-connector"></div>
+ }
+ <div class="timeline-label">
+ <strong>Started</strong>
+ <small class="d-block text-muted">
+ if data.Task.StartedAt != nil {
+ {data.Task.StartedAt.Format("01-02 15:04:05")}
+ } else {
+ —
+ }
+ </small>
+ </div>
+ </div>
+
+ <div class="timeline-step" data-step="completed">
+ if data.Task.CompletedAt != nil {
+ <div class="timeline-circle completed">
+ if data.Task.Status == maintenance.TaskStatusCompleted {
+ <i class="fas fa-check"></i>
+ } else if data.Task.Status == maintenance.TaskStatusFailed {
+ <i class="fas fa-times"></i>
+ } else {
+ <i class="fas fa-stop"></i>
+ }
+ </div>
+ } else {
+ <div class="timeline-circle pending">
+ <i class="fas fa-hourglass-half"></i>
+ </div>
+ }
+ <div class="timeline-label">
+ <strong>
+ 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
+ }
+ </strong>
+ <small class="d-block text-muted">
+ if data.Task.CompletedAt != nil {
+ {data.Task.CompletedAt.Format("01-02 15:04:05")}
+ } else {
+ —
+ }
+ </small>
+ </div>
+ </div>
+ </div>
+ </div>
+ </div>
+
+ <!-- Additional Info -->
+ if data.Task.WorkerID != "" {
+ <dl class="row">
+ <dt class="col-sm-4">Worker:</dt>
+ <dd class="col-sm-8"><code>{data.Task.WorkerID}</code></dd>
+ </dl>
+ }
+
+ <dl class="row">
+ if data.Task.TypedParams != nil && data.Task.TypedParams.VolumeSize > 0 {
+ <dt class="col-sm-4">Volume Size:</dt>
+ <dd class="col-sm-8">
+ <span class="badge bg-primary">{formatBytes(int64(data.Task.TypedParams.VolumeSize))}</span>
+ </dd>
+ }
+
+ if data.Task.TypedParams != nil && data.Task.TypedParams.Collection != "" {
+ <dt class="col-sm-4">Collection:</dt>
+ <dd class="col-sm-8">
+ <span class="badge bg-info"><i class="fas fa-folder me-1"></i>{data.Task.TypedParams.Collection}</span>
+ </dd>
+ }
+
+ if data.Task.TypedParams != nil && data.Task.TypedParams.DataCenter != "" {
+ <dt class="col-sm-4">Data Center:</dt>
+ <dd class="col-sm-8">
+ <span class="badge bg-secondary"><i class="fas fa-building me-1"></i>{data.Task.TypedParams.DataCenter}</span>
+ </dd>
+ }
+
+ if data.Task.Progress > 0 {
+ <dt class="col-sm-4">Progress:</dt>
+ <dd class="col-sm-8">
+ <div class="progress" style="height: 20px;">
+ <div class="progress-bar" role="progressbar"
+ style={fmt.Sprintf("width: %.1f%%", data.Task.Progress)}
+ aria-valuenow={fmt.Sprintf("%.1f", data.Task.Progress)}
+ aria-valuemin="0" aria-valuemax="100">
+ {fmt.Sprintf("%.1f%%", data.Task.Progress)}
+ </div>
+ </div>
+ </dd>
+ }
+ </dl>
+ </div>
+ </div>
+
+
+
+ if data.Task.DetailedReason != "" {
+ <div class="row mt-3">
+ <div class="col-12">
+ <h6>Detailed Reason:</h6>
+ <p class="text-muted">{data.Task.DetailedReason}</p>
+ </div>
+ </div>
+ }
+
+ if data.Task.Error != "" {
+ <div class="row mt-3">
+ <div class="col-12">
+ <h6>Error:</h6>
+ <div class="alert alert-danger">
+ <code>{data.Task.Error}</code>
+ </div>
+ </div>
+ </div>
+ }
+ </div>
+ </div>
+ </div>
+ </div>
+
+ <!-- Task Configuration Card -->
+ if data.Task.TypedParams != nil {
+ <div class="row mb-4">
+ <div class="col-12">
+ <div class="card">
+ <div class="card-header">
+ <h5 class="mb-0">
+ <i class="fas fa-cog me-2"></i>
+ Task Configuration
+ </h5>
+ </div>
+ <div class="card-body">
+ <!-- Source Servers (Unified) -->
+ if len(data.Task.TypedParams.Sources) > 0 {
+ <div class="mb-4">
+ <h6 class="text-info d-flex align-items-center">
+ <i class="fas fa-server me-2"></i>
+ Source Servers
+ <span class="badge bg-info ms-2">{fmt.Sprintf("%d", len(data.Task.TypedParams.Sources))}</span>
+ </h6>
+ <div class="bg-light p-3 rounded">
+ <div class="d-flex flex-column gap-2">
+ for i, source := range data.Task.TypedParams.Sources {
+ <div class="d-grid" style="grid-template-columns: auto 1fr auto auto auto auto; gap: 0.5rem; align-items: center;">
+ <span class="badge bg-primary">{fmt.Sprintf("#%d", i+1)}</span>
+ <code>{source.Node}</code>
+ <div>
+ if source.DataCenter != "" {
+ <small class="text-muted">
+ <i class="fas fa-building me-1"></i>{source.DataCenter}
+ </small>
+ }
+ </div>
+ <div>
+ if source.Rack != "" {
+ <small class="text-muted">
+ <i class="fas fa-server me-1"></i>{source.Rack}
+ </small>
+ }
+ </div>
+ <div>
+ if source.VolumeId > 0 {
+ <small class="text-muted">
+ <i class="fas fa-hdd me-1"></i>Vol:{fmt.Sprintf("%d", source.VolumeId)}
+ </small>
+ }
+ </div>
+ <div>
+ if len(source.ShardIds) > 0 {
+ <small class="text-muted">
+ <i class="fas fa-puzzle-piece me-1"></i>Shards:
+ for j, shardId := range source.ShardIds {
+ if j > 0 {
+ <span>, </span>
+ }
+ if shardId < erasure_coding.DataShardsCount {
+ <span class="badge badge-sm bg-primary ms-1" style="font-size: 0.65rem;" title={fmt.Sprintf("Data shard %d", shardId)}>{fmt.Sprintf("%d", shardId)}</span>
+ } else {
+ <span class="badge badge-sm bg-warning text-dark ms-1" style="font-size: 0.65rem;" title={fmt.Sprintf("Parity shard %d", shardId)}>{fmt.Sprintf("P%d", shardId-erasure_coding.DataShardsCount)}</span>
+ }
+ }
+ </small>
+ }
+ </div>
+ </div>
+ }
+ </div>
+ </div>
+ </div>
+ }
+
+ <!-- Task Flow Indicator -->
+ if len(data.Task.TypedParams.Sources) > 0 || len(data.Task.TypedParams.Targets) > 0 {
+ <div class="text-center mb-3">
+ <i class="fas fa-arrow-down text-primary" style="font-size: 1.5rem;"></i>
+ <br/>
+ <small class="text-muted">Task: {string(data.Task.Type)}</small>
+ </div>
+ }
+
+ <!-- Target/Destination (Generic) -->
+ if len(data.Task.TypedParams.Targets) > 0 {
+ <div class="mb-4">
+ <h6 class="text-success d-flex align-items-center">
+ <i class="fas fa-bullseye me-2"></i>
+ Target Servers
+ <span class="badge bg-success ms-2">{fmt.Sprintf("%d", len(data.Task.TypedParams.Targets))}</span>
+ </h6>
+ <div class="bg-light p-3 rounded">
+ <div class="d-flex flex-column gap-2">
+ for i, target := range data.Task.TypedParams.Targets {
+ <div class="d-grid" style="grid-template-columns: auto 1fr auto auto auto auto; gap: 0.5rem; align-items: center;">
+ <span class="badge bg-success">{fmt.Sprintf("#%d", i+1)}</span>
+ <code>{target.Node}</code>
+ <div>
+ if target.DataCenter != "" {
+ <small class="text-muted">
+ <i class="fas fa-building me-1"></i>{target.DataCenter}
+ </small>
+ }
+ </div>
+ <div>
+ if target.Rack != "" {
+ <small class="text-muted">
+ <i class="fas fa-server me-1"></i>{target.Rack}
+ </small>
+ }
+ </div>
+ <div>
+ if target.VolumeId > 0 {
+ <small class="text-muted">
+ <i class="fas fa-hdd me-1"></i>Vol:{fmt.Sprintf("%d", target.VolumeId)}
+ </small>
+ }
+ </div>
+ <div>
+ if len(target.ShardIds) > 0 {
+ <small class="text-muted">
+ <i class="fas fa-puzzle-piece me-1"></i>Shards:
+ for j, shardId := range target.ShardIds {
+ if j > 0 {
+ <span>, </span>
+ }
+ if shardId < erasure_coding.DataShardsCount {
+ <span class="badge badge-sm bg-primary ms-1" style="font-size: 0.65rem;" title={fmt.Sprintf("Data shard %d", shardId)}>{fmt.Sprintf("%d", shardId)}</span>
+ } else {
+ <span class="badge badge-sm bg-warning text-dark ms-1" style="font-size: 0.65rem;" title={fmt.Sprintf("Parity shard %d", shardId)}>{fmt.Sprintf("P%d", shardId-erasure_coding.DataShardsCount)}</span>
+ }
+ }
+ </small>
+ }
+ </div>
+ </div>
+ }
+ </div>
+ </div>
+ </div>
+ }
+ </div>
+ </div>
+ </div>
+ </div>
+ }
+
+ <!-- Worker Information Card -->
+ if data.WorkerInfo != nil {
+ <div class="row mb-4">
+ <div class="col-12">
+ <div class="card">
+ <div class="card-header">
+ <h5 class="mb-0">
+ <i class="fas fa-server me-2"></i>
+ Worker Information
+ </h5>
+ </div>
+ <div class="card-body">
+ <div class="row">
+ <div class="col-md-6">
+ <dl class="row">
+ <dt class="col-sm-4">Worker ID:</dt>
+ <dd class="col-sm-8"><code>{data.WorkerInfo.ID}</code></dd>
+
+ <dt class="col-sm-4">Address:</dt>
+ <dd class="col-sm-8"><code>{data.WorkerInfo.Address}</code></dd>
+
+ <dt class="col-sm-4">Status:</dt>
+ <dd class="col-sm-8">
+ if data.WorkerInfo.Status == "active" {
+ <span class="badge bg-success">Active</span>
+ } else if data.WorkerInfo.Status == "busy" {
+ <span class="badge bg-warning">Busy</span>
+ } else {
+ <span class="badge bg-secondary">Inactive</span>
+ }
+ </dd>
+ </dl>
+ </div>
+ <div class="col-md-6">
+ <dl class="row">
+ <dt class="col-sm-4">Last Heartbeat:</dt>
+ <dd class="col-sm-8">{data.WorkerInfo.LastHeartbeat.Format("2006-01-02 15:04:05")}</dd>
+
+ <dt class="col-sm-4">Current Load:</dt>
+ <dd class="col-sm-8">{fmt.Sprintf("%d/%d", data.WorkerInfo.CurrentLoad, data.WorkerInfo.MaxConcurrent)}</dd>
+
+ <dt class="col-sm-4">Capabilities:</dt>
+ <dd class="col-sm-8">
+ for _, capability := range data.WorkerInfo.Capabilities {
+ <span class="badge bg-info me-1">{string(capability)}</span>
+ }
+ </dd>
+ </dl>
+ </div>
+ </div>
+ </div>
+ </div>
+ </div>
+ </div>
+ }
+
+ <!-- Assignment History Card -->
+ if len(data.AssignmentHistory) > 0 {
+ <div class="row mb-4">
+ <div class="col-12">
+ <div class="card">
+ <div class="card-header">
+ <h5 class="mb-0">
+ <i class="fas fa-history me-2"></i>
+ Assignment History
+ </h5>
+ </div>
+ <div class="card-body">
+ <div class="table-responsive">
+ <table class="table table-striped">
+ <thead>
+ <tr>
+ <th>Worker ID</th>
+ <th>Worker Address</th>
+ <th>Assigned At</th>
+ <th>Unassigned At</th>
+ <th>Reason</th>
+ </tr>
+ </thead>
+ <tbody>
+ for _, assignment := range data.AssignmentHistory {
+ <tr>
+ <td><code>{assignment.WorkerID}</code></td>
+ <td><code>{assignment.WorkerAddress}</code></td>
+ <td>{assignment.AssignedAt.Format("2006-01-02 15:04:05")}</td>
+ <td>
+ if assignment.UnassignedAt != nil {
+ {assignment.UnassignedAt.Format("2006-01-02 15:04:05")}
+ } else {
+ <span class="text-muted">—</span>
+ }
+ </td>
+ <td>{assignment.Reason}</td>
+ </tr>
+ }
+ </tbody>
+ </table>
+ </div>
+ </div>
+ </div>
+ </div>
+ </div>
+ }
+
+ <!-- Execution Logs Card -->
+ if len(data.ExecutionLogs) > 0 {
+ <div class="row mb-4">
+ <div class="col-12">
+ <div class="card">
+ <div class="card-header">
+ <h5 class="mb-0">
+ <i class="fas fa-file-alt me-2"></i>
+ Execution Logs
+ </h5>
+ </div>
+ <div class="card-body">
+ <div class="table-responsive">
+ <table class="table table-striped table-sm">
+ <thead>
+ <tr>
+ <th width="150">Timestamp</th>
+ <th width="80">Level</th>
+ <th>Message</th>
+ <th>Details</th>
+ </tr>
+ </thead>
+ <tbody>
+ for _, log := range data.ExecutionLogs {
+ <tr>
+ <td><small>{log.Timestamp.Format("15:04:05")}</small></td>
+ <td>
+ if log.Level == "error" {
+ <span class="badge bg-danger">{log.Level}</span>
+ } else if log.Level == "warn" {
+ <span class="badge bg-warning">{log.Level}</span>
+ } else if log.Level == "info" {
+ <span class="badge bg-info">{log.Level}</span>
+ } else {
+ <span class="badge bg-secondary">{log.Level}</span>
+ }
+ </td>
+ <td><code>{log.Message}</code></td>
+ <td>
+ if log.Fields != nil && len(log.Fields) > 0 {
+ <small>
+ for _, k := range sortedKeys(log.Fields) {
+ <span class="badge bg-light text-dark me-1">{k}=<i>{log.Fields[k]}</i></span>
+ }
+ </small>
+ } else if log.Progress != nil || log.Status != "" {
+ <small>
+ if log.Progress != nil {
+ <span class="badge bg-secondary me-1">progress=<i>{fmt.Sprintf("%.0f%%", *log.Progress)}</i></span>
+ }
+ if log.Status != "" {
+ <span class="badge bg-secondary">status=<i>{log.Status}</i></span>
+ }
+ </small>
+ } else {
+ <span class="text-muted">-</span>
+ }
+ </td>
+ </tr>
+ }
+ </tbody>
+ </table>
+ </div>
+ </div>
+ </div>
+ </div>
+ </div>
+ }
+
+ <!-- Related Tasks Card -->
+ if len(data.RelatedTasks) > 0 {
+ <div class="row mb-4">
+ <div class="col-12">
+ <div class="card">
+ <div class="card-header">
+ <h5 class="mb-0">
+ <i class="fas fa-link me-2"></i>
+ Related Tasks
+ </h5>
+ </div>
+ <div class="card-body">
+ <div class="table-responsive">
+ <table class="table table-striped">
+ <thead>
+ <tr>
+ <th>Task ID</th>
+ <th>Type</th>
+ <th>Status</th>
+ <th>Volume ID</th>
+ <th>Server</th>
+ <th>Created</th>
+ </tr>
+ </thead>
+ <tbody>
+ for _, relatedTask := range data.RelatedTasks {
+ <tr>
+ <td>
+ <a href={fmt.Sprintf("/maintenance/tasks/%s", relatedTask.ID)}>
+ <code>{relatedTask.ID}</code>
+ </a>
+ </td>
+ <td><span class="badge bg-info">{string(relatedTask.Type)}</span></td>
+ <td>
+ if relatedTask.Status == maintenance.TaskStatusCompleted {
+ <span class="badge bg-success">Completed</span>
+ } else if relatedTask.Status == maintenance.TaskStatusFailed {
+ <span class="badge bg-danger">Failed</span>
+ } else if relatedTask.Status == maintenance.TaskStatusInProgress {
+ <span class="badge bg-warning">In Progress</span>
+ } else {
+ <span class="badge bg-secondary">{string(relatedTask.Status)}</span>
+ }
+ </td>
+ <td>
+ if relatedTask.VolumeID != 0 {
+ {fmt.Sprintf("%d", relatedTask.VolumeID)}
+ } else {
+ <span class="text-muted">-</span>
+ }
+ </td>
+ <td>
+ if relatedTask.Server != "" {
+ <code>{relatedTask.Server}</code>
+ } else {
+ <span class="text-muted">-</span>
+ }
+ </td>
+ <td><small>{relatedTask.CreatedAt.Format("2006-01-02 15:04:05")}</small></td>
+ </tr>
+ }
+ </tbody>
+ </table>
+ </div>
+ </div>
+ </div>
+ </div>
+ </div>
+ }
+
+ <!-- Actions Card -->
+ <div class="row mb-4">
+ <div class="col-12">
+ <div class="card">
+ <div class="card-header">
+ <h5 class="mb-0">
+ <i class="fas fa-cogs me-2"></i>
+ Actions
+ </h5>
+ </div>
+ <div class="card-body">
+ if data.Task.Status == maintenance.TaskStatusPending || data.Task.Status == maintenance.TaskStatusAssigned {
+ <button type="button" class="btn btn-danger me-2" data-task-id={data.Task.ID} onclick="cancelTask(this.getAttribute('data-task-id'))">
+ <i class="fas fa-times me-1"></i>
+ Cancel Task
+ </button>
+ }
+ if data.Task.WorkerID != "" {
+ <button type="button" class="btn btn-primary me-2" data-task-id={data.Task.ID} data-worker-id={data.Task.WorkerID} onclick="showTaskLogs(this.getAttribute('data-task-id'), this.getAttribute('data-worker-id'))">
+ <i class="fas fa-file-text me-1"></i>
+ Show Task Logs
+ </button>
+ }
+ <button type="button" class="btn btn-info" data-task-id={data.Task.ID} onclick="exportTaskDetail(this.getAttribute('data-task-id'))">
+ <i class="fas fa-download me-1"></i>
+ Export Details
+ </button>
+ </div>
+ </div>
+ </div>
+ </div>
+ </div>
+
+ <!-- Task Logs Modal -->
+ <div class="modal fade" id="taskLogsModal" tabindex="-1" aria-labelledby="taskLogsModalLabel" aria-hidden="true">
+ <div class="modal-dialog modal-xl">
+ <div class="modal-content">
+ <div class="modal-header">
+ <h5 class="modal-title" id="taskLogsModalLabel">
+ <i class="fas fa-file-text me-2"></i>Task Logs
+ </h5>
+ <button type="button" class="btn-close" data-bs-dismiss="modal" aria-label="Close"></button>
+ </div>
+ <div class="modal-body">
+ <div id="logsLoadingSpinner" class="text-center py-4" style="display: none;">
+ <div class="spinner-border text-primary" role="status">
+ <span class="visually-hidden">Loading logs...</span>
+ </div>
+ <p class="mt-2">Fetching logs from worker...</p>
+ </div>
+
+ <div id="logsError" class="alert alert-danger" style="display: none;">
+ <i class="fas fa-exclamation-triangle me-2"></i>
+ <span id="logsErrorMessage"></span>
+ </div>
+
+ <div id="logsContent" style="display: none;">
+ <div class="d-flex justify-content-between align-items-center mb-3">
+ <div>
+ <strong>Task:</strong> <span id="logsTaskId"></span> |
+ <strong>Worker:</strong> <span id="logsWorkerId"></span> |
+ <strong>Entries:</strong> <span id="logsCount"></span>
+ </div>
+ <div class="btn-group">
+ <button type="button" class="btn btn-sm btn-outline-primary" onclick="refreshModalLogs()">
+ <i class="fas fa-sync-alt me-1"></i>Refresh
+ </button>
+ <button type="button" class="btn btn-sm btn-outline-success" onclick="downloadTaskLogs()">
+ <i class="fas fa-download me-1"></i>Download
+ </button>
+ </div>
+ </div>
+
+ <div class="card">
+ <div class="card-header">
+ <div class="d-flex justify-content-between align-items-center">
+ <span>Log Entries (Last 100)</span>
+ <small class="text-muted">Newest entries first</small>
+ </div>
+ </div>
+ <div class="card-body p-0">
+ <pre id="logsDisplay" class="bg-dark text-light p-3 mb-0" style="max-height: 400px; overflow-y: auto; font-size: 0.85rem; line-height: 1.4;"></pre>
+ </div>
+ </div>
+ </div>
+ </div>
+ <div class="modal-footer">
+ <button type="button" class="btn btn-secondary" data-bs-dismiss="modal">Close</button>
+ </div>
+ </div>
+ </div>
+ </div>
+
+ <style>
+ .timeline-container {
+ position: relative;
+ padding: 20px 0;
+ }
+
+ .timeline-progress {
+ display: flex;
+ justify-content: space-between;
+ align-items: flex-start;
+ position: relative;
+ max-width: 100%;
+ }
+
+ .timeline-step {
+ display: flex;
+ flex-direction: column;
+ align-items: center;
+ flex: 1;
+ position: relative;
+ }
+
+ .timeline-circle {
+ width: 40px;
+ height: 40px;
+ border-radius: 50%;
+ display: flex;
+ align-items: center;
+ justify-content: center;
+ color: white;
+ font-weight: bold;
+ box-shadow: 0 2px 4px rgba(0,0,0,0.1);
+ z-index: 2;
+ position: relative;
+ }
+
+ .timeline-circle.completed {
+ background-color: #28a745;
+ border: 3px solid #1e7e34;
+ }
+
+ .timeline-circle.pending {
+ background-color: #6c757d;
+ border: 3px solid #495057;
+ }
+
+ .timeline-connector {
+ position: absolute;
+ top: 20px;
+ left: 50%;
+ right: -50%;
+ height: 4px;
+ z-index: 1;
+ margin-left: 20px;
+ margin-right: 20px;
+ }
+
+ .timeline-connector.completed {
+ background-color: #28a745;
+ }
+
+ .timeline-connector:not(.completed) {
+ background-color: #dee2e6;
+ }
+
+ .timeline-step:last-child .timeline-connector {
+ display: none;
+ }
+
+ .timeline-label {
+ margin-top: 15px;
+ text-align: center;
+ min-height: 60px;
+ }
+
+ .timeline-label strong {
+ display: block;
+ font-size: 0.9rem;
+ margin-bottom: 4px;
+ }
+
+ .timeline-label small {
+ font-size: 0.75rem;
+ line-height: 1.2;
+ }
+
+ @media (max-width: 768px) {
+ .timeline-progress {
+ flex-direction: column;
+ align-items: stretch;
+ }
+
+ .timeline-step {
+ flex-direction: row;
+ align-items: center;
+ margin-bottom: 20px;
+ }
+
+ .timeline-circle {
+ margin-right: 15px;
+ flex-shrink: 0;
+ }
+
+ .timeline-connector {
+ display: none;
+ }
+
+ .timeline-label {
+ text-align: left;
+ margin-top: 0;
+ min-height: auto;
+ }
+ }
+ </style>
+
+ <script>
+ // Global variables for current logs modal
+ let currentTaskId = '';
+ let currentWorkerId = '';
+
+ function refreshPage() {
+ location.reload();
+ }
+
+ function showTaskLogs(taskId, workerId) {
+ currentTaskId = taskId;
+ currentWorkerId = workerId;
+
+ // Show the modal
+ const modal = new bootstrap.Modal(document.getElementById('taskLogsModal'));
+ modal.show();
+
+ // Load logs
+ loadTaskLogs(taskId, workerId);
+ }
+
+ function loadTaskLogs(taskId, workerId) {
+ // Show loading spinner
+ document.getElementById('logsLoadingSpinner').style.display = 'block';
+ document.getElementById('logsError').style.display = 'none';
+ document.getElementById('logsContent').style.display = 'none';
+
+ // Update modal info
+ document.getElementById('logsTaskId').textContent = taskId;
+ document.getElementById('logsWorkerId').textContent = workerId;
+
+ // Fetch logs from the API
+ fetch(`/api/maintenance/workers/${workerId}/logs?taskId=${taskId}&maxEntries=100`)
+ .then(response => response.json())
+ .then(data => {
+ document.getElementById('logsLoadingSpinner').style.display = 'none';
+
+ if (data.error) {
+ showLogsError(data.error);
+ return;
+ }
+
+ // Display logs
+ displayLogs(data.logs, data.count || 0);
+ })
+ .catch(error => {
+ document.getElementById('logsLoadingSpinner').style.display = 'none';
+ showLogsError('Failed to fetch logs: ' + error.message);
+ });
+ }
+
+ function displayLogs(logs, count) {
+ document.getElementById('logsError').style.display = 'none';
+ document.getElementById('logsContent').style.display = 'block';
+ document.getElementById('logsCount').textContent = count;
+
+ const logsDisplay = document.getElementById('logsDisplay');
+
+ if (!logs || logs.length === 0) {
+ logsDisplay.textContent = 'No logs found for this task.';
+ return;
+ }
+
+ // Format and display logs with structured fields
+ let logText = '';
+ logs.forEach(entry => {
+ const timestamp = entry.timestamp ? new Date(entry.timestamp * 1000).toISOString() : 'N/A';
+ const level = entry.level || 'INFO';
+ const message = entry.message || '';
+
+ logText += `[${timestamp}] ${level}: ${message}`;
+
+ // Add structured fields if they exist
+ if (entry.fields && Object.keys(entry.fields).length > 0) {
+ const fieldsStr = Object.entries(entry.fields)
+ .map(([key, value]) => `${key}=${value}`)
+ .join(', ');
+ logText += ` | ${fieldsStr}`;
+ }
+
+ // Add progress if available
+ if (entry.progress !== undefined && entry.progress !== null) {
+ logText += ` | progress=${entry.progress}%`;
+ }
+
+ // Add status if available
+ if (entry.status) {
+ logText += ` | status=${entry.status}`;
+ }
+
+ logText += '\n';
+ });
+
+ logsDisplay.textContent = logText;
+
+ // Scroll to top
+ logsDisplay.scrollTop = 0;
+ }
+
+ function showLogsError(errorMessage) {
+ document.getElementById('logsError').style.display = 'block';
+ document.getElementById('logsContent').style.display = 'none';
+ document.getElementById('logsErrorMessage').textContent = errorMessage;
+ }
+
+ function refreshModalLogs() {
+ if (currentTaskId && currentWorkerId) {
+ loadTaskLogs(currentTaskId, currentWorkerId);
+ }
+ }
+
+ function downloadTaskLogs() {
+ if (!currentTaskId || !currentWorkerId) {
+ alert('No task logs to download');
+ return;
+ }
+
+ // Download all logs (without maxEntries limit)
+ const downloadUrl = `/api/maintenance/workers/${currentWorkerId}/logs?taskId=${currentTaskId}&maxEntries=0`;
+
+ fetch(downloadUrl)
+ .then(response => response.json())
+ .then(data => {
+ if (data.error) {
+ alert('Error downloading logs: ' + data.error);
+ return;
+ }
+
+ // Convert logs to text format with structured fields
+ let logContent = '';
+ if (data.logs && data.logs.length > 0) {
+ data.logs.forEach(entry => {
+ const timestamp = entry.timestamp ? new Date(entry.timestamp * 1000).toISOString() : 'N/A';
+ const level = entry.level || 'INFO';
+ const message = entry.message || '';
+
+ logContent += `[${timestamp}] ${level}: ${message}`;
+
+ // Add structured fields if they exist
+ if (entry.fields && Object.keys(entry.fields).length > 0) {
+ const fieldsStr = Object.entries(entry.fields)
+ .map(([key, value]) => `${key}=${value}`)
+ .join(', ');
+ logContent += ` | ${fieldsStr}`;
+ }
+
+ // Add progress if available
+ if (entry.progress !== undefined && entry.progress !== null) {
+ logContent += ` | progress=${entry.progress}%`;
+ }
+
+ // Add status if available
+ if (entry.status) {
+ logContent += ` | status=${entry.status}`;
+ }
+
+ logContent += '\n';
+ });
+ } else {
+ logContent = 'No logs found for this task.';
+ }
+
+ // Create and download file
+ const blob = new Blob([logContent], { type: 'text/plain' });
+ const url = URL.createObjectURL(blob);
+ const link = document.createElement('a');
+ link.href = url;
+ link.download = `task-${currentTaskId}-logs.txt`;
+ link.click();
+ URL.revokeObjectURL(url);
+ })
+ .catch(error => {
+ alert('Error downloading logs: ' + error.message);
+ });
+ }
+
+ function cancelTask(taskId) {
+ if (confirm('Are you sure you want to cancel this task?')) {
+ fetch(`/api/maintenance/tasks/${taskId}/cancel`, {
+ method: 'POST',
+ headers: {
+ 'Content-Type': 'application/json',
+ },
+ })
+ .then(response => response.json())
+ .then(data => {
+ if (data.success) {
+ alert('Task cancelled successfully');
+ location.reload();
+ } else {
+ alert('Error cancelling task: ' + data.error);
+ }
+ })
+ .catch(error => {
+ console.error('Error:', error);
+ alert('Error cancelling task');
+ });
+ }
+ }
+
+ function refreshTaskLogs(taskId) {
+ fetch(`/api/maintenance/tasks/${taskId}/detail`)
+ .then(response => response.json())
+ .then(data => {
+ location.reload();
+ })
+ .catch(error => {
+ console.error('Error:', error);
+ alert('Error refreshing logs');
+ });
+ }
+
+ function exportTaskDetail(taskId) {
+ fetch(`/api/maintenance/tasks/${taskId}/detail`)
+ .then(response => response.json())
+ .then(data => {
+ const dataStr = JSON.stringify(data, null, 2);
+ const dataBlob = new Blob([dataStr], {type: 'application/json'});
+ const url = URL.createObjectURL(dataBlob);
+ const link = document.createElement('a');
+ link.href = url;
+ link.download = `task-${taskId}-detail.json`;
+ link.click();
+ URL.revokeObjectURL(url);
+ })
+ .catch(error => {
+ console.error('Error:', error);
+ alert('Error exporting task detail');
+ });
+ }
+
+ // Auto-refresh every 30 seconds for active tasks
+ if ('{string(data.Task.Status)}' === 'in_progress') {
+ setInterval(refreshPage, 30000);
+ }
+ </script>
+}
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, "<div class=\"container-fluid\"><!-- Header --><div class=\"row mb-4\"><div class=\"col-12\"><div class=\"d-flex justify-content-between align-items-center\"><div><nav aria-label=\"breadcrumb\"><ol class=\"breadcrumb mb-1\"><li class=\"breadcrumb-item\"><a href=\"/maintenance\">Maintenance</a></li><li class=\"breadcrumb-item active\" aria-current=\"page\">Task Detail</li></ol></nav><h2 class=\"mb-0\"><i class=\"fas fa-tasks me-2\"></i> 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, "</h2></div><div class=\"btn-group\"><button type=\"button\" class=\"btn btn-secondary\" onclick=\"history.back()\"><i class=\"fas fa-arrow-left me-1\"></i> Back</button> <button type=\"button\" class=\"btn btn-secondary\" onclick=\"refreshPage()\"><i class=\"fas fa-sync-alt me-1\"></i> Refresh</button></div></div></div></div><!-- Task Overview Card --><div class=\"row mb-4\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header\"><h5 class=\"mb-0\"><i class=\"fas fa-info-circle me-2\"></i> Task Overview</h5></div><div class=\"card-body\"><div class=\"row\"><div class=\"col-md-6\"><dl class=\"row\"><dt class=\"col-sm-4\">Task ID:</dt><dd class=\"col-sm-8\"><code>")
+ 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, "</code></dd><dt class=\"col-sm-4\">Type:</dt><dd class=\"col-sm-8\"><span class=\"badge bg-info\">")
+ 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, "</span></dd><dt class=\"col-sm-4\">Status:</dt><dd class=\"col-sm-8\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if data.Task.Status == maintenance.TaskStatusPending {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 5, "<span class=\"badge bg-secondary\">Pending</span>")
+ 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, "<span class=\"badge bg-info\">Assigned</span>")
+ 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, "<span class=\"badge bg-warning\">In Progress</span>")
+ 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, "<span class=\"badge bg-success\">Completed</span>")
+ 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, "<span class=\"badge bg-danger\">Failed</span>")
+ 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, "<span class=\"badge bg-dark\">Cancelled</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 11, "</dd><dt class=\"col-sm-4\">Priority:</dt><dd class=\"col-sm-8\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if data.Task.Priority == maintenance.PriorityHigh {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 12, "<span class=\"badge bg-danger\">High</span>")
+ 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, "<span class=\"badge bg-danger\">Critical</span>")
+ 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, "<span class=\"badge bg-warning\">Normal</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 15, "<span class=\"badge bg-secondary\">Low</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 16, "</dd>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if data.Task.Reason != "" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 17, "<dt class=\"col-sm-4\">Reason:</dt><dd class=\"col-sm-8\"><span class=\"text-muted\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var5 string
+ templ_7745c5c3_Var5, templ_7745c5c3_Err = templ.JoinStringErrs(data.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, "</span></dd>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 19, "</dl></div><div class=\"col-md-6\"><!-- Task Timeline --><div class=\"mb-3\"><h6 class=\"text-primary mb-3\"><i class=\"fas fa-clock me-1\"></i>Task Timeline</h6><div class=\"timeline-container\"><div class=\"timeline-progress\"><div class=\"timeline-step\" data-step=\"created\"><div class=\"timeline-circle completed\"><i class=\"fas fa-plus\"></i></div><div class=\"timeline-connector completed\"></div><div class=\"timeline-label\"><strong>Created</strong> <small class=\"d-block text-muted\">")
+ 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, "</small></div></div><div class=\"timeline-step\" data-step=\"scheduled\"><div class=\"timeline-circle completed\"><i class=\"fas fa-calendar\"></i></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if data.Task.StartedAt != nil {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 21, "<div class=\"timeline-connector completed\"></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 22, "<div class=\"timeline-connector\"></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 23, "<div class=\"timeline-label\"><strong>Scheduled</strong> <small class=\"d-block text-muted\">")
+ 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, "</small></div></div><div class=\"timeline-step\" data-step=\"started\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if data.Task.StartedAt != nil {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 25, "<div class=\"timeline-circle completed\"><i class=\"fas fa-play\"></i></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 26, "<div class=\"timeline-circle pending\"><i class=\"fas fa-clock\"></i></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ if data.Task.CompletedAt != nil {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 27, "<div class=\"timeline-connector completed\"></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 28, "<div class=\"timeline-connector\"></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 29, "<div class=\"timeline-label\"><strong>Started</strong> <small class=\"d-block text-muted\">")
+ 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, "</small></div></div><div class=\"timeline-step\" data-step=\"completed\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if data.Task.CompletedAt != nil {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 32, "<div class=\"timeline-circle completed\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if data.Task.Status == maintenance.TaskStatusCompleted {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 33, "<i class=\"fas fa-check\"></i>")
+ 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, "<i class=\"fas fa-times\"></i>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 35, "<i class=\"fas fa-stop\"></i>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 36, "</div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 37, "<div class=\"timeline-circle pending\"><i class=\"fas fa-hourglass-half\"></i></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 38, "<div class=\"timeline-label\"><strong>")
+ 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, "</strong> <small class=\"d-block text-muted\">")
+ 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, "</small></div></div></div></div></div><!-- Additional Info -->")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if data.Task.WorkerID != "" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 46, "<dl class=\"row\"><dt class=\"col-sm-4\">Worker:</dt><dd class=\"col-sm-8\"><code>")
+ 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, "</code></dd></dl>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 48, "<dl class=\"row\">")
+ 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, "<dt class=\"col-sm-4\">Volume Size:</dt><dd class=\"col-sm-8\"><span class=\"badge bg-primary\">")
+ 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, "</span></dd>")
+ 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, "<dt class=\"col-sm-4\">Collection:</dt><dd class=\"col-sm-8\"><span class=\"badge bg-info\"><i class=\"fas fa-folder me-1\"></i>")
+ 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, "</span></dd>")
+ 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, "<dt class=\"col-sm-4\">Data Center:</dt><dd class=\"col-sm-8\"><span class=\"badge bg-secondary\"><i class=\"fas fa-building me-1\"></i>")
+ 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, "</span></dd>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ if data.Task.Progress > 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 55, "<dt class=\"col-sm-4\">Progress:</dt><dd class=\"col-sm-8\"><div class=\"progress\" style=\"height: 20px;\"><div class=\"progress-bar\" role=\"progressbar\" style=\"")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var14 string
+ templ_7745c5c3_Var14, templ_7745c5c3_Err = templruntime.SanitizeStyleAttributeValues(fmt.Sprintf("width: %.1f%%", data.Task.Progress))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 249, Col: 108}
+ }
+ _, 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, 56, "\" aria-valuenow=\"")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var15 string
+ templ_7745c5c3_Var15, 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: 250, Col: 107}
+ }
+ _, 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, 57, "\" aria-valuemin=\"0\" aria-valuemax=\"100\">")
+ 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, "</div></div></dd>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 59, "</dl></div></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if data.Task.DetailedReason != "" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 60, "<div class=\"row mt-3\"><div class=\"col-12\"><h6>Detailed Reason:</h6><p class=\"text-muted\">")
+ 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, "</p></div></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ if data.Task.Error != "" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 62, "<div class=\"row mt-3\"><div class=\"col-12\"><h6>Error:</h6><div class=\"alert alert-danger\"><code>")
+ 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, "</code></div></div></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 64, "</div></div></div></div><!-- Task Configuration Card -->")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if data.Task.TypedParams != nil {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 65, "<div class=\"row mb-4\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header\"><h5 class=\"mb-0\"><i class=\"fas fa-cog me-2\"></i> Task Configuration</h5></div><div class=\"card-body\"><!-- Source Servers (Unified) -->")
+ 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, "<div class=\"mb-4\"><h6 class=\"text-info d-flex align-items-center\"><i class=\"fas fa-server me-2\"></i> Source Servers <span class=\"badge bg-info ms-2\">")
+ 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, "</span></h6><div class=\"bg-light p-3 rounded\"><div class=\"d-flex flex-column gap-2\">")
+ 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, "<div class=\"d-grid\" style=\"grid-template-columns: auto 1fr auto auto auto auto; gap: 0.5rem; align-items: center;\"><span class=\"badge bg-primary\">")
+ 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, "</span> <code>")
+ 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, "</code><div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if source.DataCenter != "" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 71, "<small class=\"text-muted\"><i class=\"fas fa-building me-1\"></i>")
+ 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, "</small>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 73, "</div><div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if source.Rack != "" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 74, "<small class=\"text-muted\"><i class=\"fas fa-server me-1\"></i>")
+ 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, "</small>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 76, "</div><div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if source.VolumeId > 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 77, "<small class=\"text-muted\"><i class=\"fas fa-hdd me-1\"></i>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, "</small>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 79, "</div><div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if len(source.ShardIds) > 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 80, "<small class=\"text-muted\"><i class=\"fas fa-puzzle-piece me-1\"></i>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, "<span>, </span>")
+ 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, "<span class=\"badge badge-sm bg-primary ms-1\" style=\"font-size: 0.65rem;\" title=\"")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var25 string
+ templ_7745c5c3_Var25, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("Data shard %d", shardId))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 343, Col: 173}
+ }
+ _, 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, 84, "\">")
+ 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, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 86, "<span class=\"badge badge-sm bg-warning text-dark ms-1\" style=\"font-size: 0.65rem;\" title=\"")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var27 string
+ templ_7745c5c3_Var27, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("Parity shard %d", shardId))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 345, Col: 185}
+ }
+ _, 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, 87, "\">")
+ 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, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 89, "</small>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 90, "</div></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 91, "</div></div></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 92, "<!-- Task Flow Indicator -->")
+ 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, "<div class=\"text-center mb-3\"><i class=\"fas fa-arrow-down text-primary\" style=\"font-size: 1.5rem;\"></i><br><small class=\"text-muted\">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, "</small></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 95, "<!-- Target/Destination (Generic) -->")
+ 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, "<div class=\"mb-4\"><h6 class=\"text-success d-flex align-items-center\"><i class=\"fas fa-bullseye me-2\"></i> Target Servers <span class=\"badge bg-success ms-2\">")
+ 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, "</span></h6><div class=\"bg-light p-3 rounded\"><div class=\"d-flex flex-column gap-2\">")
+ 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, "<div class=\"d-grid\" style=\"grid-template-columns: auto 1fr auto auto auto auto; gap: 0.5rem; align-items: center;\"><span class=\"badge bg-success\">")
+ 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, "</span> <code>")
+ 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, "</code><div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if target.DataCenter != "" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 101, "<small class=\"text-muted\"><i class=\"fas fa-building me-1\"></i>")
+ 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, "</small>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 103, "</div><div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if target.Rack != "" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 104, "<small class=\"text-muted\"><i class=\"fas fa-server me-1\"></i>")
+ 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, "</small>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 106, "</div><div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if target.VolumeId > 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 107, "<small class=\"text-muted\"><i class=\"fas fa-hdd me-1\"></i>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, "</small>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 109, "</div><div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if len(target.ShardIds) > 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 110, "<small class=\"text-muted\"><i class=\"fas fa-puzzle-piece me-1\"></i>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, "<span>, </span>")
+ 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, "<span class=\"badge badge-sm bg-primary ms-1\" style=\"font-size: 0.65rem;\" title=\"")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var36 string
+ templ_7745c5c3_Var36, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("Data shard %d", shardId))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 411, Col: 173}
+ }
+ _, 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, 114, "\">")
+ 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, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 116, "<span class=\"badge badge-sm bg-warning text-dark ms-1\" style=\"font-size: 0.65rem;\" title=\"")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var38 string
+ templ_7745c5c3_Var38, templ_7745c5c3_Err = templ.JoinStringErrs(fmt.Sprintf("Parity shard %d", shardId))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 413, Col: 185}
+ }
+ _, 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, 117, "\">")
+ 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, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 119, "</small>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 120, "</div></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 121, "</div></div></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 122, "</div></div></div></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 123, "<!-- Worker Information Card -->")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if data.WorkerInfo != nil {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 124, "<div class=\"row mb-4\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header\"><h5 class=\"mb-0\"><i class=\"fas fa-server me-2\"></i> Worker Information</h5></div><div class=\"card-body\"><div class=\"row\"><div class=\"col-md-6\"><dl class=\"row\"><dt class=\"col-sm-4\">Worker ID:</dt><dd class=\"col-sm-8\"><code>")
+ 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, "</code></dd><dt class=\"col-sm-4\">Address:</dt><dd class=\"col-sm-8\"><code>")
+ 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, "</code></dd><dt class=\"col-sm-4\">Status:</dt><dd class=\"col-sm-8\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if data.WorkerInfo.Status == "active" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 127, "<span class=\"badge bg-success\">Active</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else if data.WorkerInfo.Status == "busy" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 128, "<span class=\"badge bg-warning\">Busy</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 129, "<span class=\"badge bg-secondary\">Inactive</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 130, "</dd></dl></div><div class=\"col-md-6\"><dl class=\"row\"><dt class=\"col-sm-4\">Last Heartbeat:</dt><dd class=\"col-sm-8\">")
+ 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, "</dd><dt class=\"col-sm-4\">Current Load:</dt><dd class=\"col-sm-8\">")
+ 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, "</dd><dt class=\"col-sm-4\">Capabilities:</dt><dd class=\"col-sm-8\">")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ for _, capability := range data.WorkerInfo.Capabilities {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 133, "<span class=\"badge bg-info me-1\">")
+ 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, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 135, "</dd></dl></div></div></div></div></div></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 136, "<!-- Assignment History Card -->")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if len(data.AssignmentHistory) > 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 137, "<div class=\"row mb-4\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header\"><h5 class=\"mb-0\"><i class=\"fas fa-history me-2\"></i> Assignment History</h5></div><div class=\"card-body\"><div class=\"table-responsive\"><table class=\"table table-striped\"><thead><tr><th>Worker ID</th><th>Worker Address</th><th>Assigned At</th><th>Unassigned At</th><th>Reason</th></tr></thead> <tbody>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ for _, assignment := range data.AssignmentHistory {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 138, "<tr><td><code>")
+ 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, "</code></td><td><code>")
+ 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, "</code></td><td>")
+ 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, "</td><td>")
+ 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, "<span class=\"text-muted\">—</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 143, "</td><td>")
+ 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, "</td></tr>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 145, "</tbody></table></div></div></div></div></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 146, "<!-- Execution Logs Card -->")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if len(data.ExecutionLogs) > 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 147, "<div class=\"row mb-4\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header\"><h5 class=\"mb-0\"><i class=\"fas fa-file-alt me-2\"></i> Execution Logs</h5></div><div class=\"card-body\"><div class=\"table-responsive\"><table class=\"table table-striped table-sm\"><thead><tr><th width=\"150\">Timestamp</th><th width=\"80\">Level</th><th>Message</th><th>Details</th></tr></thead> <tbody>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ for _, log := range data.ExecutionLogs {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 148, "<tr><td><small>")
+ 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, "</small></td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if log.Level == "error" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 150, "<span class=\"badge bg-danger\">")
+ 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, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else if log.Level == "warn" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 152, "<span class=\"badge bg-warning\">")
+ 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, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else if log.Level == "info" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 154, "<span class=\"badge bg-info\">")
+ 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, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 156, "<span class=\"badge bg-secondary\">")
+ 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, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 158, "</td><td><code>")
+ 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, "</code></td><td>")
+ 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, "<small>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ for _, k := range sortedKeys(log.Fields) {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 161, "<span class=\"badge bg-light text-dark me-1\">")
+ 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, "=<i>")
+ 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, "</i></span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 164, "</small>")
+ 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, "<small>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if log.Progress != nil {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 166, "<span class=\"badge bg-secondary me-1\">progress=<i>")
+ 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, "</i></span> ")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ if log.Status != "" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 168, "<span class=\"badge bg-secondary\">status=<i>")
+ 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, "</i></span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 170, "</small>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 171, "<span class=\"text-muted\">-</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 172, "</td></tr>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 173, "</tbody></table></div></div></div></div></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 174, "<!-- Related Tasks Card -->")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if len(data.RelatedTasks) > 0 {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 175, "<div class=\"row mb-4\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header\"><h5 class=\"mb-0\"><i class=\"fas fa-link me-2\"></i> Related Tasks</h5></div><div class=\"card-body\"><div class=\"table-responsive\"><table class=\"table table-striped\"><thead><tr><th>Task ID</th><th>Type</th><th>Status</th><th>Volume ID</th><th>Server</th><th>Created</th></tr></thead> <tbody>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ for _, relatedTask := range data.RelatedTasks {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 176, "<tr><td><a href=\"")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var60 templ.SafeURL
+ templ_7745c5c3_Var60, templ_7745c5c3_Err = templ.JoinURLErrs(fmt.Sprintf("/maintenance/tasks/%s", relatedTask.ID))
+ if templ_7745c5c3_Err != nil {
+ return templ.Error{Err: templ_7745c5c3_Err, FileName: `view/app/task_detail.templ`, Line: 632, Col: 113}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var60))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 177, "\"><code>")
+ 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, "</code></a></td><td><span class=\"badge bg-info\">")
+ 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, "</span></td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if relatedTask.Status == maintenance.TaskStatusCompleted {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 180, "<span class=\"badge bg-success\">Completed</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else if relatedTask.Status == maintenance.TaskStatusFailed {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 181, "<span class=\"badge bg-danger\">Failed</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else if relatedTask.Status == maintenance.TaskStatusInProgress {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 182, "<span class=\"badge bg-warning\">In Progress</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 183, "<span class=\"badge bg-secondary\">")
+ 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, "</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 185, "</td><td>")
+ 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, "<span class=\"text-muted\">-</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 187, "</td><td>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ if relatedTask.Server != "" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 188, "<code>")
+ 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, "</code>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ } else {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 190, "<span class=\"text-muted\">-</span>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 191, "</td><td><small>")
+ 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, "</small></td></tr>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 193, "</tbody></table></div></div></div></div></div>")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 194, "<!-- Actions Card --><div class=\"row mb-4\"><div class=\"col-12\"><div class=\"card\"><div class=\"card-header\"><h5 class=\"mb-0\"><i class=\"fas fa-cogs me-2\"></i> Actions</h5></div><div class=\"card-body\">")
+ 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, "<button type=\"button\" class=\"btn btn-danger me-2\" data-task-id=\"")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var67 string
+ templ_7745c5c3_Var67, 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: 686, Col: 104}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var67))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 196, "\" onclick=\"cancelTask(this.getAttribute('data-task-id'))\"><i class=\"fas fa-times me-1\"></i> Cancel Task</button> ")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ if data.Task.WorkerID != "" {
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 197, "<button type=\"button\" class=\"btn btn-primary me-2\" data-task-id=\"")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var68 string
+ templ_7745c5c3_Var68, 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: 692, Col: 105}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var68))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 198, "\" data-worker-id=\"")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var69 string
+ templ_7745c5c3_Var69, 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: 692, Col: 141}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var69))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 199, "\" onclick=\"showTaskLogs(this.getAttribute('data-task-id'), this.getAttribute('data-worker-id'))\"><i class=\"fas fa-file-text me-1\"></i> Show Task Logs</button> ")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 200, "<button type=\"button\" class=\"btn btn-info\" data-task-id=\"")
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ var templ_7745c5c3_Var70 string
+ templ_7745c5c3_Var70, 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: 697, Col: 93}
+ }
+ _, templ_7745c5c3_Err = templ_7745c5c3_Buffer.WriteString(templ.EscapeString(templ_7745c5c3_Var70))
+ if templ_7745c5c3_Err != nil {
+ return templ_7745c5c3_Err
+ }
+ templ_7745c5c3_Err = templruntime.WriteString(templ_7745c5c3_Buffer, 201, "\" onclick=\"exportTaskDetail(this.getAttribute('data-task-id'))\"><i class=\"fas fa-download me-1\"></i> Export Details</button></div></div></div></div></div><!-- Task Logs Modal --><div class=\"modal fade\" id=\"taskLogsModal\" tabindex=\"-1\" aria-labelledby=\"taskLogsModalLabel\" aria-hidden=\"true\"><div class=\"modal-dialog modal-xl\"><div class=\"modal-content\"><div class=\"modal-header\"><h5 class=\"modal-title\" id=\"taskLogsModalLabel\"><i class=\"fas fa-file-text me-2\"></i>Task Logs</h5><button type=\"button\" class=\"btn-close\" data-bs-dismiss=\"modal\" aria-label=\"Close\"></button></div><div class=\"modal-body\"><div id=\"logsLoadingSpinner\" class=\"text-center py-4\" style=\"display: none;\"><div class=\"spinner-border text-primary\" role=\"status\"><span class=\"visually-hidden\">Loading logs...</span></div><p class=\"mt-2\">Fetching logs from worker...</p></div><div id=\"logsError\" class=\"alert alert-danger\" style=\"display: none;\"><i class=\"fas fa-exclamation-triangle me-2\"></i> <span id=\"logsErrorMessage\"></span></div><div id=\"logsContent\" style=\"display: none;\"><div class=\"d-flex justify-content-between align-items-center mb-3\"><div><strong>Task:</strong> <span id=\"logsTaskId\"></span> | <strong>Worker:</strong> <span id=\"logsWorkerId\"></span> | <strong>Entries:</strong> <span id=\"logsCount\"></span></div><div class=\"btn-group\"><button type=\"button\" class=\"btn btn-sm btn-outline-primary\" onclick=\"refreshModalLogs()\"><i class=\"fas fa-sync-alt me-1\"></i>Refresh</button> <button type=\"button\" class=\"btn btn-sm btn-outline-success\" onclick=\"downloadTaskLogs()\"><i class=\"fas fa-download me-1\"></i>Download</button></div></div><div class=\"card\"><div class=\"card-header\"><div class=\"d-flex justify-content-between align-items-center\"><span>Log Entries (Last 100)</span> <small class=\"text-muted\">Newest entries first</small></div></div><div class=\"card-body p-0\"><pre id=\"logsDisplay\" class=\"bg-dark text-light p-3 mb-0\" style=\"max-height: 400px; overflow-y: auto; font-size: 0.85rem; line-height: 1.4;\"></pre></div></div></div></div><div class=\"modal-footer\"><button type=\"button\" class=\"btn btn-secondary\" data-bs-dismiss=\"modal\">Close</button></div></div></div></div><style>\n .timeline-container {\n position: relative;\n padding: 20px 0;\n }\n \n .timeline-progress {\n display: flex;\n justify-content: space-between;\n align-items: flex-start;\n position: relative;\n max-width: 100%;\n }\n \n .timeline-step {\n display: flex;\n flex-direction: column;\n align-items: center;\n flex: 1;\n position: relative;\n }\n \n .timeline-circle {\n width: 40px;\n height: 40px;\n border-radius: 50%;\n display: flex;\n align-items: center;\n justify-content: center;\n color: white;\n font-weight: bold;\n box-shadow: 0 2px 4px rgba(0,0,0,0.1);\n z-index: 2;\n position: relative;\n }\n \n .timeline-circle.completed {\n background-color: #28a745;\n border: 3px solid #1e7e34;\n }\n \n .timeline-circle.pending {\n background-color: #6c757d;\n border: 3px solid #495057;\n }\n \n .timeline-connector {\n position: absolute;\n top: 20px;\n left: 50%;\n right: -50%;\n height: 4px;\n z-index: 1;\n margin-left: 20px;\n margin-right: 20px;\n }\n \n .timeline-connector.completed {\n background-color: #28a745;\n }\n \n .timeline-connector:not(.completed) {\n background-color: #dee2e6;\n }\n \n .timeline-step:last-child .timeline-connector {\n display: none;\n }\n \n .timeline-label {\n margin-top: 15px;\n text-align: center;\n min-height: 60px;\n }\n \n .timeline-label strong {\n display: block;\n font-size: 0.9rem;\n margin-bottom: 4px;\n }\n \n .timeline-label small {\n font-size: 0.75rem;\n line-height: 1.2;\n }\n \n @media (max-width: 768px) {\n .timeline-progress {\n flex-direction: column;\n align-items: stretch;\n }\n \n .timeline-step {\n flex-direction: row;\n align-items: center;\n margin-bottom: 20px;\n }\n \n .timeline-circle {\n margin-right: 15px;\n flex-shrink: 0;\n }\n \n .timeline-connector {\n display: none;\n }\n \n .timeline-label {\n text-align: left;\n margin-top: 0;\n min-height: auto;\n }\n }\n </style><script>\n // Global variables for current logs modal\n let currentTaskId = '';\n let currentWorkerId = '';\n\n function refreshPage() {\n location.reload();\n }\n\n function showTaskLogs(taskId, workerId) {\n currentTaskId = taskId;\n currentWorkerId = workerId;\n \n // Show the modal\n const modal = new bootstrap.Modal(document.getElementById('taskLogsModal'));\n modal.show();\n \n // Load logs\n loadTaskLogs(taskId, workerId);\n }\n\n function loadTaskLogs(taskId, workerId) {\n // Show loading spinner\n document.getElementById('logsLoadingSpinner').style.display = 'block';\n document.getElementById('logsError').style.display = 'none';\n document.getElementById('logsContent').style.display = 'none';\n \n // Update modal info\n document.getElementById('logsTaskId').textContent = taskId;\n document.getElementById('logsWorkerId').textContent = workerId;\n \n // Fetch logs from the API\n fetch(`/api/maintenance/workers/${workerId}/logs?taskId=${taskId}&maxEntries=100`)\n .then(response => response.json())\n .then(data => {\n document.getElementById('logsLoadingSpinner').style.display = 'none';\n \n if (data.error) {\n showLogsError(data.error);\n return;\n }\n \n // Display logs\n displayLogs(data.logs, data.count || 0);\n })\n .catch(error => {\n document.getElementById('logsLoadingSpinner').style.display = 'none';\n showLogsError('Failed to fetch logs: ' + error.message);\n });\n }\n\n function displayLogs(logs, count) {\n document.getElementById('logsError').style.display = 'none';\n document.getElementById('logsContent').style.display = 'block';\n document.getElementById('logsCount').textContent = count;\n \n const logsDisplay = document.getElementById('logsDisplay');\n \n if (!logs || logs.length === 0) {\n logsDisplay.textContent = 'No logs found for this task.';\n return;\n }\n \n // Format and display logs with structured fields\n let logText = '';\n logs.forEach(entry => {\n const timestamp = entry.timestamp ? new Date(entry.timestamp * 1000).toISOString() : 'N/A';\n const level = entry.level || 'INFO';\n const message = entry.message || '';\n \n logText += `[${timestamp}] ${level}: ${message}`;\n \n // Add structured fields if they exist\n if (entry.fields && Object.keys(entry.fields).length > 0) {\n const fieldsStr = Object.entries(entry.fields)\n .map(([key, value]) => `${key}=${value}`)\n .join(', ');\n logText += ` | ${fieldsStr}`;\n }\n \n // Add progress if available\n if (entry.progress !== undefined && entry.progress !== null) {\n logText += ` | progress=${entry.progress}%`;\n }\n \n // Add status if available\n if (entry.status) {\n logText += ` | status=${entry.status}`;\n }\n \n logText += '\\n';\n });\n \n logsDisplay.textContent = logText;\n \n // Scroll to top\n logsDisplay.scrollTop = 0;\n }\n\n function showLogsError(errorMessage) {\n document.getElementById('logsError').style.display = 'block';\n document.getElementById('logsContent').style.display = 'none';\n document.getElementById('logsErrorMessage').textContent = errorMessage;\n }\n\n function refreshModalLogs() {\n if (currentTaskId && currentWorkerId) {\n loadTaskLogs(currentTaskId, currentWorkerId);\n }\n }\n\n function downloadTaskLogs() {\n if (!currentTaskId || !currentWorkerId) {\n alert('No task logs to download');\n return;\n }\n \n // Download all logs (without maxEntries limit)\n const downloadUrl = `/api/maintenance/workers/${currentWorkerId}/logs?taskId=${currentTaskId}&maxEntries=0`;\n \n fetch(downloadUrl)\n .then(response => response.json())\n .then(data => {\n if (data.error) {\n alert('Error downloading logs: ' + data.error);\n return;\n }\n \n // Convert logs to text format with structured fields\n let logContent = '';\n if (data.logs && data.logs.length > 0) {\n data.logs.forEach(entry => {\n const timestamp = entry.timestamp ? new Date(entry.timestamp * 1000).toISOString() : 'N/A';\n const level = entry.level || 'INFO';\n const message = entry.message || '';\n \n logContent += `[${timestamp}] ${level}: ${message}`;\n \n // Add structured fields if they exist\n if (entry.fields && Object.keys(entry.fields).length > 0) {\n const fieldsStr = Object.entries(entry.fields)\n .map(([key, value]) => `${key}=${value}`)\n .join(', ');\n logContent += ` | ${fieldsStr}`;\n }\n \n // Add progress if available\n if (entry.progress !== undefined && entry.progress !== null) {\n logContent += ` | progress=${entry.progress}%`;\n }\n \n // Add status if available\n if (entry.status) {\n logContent += ` | status=${entry.status}`;\n }\n \n logContent += '\\n';\n });\n } else {\n logContent = 'No logs found for this task.';\n }\n \n // Create and download file\n const blob = new Blob([logContent], { type: 'text/plain' });\n const url = URL.createObjectURL(blob);\n const link = document.createElement('a');\n link.href = url;\n link.download = `task-${currentTaskId}-logs.txt`;\n link.click();\n URL.revokeObjectURL(url);\n })\n .catch(error => {\n alert('Error downloading logs: ' + error.message);\n });\n }\n\n function cancelTask(taskId) {\n if (confirm('Are you sure you want to cancel this task?')) {\n fetch(`/api/maintenance/tasks/${taskId}/cancel`, {\n method: 'POST',\n headers: {\n 'Content-Type': 'application/json',\n },\n })\n .then(response => response.json())\n .then(data => {\n if (data.success) {\n alert('Task cancelled successfully');\n location.reload();\n } else {\n alert('Error cancelling task: ' + data.error);\n }\n })\n .catch(error => {\n console.error('Error:', error);\n alert('Error cancelling task');\n });\n }\n }\n\n function refreshTaskLogs(taskId) {\n fetch(`/api/maintenance/tasks/${taskId}/detail`)\n .then(response => response.json())\n .then(data => {\n location.reload();\n })\n .catch(error => {\n console.error('Error:', error);\n alert('Error refreshing logs');\n });\n }\n\n function exportTaskDetail(taskId) {\n fetch(`/api/maintenance/tasks/${taskId}/detail`)\n .then(response => response.json())\n .then(data => {\n const dataStr = JSON.stringify(data, null, 2);\n const dataBlob = new Blob([dataStr], {type: 'application/json'});\n const url = URL.createObjectURL(dataBlob);\n const link = document.createElement('a');\n link.href = url;\n link.download = `task-${taskId}-detail.json`;\n link.click();\n URL.revokeObjectURL(url);\n })\n .catch(error => {\n console.error('Error:', error);\n alert('Error exporting task detail');\n });\n }\n\n // Auto-refresh every 30 seconds for active tasks\n if ('{string(data.Task.Status)}' === 'in_progress') {\n setInterval(refreshPage, 30000);\n }\n </script>")
+ 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<string, string> tags = 23;
+ TaskCreationMetrics creation_metrics = 24;
+}
+
+// TaskAssignmentRecord tracks worker assignments for a task
+message TaskAssignmentRecord {
+ string worker_id = 1;
+ string worker_address = 2;
+ int64 assigned_at = 3;
+ int64 unassigned_at = 4; // Optional: when worker was unassigned
+ string reason = 5; // Reason for assignment/unassignment
+}
+
+// TaskCreationMetrics tracks why and how a task was created
+message TaskCreationMetrics {
+ string trigger_metric = 1; // Name of metric that triggered creation
+ double metric_value = 2; // Value that triggered creation
+ double threshold = 3; // Threshold that was exceeded
+ VolumeHealthMetrics volume_metrics = 4; // Volume health at creation time
+ map<string, string> additional_data = 5; // Additional context data
+}
+
+// VolumeHealthMetrics captures volume state at task creation
+message VolumeHealthMetrics {
+ uint64 total_size = 1;
+ uint64 used_size = 2;
+ uint64 garbage_size = 3;
+ double garbage_ratio = 4;
+ int32 file_count = 5;
+ int32 deleted_file_count = 6;
+ int64 last_modified = 7;
+ int32 replica_count = 8;
+ bool is_ec_volume = 9;
+ string collection = 10;
+}
+
+// TaskStateFile wraps task data with metadata for persistence
+message TaskStateFile {
+ MaintenanceTaskData task = 1;
+ int64 last_updated = 2;
+ string admin_version = 3;
} \ 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.Replicas
+ return x.VolumeSize
+ }
+ return 0
+}
+
+func (x *TaskParams) GetSources() []*TaskSource {
+ if x != nil {
+ 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,6 +2749,568 @@ func (x *ReplicationTaskConfig) GetTargetReplicaCount() int32 {
return 0
}
+// 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)
+}
+
+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 = "" +
@@ -2877,26 +3376,26 @@ const file_worker_proto_rawDesc = "" +
"\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" +
+ "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"\xe1\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" +
+ "\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\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" +
+ "\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" +
@@ -2905,49 +3404,43 @@ const file_worker_proto_rawDesc = "" +
"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" +
+ "\x0fverify_checksum\x18\x05 \x01(\bR\x0everifyChecksum\"\xfe\x01\n" +
"\x17ErasureCodingTaskParams\x120\n" +
- "\x14estimated_shard_size\x18\x03 \x01(\x04R\x12estimatedShardSize\x12\x1f\n" +
- "\vdata_shards\x18\x04 \x01(\x05R\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\x05 \x01(\x05R\fparityShards\x12\x1f\n" +
- "\vworking_dir\x18\x06 \x01(\tR\n" +
+ "\rparity_shards\x18\x03 \x01(\x05R\fparityShards\x12\x1f\n" +
+ "\vworking_dir\x18\x04 \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" +
+ "\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'\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" +
+ "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" +
- "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" +
+ "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" +
@@ -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),