mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2025-11-24 16:53:14 +08:00
Admin: misc improvements on admin server and workers. EC now works. (#7055)
* initial design * added simulation as tests * reorganized the codebase to move the simulation framework and tests into their own dedicated package * integration test. ec worker task * remove "enhanced" reference * start master, volume servers, filer Current Status ✅ Master: Healthy and running (port 9333) ✅ Filer: Healthy and running (port 8888) ✅ Volume Servers: All 6 servers running (ports 8080-8085) 🔄 Admin/Workers: Will start when dependencies are ready * generate write load * tasks are assigned * admin start wtih grpc port. worker has its own working directory * Update .gitignore * working worker and admin. Task detection is not working yet. * compiles, detection uses volumeSizeLimitMB from master * compiles * worker retries connecting to admin * build and restart * rendering pending tasks * skip task ID column * sticky worker id * test canScheduleTaskNow * worker reconnect to admin * clean up logs * worker register itself first * worker can run ec work and report status but: 1. one volume should not be repeatedly worked on. 2. ec shards needs to be distributed and source data should be deleted. * move ec task logic * listing ec shards * local copy, ec. Need to distribute. * ec is mostly working now * distribution of ec shards needs improvement * need configuration to enable ec * show ec volumes * interval field UI component * rename * integration test with vauuming * garbage percentage threshold * fix warning * display ec shard sizes * fix ec volumes list * Update ui.go * show default values * ensure correct default value * MaintenanceConfig use ConfigField * use schema defined defaults * config * reduce duplication * refactor to use BaseUIProvider * each task register its schema * checkECEncodingCandidate use ecDetector * use vacuumDetector * use volumeSizeLimitMB * remove remove * remove unused * refactor * use new framework * remove v2 reference * refactor * left menu can scroll now * The maintenance manager was not being initialized when no data directory was configured for persistent storage. * saving config * Update task_config_schema_templ.go * enable/disable tasks * protobuf encoded task configurations * fix system settings * use ui component * remove logs * interface{} Reduction * reduce interface{} * reduce interface{} * avoid from/to map * reduce interface{} * refactor * keep it DRY * added logging * debug messages * debug level * debug * show the log caller line * use configured task policy * log level * handle admin heartbeat response * Update worker.go * fix EC rack and dc count * Report task status to admin server * fix task logging, simplify interface checking, use erasure_coding constants * factor in empty volume server during task planning * volume.list adds disk id * track disk id also * fix locking scheduled and manual scanning * add active topology * simplify task detector * ec task completed, but shards are not showing up * implement ec in ec_typed.go * adjust log level * dedup * implementing ec copying shards and only ecx files * use disk id when distributing ec shards 🎯 Planning: ActiveTopology creates DestinationPlan with specific TargetDisk 📦 Task Creation: maintenance_integration.go creates ECDestination with DiskId 🚀 Task Execution: EC task passes DiskId in VolumeEcShardsCopyRequest 💾 Volume Server: Receives disk_id and stores shards on specific disk (vs.store.Locations[req.DiskId]) 📂 File System: EC shards and metadata land in the exact disk directory planned * Delete original volume from all locations * clean up existing shard locations * local encoding and distributing * Update docker/admin_integration/EC-TESTING-README.md Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com> * check volume id range * simplify * fix tests * fix types * clean up logs and tests --------- Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>
This commit is contained in:
741
weed/admin/topology/active_topology.go
Normal file
741
weed/admin/topology/active_topology.go
Normal file
@@ -0,0 +1,741 @@
|
||||
package topology
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
|
||||
)
|
||||
|
||||
// TaskType represents different types of maintenance operations
|
||||
type TaskType string
|
||||
|
||||
// TaskStatus represents the current status of a task
|
||||
type TaskStatus string
|
||||
|
||||
// Common task type constants
|
||||
const (
|
||||
TaskTypeVacuum TaskType = "vacuum"
|
||||
TaskTypeBalance TaskType = "balance"
|
||||
TaskTypeErasureCoding TaskType = "erasure_coding"
|
||||
TaskTypeReplication TaskType = "replication"
|
||||
)
|
||||
|
||||
// Common task status constants
|
||||
const (
|
||||
TaskStatusPending TaskStatus = "pending"
|
||||
TaskStatusInProgress TaskStatus = "in_progress"
|
||||
TaskStatusCompleted TaskStatus = "completed"
|
||||
)
|
||||
|
||||
// taskState represents the current state of tasks affecting the topology (internal)
|
||||
type taskState struct {
|
||||
VolumeID uint32 `json:"volume_id"`
|
||||
TaskType TaskType `json:"task_type"`
|
||||
SourceServer string `json:"source_server"`
|
||||
SourceDisk uint32 `json:"source_disk"`
|
||||
TargetServer string `json:"target_server,omitempty"`
|
||||
TargetDisk uint32 `json:"target_disk,omitempty"`
|
||||
Status TaskStatus `json:"status"`
|
||||
StartedAt time.Time `json:"started_at"`
|
||||
CompletedAt time.Time `json:"completed_at,omitempty"`
|
||||
}
|
||||
|
||||
// DiskInfo represents a disk with its current state and ongoing tasks (public for external access)
|
||||
type DiskInfo struct {
|
||||
NodeID string `json:"node_id"`
|
||||
DiskID uint32 `json:"disk_id"`
|
||||
DiskType string `json:"disk_type"`
|
||||
DataCenter string `json:"data_center"`
|
||||
Rack string `json:"rack"`
|
||||
DiskInfo *master_pb.DiskInfo `json:"disk_info"`
|
||||
LoadCount int `json:"load_count"` // Number of active tasks
|
||||
}
|
||||
|
||||
// activeDisk represents internal disk state (private)
|
||||
type activeDisk struct {
|
||||
*DiskInfo
|
||||
pendingTasks []*taskState
|
||||
assignedTasks []*taskState
|
||||
recentTasks []*taskState // Completed in last N seconds
|
||||
}
|
||||
|
||||
// activeNode represents a node with its disks (private)
|
||||
type activeNode struct {
|
||||
nodeID string
|
||||
dataCenter string
|
||||
rack string
|
||||
nodeInfo *master_pb.DataNodeInfo
|
||||
disks map[uint32]*activeDisk // DiskID -> activeDisk
|
||||
}
|
||||
|
||||
// ActiveTopology provides a real-time view of cluster state with task awareness
|
||||
type ActiveTopology struct {
|
||||
// Core topology from master
|
||||
topologyInfo *master_pb.TopologyInfo
|
||||
lastUpdated time.Time
|
||||
|
||||
// Structured topology for easy access (private)
|
||||
nodes map[string]*activeNode // NodeID -> activeNode
|
||||
disks map[string]*activeDisk // "NodeID:DiskID" -> activeDisk
|
||||
|
||||
// Task states affecting the topology (private)
|
||||
pendingTasks map[string]*taskState
|
||||
assignedTasks map[string]*taskState
|
||||
recentTasks map[string]*taskState
|
||||
|
||||
// Configuration
|
||||
recentTaskWindowSeconds int
|
||||
|
||||
// Synchronization
|
||||
mutex sync.RWMutex
|
||||
}
|
||||
|
||||
// NewActiveTopology creates a new ActiveTopology instance
|
||||
func NewActiveTopology(recentTaskWindowSeconds int) *ActiveTopology {
|
||||
if recentTaskWindowSeconds <= 0 {
|
||||
recentTaskWindowSeconds = 10 // Default 10 seconds
|
||||
}
|
||||
|
||||
return &ActiveTopology{
|
||||
nodes: make(map[string]*activeNode),
|
||||
disks: make(map[string]*activeDisk),
|
||||
pendingTasks: make(map[string]*taskState),
|
||||
assignedTasks: make(map[string]*taskState),
|
||||
recentTasks: make(map[string]*taskState),
|
||||
recentTaskWindowSeconds: recentTaskWindowSeconds,
|
||||
}
|
||||
}
|
||||
|
||||
// UpdateTopology updates the topology information from master
|
||||
func (at *ActiveTopology) UpdateTopology(topologyInfo *master_pb.TopologyInfo) error {
|
||||
at.mutex.Lock()
|
||||
defer at.mutex.Unlock()
|
||||
|
||||
at.topologyInfo = topologyInfo
|
||||
at.lastUpdated = time.Now()
|
||||
|
||||
// Rebuild structured topology
|
||||
at.nodes = make(map[string]*activeNode)
|
||||
at.disks = make(map[string]*activeDisk)
|
||||
|
||||
for _, dc := range topologyInfo.DataCenterInfos {
|
||||
for _, rack := range dc.RackInfos {
|
||||
for _, nodeInfo := range rack.DataNodeInfos {
|
||||
node := &activeNode{
|
||||
nodeID: nodeInfo.Id,
|
||||
dataCenter: dc.Id,
|
||||
rack: rack.Id,
|
||||
nodeInfo: nodeInfo,
|
||||
disks: make(map[uint32]*activeDisk),
|
||||
}
|
||||
|
||||
// Add disks for this node
|
||||
for diskType, diskInfo := range nodeInfo.DiskInfos {
|
||||
disk := &activeDisk{
|
||||
DiskInfo: &DiskInfo{
|
||||
NodeID: nodeInfo.Id,
|
||||
DiskID: diskInfo.DiskId,
|
||||
DiskType: diskType,
|
||||
DataCenter: dc.Id,
|
||||
Rack: rack.Id,
|
||||
DiskInfo: diskInfo,
|
||||
},
|
||||
}
|
||||
|
||||
diskKey := fmt.Sprintf("%s:%d", nodeInfo.Id, diskInfo.DiskId)
|
||||
node.disks[diskInfo.DiskId] = disk
|
||||
at.disks[diskKey] = disk
|
||||
}
|
||||
|
||||
at.nodes[nodeInfo.Id] = node
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Reassign task states to updated topology
|
||||
at.reassignTaskStates()
|
||||
|
||||
glog.V(1).Infof("ActiveTopology updated: %d nodes, %d disks", len(at.nodes), len(at.disks))
|
||||
return nil
|
||||
}
|
||||
|
||||
// AddPendingTask adds a pending task to the topology
|
||||
func (at *ActiveTopology) AddPendingTask(taskID string, taskType TaskType, volumeID uint32,
|
||||
sourceServer string, sourceDisk uint32, targetServer string, targetDisk uint32) {
|
||||
at.mutex.Lock()
|
||||
defer at.mutex.Unlock()
|
||||
|
||||
task := &taskState{
|
||||
VolumeID: volumeID,
|
||||
TaskType: taskType,
|
||||
SourceServer: sourceServer,
|
||||
SourceDisk: sourceDisk,
|
||||
TargetServer: targetServer,
|
||||
TargetDisk: targetDisk,
|
||||
Status: TaskStatusPending,
|
||||
StartedAt: time.Now(),
|
||||
}
|
||||
|
||||
at.pendingTasks[taskID] = task
|
||||
at.assignTaskToDisk(task)
|
||||
}
|
||||
|
||||
// AssignTask moves a task from pending to assigned
|
||||
func (at *ActiveTopology) AssignTask(taskID string) error {
|
||||
at.mutex.Lock()
|
||||
defer at.mutex.Unlock()
|
||||
|
||||
task, exists := at.pendingTasks[taskID]
|
||||
if !exists {
|
||||
return fmt.Errorf("pending task %s not found", taskID)
|
||||
}
|
||||
|
||||
delete(at.pendingTasks, taskID)
|
||||
task.Status = TaskStatusInProgress
|
||||
at.assignedTasks[taskID] = task
|
||||
at.reassignTaskStates()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// CompleteTask moves a task from assigned to recent
|
||||
func (at *ActiveTopology) CompleteTask(taskID string) error {
|
||||
at.mutex.Lock()
|
||||
defer at.mutex.Unlock()
|
||||
|
||||
task, exists := at.assignedTasks[taskID]
|
||||
if !exists {
|
||||
return fmt.Errorf("assigned task %s not found", taskID)
|
||||
}
|
||||
|
||||
delete(at.assignedTasks, taskID)
|
||||
task.Status = TaskStatusCompleted
|
||||
task.CompletedAt = time.Now()
|
||||
at.recentTasks[taskID] = task
|
||||
at.reassignTaskStates()
|
||||
|
||||
// Clean up old recent tasks
|
||||
at.cleanupRecentTasks()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// GetAvailableDisks returns disks that can accept new tasks of the given type
|
||||
func (at *ActiveTopology) GetAvailableDisks(taskType TaskType, excludeNodeID string) []*DiskInfo {
|
||||
at.mutex.RLock()
|
||||
defer at.mutex.RUnlock()
|
||||
|
||||
var available []*DiskInfo
|
||||
|
||||
for _, disk := range at.disks {
|
||||
if disk.NodeID == excludeNodeID {
|
||||
continue // Skip excluded node
|
||||
}
|
||||
|
||||
if at.isDiskAvailable(disk, taskType) {
|
||||
// Create a copy with current load count
|
||||
diskCopy := *disk.DiskInfo
|
||||
diskCopy.LoadCount = len(disk.pendingTasks) + len(disk.assignedTasks)
|
||||
available = append(available, &diskCopy)
|
||||
}
|
||||
}
|
||||
|
||||
return available
|
||||
}
|
||||
|
||||
// GetDiskLoad returns the current load on a disk (number of active tasks)
|
||||
func (at *ActiveTopology) GetDiskLoad(nodeID string, diskID uint32) int {
|
||||
at.mutex.RLock()
|
||||
defer at.mutex.RUnlock()
|
||||
|
||||
diskKey := fmt.Sprintf("%s:%d", nodeID, diskID)
|
||||
disk, exists := at.disks[diskKey]
|
||||
if !exists {
|
||||
return 0
|
||||
}
|
||||
|
||||
return len(disk.pendingTasks) + len(disk.assignedTasks)
|
||||
}
|
||||
|
||||
// HasRecentTaskForVolume checks if a volume had a recent task (to avoid immediate re-detection)
|
||||
func (at *ActiveTopology) HasRecentTaskForVolume(volumeID uint32, taskType TaskType) bool {
|
||||
at.mutex.RLock()
|
||||
defer at.mutex.RUnlock()
|
||||
|
||||
for _, task := range at.recentTasks {
|
||||
if task.VolumeID == volumeID && task.TaskType == taskType {
|
||||
return true
|
||||
}
|
||||
}
|
||||
|
||||
return false
|
||||
}
|
||||
|
||||
// GetAllNodes returns information about all nodes (public interface)
|
||||
func (at *ActiveTopology) GetAllNodes() map[string]*master_pb.DataNodeInfo {
|
||||
at.mutex.RLock()
|
||||
defer at.mutex.RUnlock()
|
||||
|
||||
result := make(map[string]*master_pb.DataNodeInfo)
|
||||
for nodeID, node := range at.nodes {
|
||||
result[nodeID] = node.nodeInfo
|
||||
}
|
||||
return result
|
||||
}
|
||||
|
||||
// GetTopologyInfo returns the current topology information (read-only access)
|
||||
func (at *ActiveTopology) GetTopologyInfo() *master_pb.TopologyInfo {
|
||||
at.mutex.RLock()
|
||||
defer at.mutex.RUnlock()
|
||||
return at.topologyInfo
|
||||
}
|
||||
|
||||
// GetNodeDisks returns all disks for a specific node
|
||||
func (at *ActiveTopology) GetNodeDisks(nodeID string) []*DiskInfo {
|
||||
at.mutex.RLock()
|
||||
defer at.mutex.RUnlock()
|
||||
|
||||
node, exists := at.nodes[nodeID]
|
||||
if !exists {
|
||||
return nil
|
||||
}
|
||||
|
||||
var disks []*DiskInfo
|
||||
for _, disk := range node.disks {
|
||||
diskCopy := *disk.DiskInfo
|
||||
diskCopy.LoadCount = len(disk.pendingTasks) + len(disk.assignedTasks)
|
||||
disks = append(disks, &diskCopy)
|
||||
}
|
||||
|
||||
return disks
|
||||
}
|
||||
|
||||
// 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"`
|
||||
}
|
||||
|
||||
// MultiDestinationPlan represents multiple planned destinations for operations like EC
|
||||
type MultiDestinationPlan struct {
|
||||
Plans []*DestinationPlan `json:"plans"`
|
||||
TotalShards int `json:"total_shards"`
|
||||
SuccessfulRack int `json:"successful_racks"`
|
||||
SuccessfulDCs int `json:"successful_dcs"`
|
||||
}
|
||||
|
||||
// PlanBalanceDestination finds the best destination for a balance operation
|
||||
func (at *ActiveTopology) PlanBalanceDestination(volumeID uint32, sourceNode string, sourceRack string, sourceDC string, volumeSize uint64) (*DestinationPlan, error) {
|
||||
at.mutex.RLock()
|
||||
defer at.mutex.RUnlock()
|
||||
|
||||
// Get available disks, excluding the source node
|
||||
availableDisks := at.getAvailableDisksForPlanning(TaskTypeBalance, sourceNode)
|
||||
if len(availableDisks) == 0 {
|
||||
return nil, fmt.Errorf("no available disks for balance operation")
|
||||
}
|
||||
|
||||
// Score each disk for balance placement
|
||||
bestDisk := at.selectBestBalanceDestination(availableDisks, sourceRack, sourceDC, volumeSize)
|
||||
if bestDisk == nil {
|
||||
return nil, fmt.Errorf("no suitable destination found for balance operation")
|
||||
}
|
||||
|
||||
return &DestinationPlan{
|
||||
TargetNode: bestDisk.NodeID,
|
||||
TargetDisk: bestDisk.DiskID,
|
||||
TargetRack: bestDisk.Rack,
|
||||
TargetDC: bestDisk.DataCenter,
|
||||
ExpectedSize: volumeSize,
|
||||
PlacementScore: at.calculatePlacementScore(bestDisk, sourceRack, sourceDC),
|
||||
Conflicts: at.checkPlacementConflicts(bestDisk, TaskTypeBalance),
|
||||
}, nil
|
||||
}
|
||||
|
||||
// PlanECDestinations finds multiple destinations for EC shard distribution
|
||||
func (at *ActiveTopology) PlanECDestinations(volumeID uint32, sourceNode string, sourceRack string, sourceDC string, shardsNeeded int) (*MultiDestinationPlan, error) {
|
||||
at.mutex.RLock()
|
||||
defer at.mutex.RUnlock()
|
||||
|
||||
// Get available disks for EC placement
|
||||
availableDisks := at.getAvailableDisksForPlanning(TaskTypeErasureCoding, "")
|
||||
if len(availableDisks) < shardsNeeded {
|
||||
return nil, fmt.Errorf("insufficient disks for EC placement: need %d, have %d", shardsNeeded, len(availableDisks))
|
||||
}
|
||||
|
||||
// Select best disks for EC placement with rack/DC diversity
|
||||
selectedDisks := at.selectBestECDestinations(availableDisks, sourceRack, sourceDC, shardsNeeded)
|
||||
if len(selectedDisks) < shardsNeeded {
|
||||
return nil, fmt.Errorf("could not find %d suitable destinations for EC placement", shardsNeeded)
|
||||
}
|
||||
|
||||
var plans []*DestinationPlan
|
||||
rackCount := make(map[string]int)
|
||||
dcCount := make(map[string]int)
|
||||
|
||||
for _, disk := range selectedDisks {
|
||||
plan := &DestinationPlan{
|
||||
TargetNode: disk.NodeID,
|
||||
TargetDisk: disk.DiskID,
|
||||
TargetRack: disk.Rack,
|
||||
TargetDC: disk.DataCenter,
|
||||
ExpectedSize: 0, // EC shards don't have predetermined size
|
||||
PlacementScore: at.calculatePlacementScore(disk, sourceRack, sourceDC),
|
||||
Conflicts: at.checkPlacementConflicts(disk, TaskTypeErasureCoding),
|
||||
}
|
||||
plans = append(plans, plan)
|
||||
|
||||
// Count rack and DC diversity
|
||||
rackKey := fmt.Sprintf("%s:%s", disk.DataCenter, disk.Rack)
|
||||
rackCount[rackKey]++
|
||||
dcCount[disk.DataCenter]++
|
||||
}
|
||||
|
||||
return &MultiDestinationPlan{
|
||||
Plans: plans,
|
||||
TotalShards: len(plans),
|
||||
SuccessfulRack: len(rackCount),
|
||||
SuccessfulDCs: len(dcCount),
|
||||
}, nil
|
||||
}
|
||||
|
||||
// getAvailableDisksForPlanning returns disks available for destination planning
|
||||
func (at *ActiveTopology) getAvailableDisksForPlanning(taskType TaskType, excludeNodeID string) []*activeDisk {
|
||||
var available []*activeDisk
|
||||
|
||||
for _, disk := range at.disks {
|
||||
if excludeNodeID != "" && disk.NodeID == excludeNodeID {
|
||||
continue // Skip excluded node
|
||||
}
|
||||
|
||||
if at.isDiskAvailable(disk, taskType) {
|
||||
available = append(available, disk)
|
||||
}
|
||||
}
|
||||
|
||||
return available
|
||||
}
|
||||
|
||||
// selectBestBalanceDestination selects the best disk for balance operation
|
||||
func (at *ActiveTopology) selectBestBalanceDestination(disks []*activeDisk, sourceRack string, sourceDC string, volumeSize uint64) *activeDisk {
|
||||
if len(disks) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
var bestDisk *activeDisk
|
||||
bestScore := -1.0
|
||||
|
||||
for _, disk := range disks {
|
||||
score := at.calculateBalanceScore(disk, sourceRack, sourceDC, volumeSize)
|
||||
if score > bestScore {
|
||||
bestScore = score
|
||||
bestDisk = disk
|
||||
}
|
||||
}
|
||||
|
||||
return bestDisk
|
||||
}
|
||||
|
||||
// selectBestECDestinations selects multiple disks for EC shard placement with diversity
|
||||
func (at *ActiveTopology) selectBestECDestinations(disks []*activeDisk, sourceRack string, sourceDC string, shardsNeeded int) []*activeDisk {
|
||||
if len(disks) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Group disks by rack and DC for diversity
|
||||
rackGroups := make(map[string][]*activeDisk)
|
||||
for _, disk := range disks {
|
||||
rackKey := fmt.Sprintf("%s:%s", disk.DataCenter, disk.Rack)
|
||||
rackGroups[rackKey] = append(rackGroups[rackKey], disk)
|
||||
}
|
||||
|
||||
var selected []*activeDisk
|
||||
usedRacks := make(map[string]bool)
|
||||
|
||||
// First pass: select one disk from each rack for maximum diversity
|
||||
for rackKey, rackDisks := range rackGroups {
|
||||
if len(selected) >= shardsNeeded {
|
||||
break
|
||||
}
|
||||
|
||||
// Select best disk from this rack
|
||||
bestDisk := at.selectBestFromRack(rackDisks, sourceRack, sourceDC)
|
||||
if bestDisk != nil {
|
||||
selected = append(selected, bestDisk)
|
||||
usedRacks[rackKey] = true
|
||||
}
|
||||
}
|
||||
|
||||
// Second pass: if we need more disks, select from racks we've already used
|
||||
if len(selected) < shardsNeeded {
|
||||
for _, disk := range disks {
|
||||
if len(selected) >= shardsNeeded {
|
||||
break
|
||||
}
|
||||
|
||||
// Skip if already selected
|
||||
alreadySelected := false
|
||||
for _, sel := range selected {
|
||||
if sel.NodeID == disk.NodeID && sel.DiskID == disk.DiskID {
|
||||
alreadySelected = true
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
if !alreadySelected && at.isDiskAvailable(disk, TaskTypeErasureCoding) {
|
||||
selected = append(selected, disk)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return selected
|
||||
}
|
||||
|
||||
// selectBestFromRack selects the best disk from a rack
|
||||
func (at *ActiveTopology) selectBestFromRack(disks []*activeDisk, sourceRack string, sourceDC string) *activeDisk {
|
||||
if len(disks) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
var bestDisk *activeDisk
|
||||
bestScore := -1.0
|
||||
|
||||
for _, disk := range disks {
|
||||
if !at.isDiskAvailable(disk, TaskTypeErasureCoding) {
|
||||
continue
|
||||
}
|
||||
|
||||
score := at.calculateECScore(disk, sourceRack, sourceDC)
|
||||
if score > bestScore {
|
||||
bestScore = score
|
||||
bestDisk = disk
|
||||
}
|
||||
}
|
||||
|
||||
return bestDisk
|
||||
}
|
||||
|
||||
// calculateBalanceScore calculates placement score for balance operations
|
||||
func (at *ActiveTopology) calculateBalanceScore(disk *activeDisk, sourceRack string, sourceDC string, volumeSize uint64) float64 {
|
||||
score := 0.0
|
||||
|
||||
// Prefer disks with lower load
|
||||
activeLoad := len(disk.pendingTasks) + len(disk.assignedTasks)
|
||||
score += (2.0 - float64(activeLoad)) * 40.0 // Max 80 points for load
|
||||
|
||||
// Prefer disks with more free space
|
||||
if disk.DiskInfo.DiskInfo.MaxVolumeCount > 0 {
|
||||
freeRatio := float64(disk.DiskInfo.DiskInfo.MaxVolumeCount-disk.DiskInfo.DiskInfo.VolumeCount) / float64(disk.DiskInfo.DiskInfo.MaxVolumeCount)
|
||||
score += freeRatio * 20.0 // Max 20 points for free space
|
||||
}
|
||||
|
||||
// Rack diversity bonus (prefer different rack)
|
||||
if disk.Rack != sourceRack {
|
||||
score += 10.0
|
||||
}
|
||||
|
||||
// DC diversity bonus (prefer different DC)
|
||||
if disk.DataCenter != sourceDC {
|
||||
score += 5.0
|
||||
}
|
||||
|
||||
return score
|
||||
}
|
||||
|
||||
// calculateECScore calculates placement score for EC operations
|
||||
func (at *ActiveTopology) calculateECScore(disk *activeDisk, sourceRack string, sourceDC string) float64 {
|
||||
score := 0.0
|
||||
|
||||
// Prefer disks with lower load
|
||||
activeLoad := len(disk.pendingTasks) + len(disk.assignedTasks)
|
||||
score += (2.0 - float64(activeLoad)) * 30.0 // Max 60 points for load
|
||||
|
||||
// Prefer disks with more free space
|
||||
if disk.DiskInfo.DiskInfo.MaxVolumeCount > 0 {
|
||||
freeRatio := float64(disk.DiskInfo.DiskInfo.MaxVolumeCount-disk.DiskInfo.DiskInfo.VolumeCount) / float64(disk.DiskInfo.DiskInfo.MaxVolumeCount)
|
||||
score += freeRatio * 20.0 // Max 20 points for free space
|
||||
}
|
||||
|
||||
// Strong rack diversity preference for EC
|
||||
if disk.Rack != sourceRack {
|
||||
score += 20.0
|
||||
}
|
||||
|
||||
// Strong DC diversity preference for EC
|
||||
if disk.DataCenter != sourceDC {
|
||||
score += 15.0
|
||||
}
|
||||
|
||||
return score
|
||||
}
|
||||
|
||||
// calculatePlacementScore calculates overall placement quality score
|
||||
func (at *ActiveTopology) calculatePlacementScore(disk *activeDisk, sourceRack string, sourceDC string) float64 {
|
||||
score := 0.0
|
||||
|
||||
// Load factor
|
||||
activeLoad := len(disk.pendingTasks) + len(disk.assignedTasks)
|
||||
loadScore := (2.0 - float64(activeLoad)) / 2.0 // Normalize to 0-1
|
||||
score += loadScore * 0.4
|
||||
|
||||
// Capacity factor
|
||||
if disk.DiskInfo.DiskInfo.MaxVolumeCount > 0 {
|
||||
freeRatio := float64(disk.DiskInfo.DiskInfo.MaxVolumeCount-disk.DiskInfo.DiskInfo.VolumeCount) / float64(disk.DiskInfo.DiskInfo.MaxVolumeCount)
|
||||
score += freeRatio * 0.3
|
||||
}
|
||||
|
||||
// Diversity factor
|
||||
diversityScore := 0.0
|
||||
if disk.Rack != sourceRack {
|
||||
diversityScore += 0.5
|
||||
}
|
||||
if disk.DataCenter != sourceDC {
|
||||
diversityScore += 0.5
|
||||
}
|
||||
score += diversityScore * 0.3
|
||||
|
||||
return score // Score between 0.0 and 1.0
|
||||
}
|
||||
|
||||
// checkPlacementConflicts checks for placement rule violations
|
||||
func (at *ActiveTopology) checkPlacementConflicts(disk *activeDisk, taskType TaskType) []string {
|
||||
var conflicts []string
|
||||
|
||||
// Check load limits
|
||||
activeLoad := len(disk.pendingTasks) + len(disk.assignedTasks)
|
||||
if activeLoad >= 2 {
|
||||
conflicts = append(conflicts, fmt.Sprintf("disk_load_high_%d", activeLoad))
|
||||
}
|
||||
|
||||
// Check capacity limits
|
||||
if disk.DiskInfo.DiskInfo.MaxVolumeCount > 0 {
|
||||
usageRatio := float64(disk.DiskInfo.DiskInfo.VolumeCount) / float64(disk.DiskInfo.DiskInfo.MaxVolumeCount)
|
||||
if usageRatio > 0.9 {
|
||||
conflicts = append(conflicts, "disk_capacity_high")
|
||||
}
|
||||
}
|
||||
|
||||
// Check for conflicting task types
|
||||
for _, task := range disk.assignedTasks {
|
||||
if at.areTaskTypesConflicting(task.TaskType, taskType) {
|
||||
conflicts = append(conflicts, fmt.Sprintf("task_conflict_%s", task.TaskType))
|
||||
}
|
||||
}
|
||||
|
||||
return conflicts
|
||||
}
|
||||
|
||||
// Private methods
|
||||
|
||||
// reassignTaskStates assigns tasks to the appropriate disks
|
||||
func (at *ActiveTopology) reassignTaskStates() {
|
||||
// Clear existing task assignments
|
||||
for _, disk := range at.disks {
|
||||
disk.pendingTasks = nil
|
||||
disk.assignedTasks = nil
|
||||
disk.recentTasks = nil
|
||||
}
|
||||
|
||||
// Reassign pending tasks
|
||||
for _, task := range at.pendingTasks {
|
||||
at.assignTaskToDisk(task)
|
||||
}
|
||||
|
||||
// Reassign assigned tasks
|
||||
for _, task := range at.assignedTasks {
|
||||
at.assignTaskToDisk(task)
|
||||
}
|
||||
|
||||
// Reassign recent tasks
|
||||
for _, task := range at.recentTasks {
|
||||
at.assignTaskToDisk(task)
|
||||
}
|
||||
}
|
||||
|
||||
// assignTaskToDisk assigns a task to the appropriate disk(s)
|
||||
func (at *ActiveTopology) assignTaskToDisk(task *taskState) {
|
||||
// Assign to source disk
|
||||
sourceKey := fmt.Sprintf("%s:%d", task.SourceServer, task.SourceDisk)
|
||||
if sourceDisk, exists := at.disks[sourceKey]; exists {
|
||||
switch task.Status {
|
||||
case TaskStatusPending:
|
||||
sourceDisk.pendingTasks = append(sourceDisk.pendingTasks, task)
|
||||
case TaskStatusInProgress:
|
||||
sourceDisk.assignedTasks = append(sourceDisk.assignedTasks, task)
|
||||
case TaskStatusCompleted:
|
||||
sourceDisk.recentTasks = append(sourceDisk.recentTasks, task)
|
||||
}
|
||||
}
|
||||
|
||||
// Assign to target disk if it exists and is different from source
|
||||
if task.TargetServer != "" && (task.TargetServer != task.SourceServer || task.TargetDisk != task.SourceDisk) {
|
||||
targetKey := fmt.Sprintf("%s:%d", task.TargetServer, task.TargetDisk)
|
||||
if targetDisk, exists := at.disks[targetKey]; exists {
|
||||
switch task.Status {
|
||||
case TaskStatusPending:
|
||||
targetDisk.pendingTasks = append(targetDisk.pendingTasks, task)
|
||||
case TaskStatusInProgress:
|
||||
targetDisk.assignedTasks = append(targetDisk.assignedTasks, task)
|
||||
case TaskStatusCompleted:
|
||||
targetDisk.recentTasks = append(targetDisk.recentTasks, task)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// isDiskAvailable checks if a disk can accept new tasks
|
||||
func (at *ActiveTopology) isDiskAvailable(disk *activeDisk, taskType TaskType) bool {
|
||||
// Check if disk has too many active tasks
|
||||
activeLoad := len(disk.pendingTasks) + len(disk.assignedTasks)
|
||||
if activeLoad >= 2 { // Max 2 concurrent tasks per disk
|
||||
return false
|
||||
}
|
||||
|
||||
// Check for conflicting task types
|
||||
for _, task := range disk.assignedTasks {
|
||||
if at.areTaskTypesConflicting(task.TaskType, taskType) {
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
// areTaskTypesConflicting checks if two task types conflict
|
||||
func (at *ActiveTopology) areTaskTypesConflicting(existing, new TaskType) bool {
|
||||
// Examples of conflicting task types
|
||||
conflictMap := map[TaskType][]TaskType{
|
||||
TaskTypeVacuum: {TaskTypeBalance, TaskTypeErasureCoding},
|
||||
TaskTypeBalance: {TaskTypeVacuum, TaskTypeErasureCoding},
|
||||
TaskTypeErasureCoding: {TaskTypeVacuum, TaskTypeBalance},
|
||||
}
|
||||
|
||||
if conflicts, exists := conflictMap[existing]; exists {
|
||||
for _, conflictType := range conflicts {
|
||||
if conflictType == new {
|
||||
return true
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return false
|
||||
}
|
||||
|
||||
// cleanupRecentTasks removes old recent tasks
|
||||
func (at *ActiveTopology) cleanupRecentTasks() {
|
||||
cutoff := time.Now().Add(-time.Duration(at.recentTaskWindowSeconds) * time.Second)
|
||||
|
||||
for taskID, task := range at.recentTasks {
|
||||
if task.CompletedAt.Before(cutoff) {
|
||||
delete(at.recentTasks, taskID)
|
||||
}
|
||||
}
|
||||
}
|
||||
654
weed/admin/topology/active_topology_test.go
Normal file
654
weed/admin/topology/active_topology_test.go
Normal file
@@ -0,0 +1,654 @@
|
||||
package topology
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
// TestActiveTopologyBasicOperations tests basic topology management
|
||||
func TestActiveTopologyBasicOperations(t *testing.T) {
|
||||
topology := NewActiveTopology(10)
|
||||
assert.NotNil(t, topology)
|
||||
assert.Equal(t, 10, topology.recentTaskWindowSeconds)
|
||||
|
||||
// Test empty topology
|
||||
assert.Equal(t, 0, len(topology.nodes))
|
||||
assert.Equal(t, 0, len(topology.disks))
|
||||
assert.Equal(t, 0, len(topology.pendingTasks))
|
||||
}
|
||||
|
||||
// TestActiveTopologyUpdate tests topology updates from master
|
||||
func TestActiveTopologyUpdate(t *testing.T) {
|
||||
topology := NewActiveTopology(10)
|
||||
|
||||
// Create sample topology info
|
||||
topologyInfo := createSampleTopology()
|
||||
|
||||
err := topology.UpdateTopology(topologyInfo)
|
||||
require.NoError(t, err)
|
||||
|
||||
// Verify topology structure
|
||||
assert.Equal(t, 2, len(topology.nodes)) // 2 nodes
|
||||
assert.Equal(t, 4, len(topology.disks)) // 4 disks total (2 per node)
|
||||
|
||||
// Verify node structure
|
||||
node1, exists := topology.nodes["10.0.0.1:8080"]
|
||||
require.True(t, exists)
|
||||
assert.Equal(t, "dc1", node1.dataCenter)
|
||||
assert.Equal(t, "rack1", node1.rack)
|
||||
assert.Equal(t, 2, len(node1.disks))
|
||||
|
||||
// Verify disk structure
|
||||
disk1, exists := topology.disks["10.0.0.1:8080:0"]
|
||||
require.True(t, exists)
|
||||
assert.Equal(t, uint32(0), disk1.DiskID)
|
||||
assert.Equal(t, "hdd", disk1.DiskType)
|
||||
assert.Equal(t, "dc1", disk1.DataCenter)
|
||||
}
|
||||
|
||||
// TestTaskLifecycle tests the complete task lifecycle
|
||||
func TestTaskLifecycle(t *testing.T) {
|
||||
topology := NewActiveTopology(10)
|
||||
topology.UpdateTopology(createSampleTopology())
|
||||
|
||||
taskID := "balance-001"
|
||||
|
||||
// 1. Add pending task
|
||||
topology.AddPendingTask(taskID, TaskTypeBalance, 1001,
|
||||
"10.0.0.1:8080", 0, "10.0.0.2:8080", 1)
|
||||
|
||||
// Verify pending state
|
||||
assert.Equal(t, 1, len(topology.pendingTasks))
|
||||
assert.Equal(t, 0, len(topology.assignedTasks))
|
||||
assert.Equal(t, 0, len(topology.recentTasks))
|
||||
|
||||
task := topology.pendingTasks[taskID]
|
||||
assert.Equal(t, TaskStatusPending, task.Status)
|
||||
assert.Equal(t, uint32(1001), task.VolumeID)
|
||||
|
||||
// Verify task assigned to disks
|
||||
sourceDisk := topology.disks["10.0.0.1:8080:0"]
|
||||
targetDisk := topology.disks["10.0.0.2:8080:1"]
|
||||
assert.Equal(t, 1, len(sourceDisk.pendingTasks))
|
||||
assert.Equal(t, 1, len(targetDisk.pendingTasks))
|
||||
|
||||
// 2. Assign task
|
||||
err := topology.AssignTask(taskID)
|
||||
require.NoError(t, err)
|
||||
|
||||
// Verify assigned state
|
||||
assert.Equal(t, 0, len(topology.pendingTasks))
|
||||
assert.Equal(t, 1, len(topology.assignedTasks))
|
||||
assert.Equal(t, 0, len(topology.recentTasks))
|
||||
|
||||
task = topology.assignedTasks[taskID]
|
||||
assert.Equal(t, TaskStatusInProgress, task.Status)
|
||||
|
||||
// Verify task moved to assigned on disks
|
||||
assert.Equal(t, 0, len(sourceDisk.pendingTasks))
|
||||
assert.Equal(t, 1, len(sourceDisk.assignedTasks))
|
||||
assert.Equal(t, 0, len(targetDisk.pendingTasks))
|
||||
assert.Equal(t, 1, len(targetDisk.assignedTasks))
|
||||
|
||||
// 3. Complete task
|
||||
err = topology.CompleteTask(taskID)
|
||||
require.NoError(t, err)
|
||||
|
||||
// Verify completed state
|
||||
assert.Equal(t, 0, len(topology.pendingTasks))
|
||||
assert.Equal(t, 0, len(topology.assignedTasks))
|
||||
assert.Equal(t, 1, len(topology.recentTasks))
|
||||
|
||||
task = topology.recentTasks[taskID]
|
||||
assert.Equal(t, TaskStatusCompleted, task.Status)
|
||||
assert.False(t, task.CompletedAt.IsZero())
|
||||
}
|
||||
|
||||
// TestTaskDetectionScenarios tests various task detection scenarios
|
||||
func TestTaskDetectionScenarios(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
scenario func() *ActiveTopology
|
||||
expectedTasks map[string]bool // taskType -> shouldDetect
|
||||
}{
|
||||
{
|
||||
name: "Empty cluster - no tasks needed",
|
||||
scenario: func() *ActiveTopology {
|
||||
topology := NewActiveTopology(10)
|
||||
topology.UpdateTopology(createEmptyTopology())
|
||||
return topology
|
||||
},
|
||||
expectedTasks: map[string]bool{
|
||||
"balance": false,
|
||||
"vacuum": false,
|
||||
"ec": false,
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "Unbalanced cluster - balance task needed",
|
||||
scenario: func() *ActiveTopology {
|
||||
topology := NewActiveTopology(10)
|
||||
topology.UpdateTopology(createUnbalancedTopology())
|
||||
return topology
|
||||
},
|
||||
expectedTasks: map[string]bool{
|
||||
"balance": true,
|
||||
"vacuum": false,
|
||||
"ec": false,
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "High garbage ratio - vacuum task needed",
|
||||
scenario: func() *ActiveTopology {
|
||||
topology := NewActiveTopology(10)
|
||||
topology.UpdateTopology(createHighGarbageTopology())
|
||||
return topology
|
||||
},
|
||||
expectedTasks: map[string]bool{
|
||||
"balance": false,
|
||||
"vacuum": true,
|
||||
"ec": false,
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "Large volumes - EC task needed",
|
||||
scenario: func() *ActiveTopology {
|
||||
topology := NewActiveTopology(10)
|
||||
topology.UpdateTopology(createLargeVolumeTopology())
|
||||
return topology
|
||||
},
|
||||
expectedTasks: map[string]bool{
|
||||
"balance": false,
|
||||
"vacuum": false,
|
||||
"ec": true,
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "Recent tasks - no immediate re-detection",
|
||||
scenario: func() *ActiveTopology {
|
||||
topology := NewActiveTopology(10)
|
||||
topology.UpdateTopology(createUnbalancedTopology())
|
||||
// Add recent balance task
|
||||
topology.recentTasks["recent-balance"] = &taskState{
|
||||
VolumeID: 1001,
|
||||
TaskType: TaskTypeBalance,
|
||||
Status: TaskStatusCompleted,
|
||||
CompletedAt: time.Now().Add(-5 * time.Second), // 5 seconds ago
|
||||
}
|
||||
return topology
|
||||
},
|
||||
expectedTasks: map[string]bool{
|
||||
"balance": false, // Should not detect due to recent task
|
||||
"vacuum": false,
|
||||
"ec": false,
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
topology := tt.scenario()
|
||||
|
||||
// Test balance task detection
|
||||
shouldDetectBalance := tt.expectedTasks["balance"]
|
||||
actualDetectBalance := !topology.HasRecentTaskForVolume(1001, TaskTypeBalance)
|
||||
if shouldDetectBalance {
|
||||
assert.True(t, actualDetectBalance, "Should detect balance task")
|
||||
} else {
|
||||
// Note: In real implementation, task detection would be more sophisticated
|
||||
// This is a simplified test of the recent task prevention mechanism
|
||||
}
|
||||
|
||||
// Test that recent tasks prevent re-detection
|
||||
if len(topology.recentTasks) > 0 {
|
||||
for _, task := range topology.recentTasks {
|
||||
hasRecent := topology.HasRecentTaskForVolume(task.VolumeID, task.TaskType)
|
||||
assert.True(t, hasRecent, "Should find recent task for volume %d", task.VolumeID)
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
// TestTargetSelectionScenarios tests target selection for different task types
|
||||
func TestTargetSelectionScenarios(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
topology *ActiveTopology
|
||||
taskType TaskType
|
||||
excludeNode string
|
||||
expectedTargets int
|
||||
expectedBestTarget string
|
||||
}{
|
||||
{
|
||||
name: "Balance task - find least loaded disk",
|
||||
topology: createTopologyWithLoad(),
|
||||
taskType: TaskTypeBalance,
|
||||
excludeNode: "10.0.0.1:8080", // Exclude source node
|
||||
expectedTargets: 2, // 2 disks on other node
|
||||
},
|
||||
{
|
||||
name: "EC task - find multiple available disks",
|
||||
topology: createTopologyForEC(),
|
||||
taskType: TaskTypeErasureCoding,
|
||||
excludeNode: "", // Don't exclude any nodes
|
||||
expectedTargets: 4, // All 4 disks available
|
||||
},
|
||||
{
|
||||
name: "Vacuum task - avoid conflicting disks",
|
||||
topology: createTopologyWithConflicts(),
|
||||
taskType: TaskTypeVacuum,
|
||||
excludeNode: "",
|
||||
expectedTargets: 1, // Only 1 disk without conflicts (conflicts exclude more disks)
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
availableDisks := tt.topology.GetAvailableDisks(tt.taskType, tt.excludeNode)
|
||||
assert.Equal(t, tt.expectedTargets, len(availableDisks),
|
||||
"Expected %d available disks, got %d", tt.expectedTargets, len(availableDisks))
|
||||
|
||||
// Verify disks are actually available
|
||||
for _, disk := range availableDisks {
|
||||
assert.NotEqual(t, tt.excludeNode, disk.NodeID,
|
||||
"Available disk should not be on excluded node")
|
||||
|
||||
load := tt.topology.GetDiskLoad(disk.NodeID, disk.DiskID)
|
||||
assert.Less(t, load, 2, "Disk load should be less than 2")
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
// TestDiskLoadCalculation tests disk load calculation
|
||||
func TestDiskLoadCalculation(t *testing.T) {
|
||||
topology := NewActiveTopology(10)
|
||||
topology.UpdateTopology(createSampleTopology())
|
||||
|
||||
// Initially no load
|
||||
load := topology.GetDiskLoad("10.0.0.1:8080", 0)
|
||||
assert.Equal(t, 0, load)
|
||||
|
||||
// Add pending task
|
||||
topology.AddPendingTask("task1", TaskTypeBalance, 1001,
|
||||
"10.0.0.1:8080", 0, "10.0.0.2:8080", 1)
|
||||
|
||||
// Check load increased
|
||||
load = topology.GetDiskLoad("10.0.0.1:8080", 0)
|
||||
assert.Equal(t, 1, load)
|
||||
|
||||
// Add another task to same disk
|
||||
topology.AddPendingTask("task2", TaskTypeVacuum, 1002,
|
||||
"10.0.0.1:8080", 0, "", 0)
|
||||
|
||||
load = topology.GetDiskLoad("10.0.0.1:8080", 0)
|
||||
assert.Equal(t, 2, load)
|
||||
|
||||
// Move one task to assigned
|
||||
topology.AssignTask("task1")
|
||||
|
||||
// Load should still be 2 (1 pending + 1 assigned)
|
||||
load = topology.GetDiskLoad("10.0.0.1:8080", 0)
|
||||
assert.Equal(t, 2, load)
|
||||
|
||||
// Complete one task
|
||||
topology.CompleteTask("task1")
|
||||
|
||||
// Load should decrease to 1
|
||||
load = topology.GetDiskLoad("10.0.0.1:8080", 0)
|
||||
assert.Equal(t, 1, load)
|
||||
}
|
||||
|
||||
// TestTaskConflictDetection tests task conflict detection
|
||||
func TestTaskConflictDetection(t *testing.T) {
|
||||
topology := NewActiveTopology(10)
|
||||
topology.UpdateTopology(createSampleTopology())
|
||||
|
||||
// Add a balance task
|
||||
topology.AddPendingTask("balance1", TaskTypeBalance, 1001,
|
||||
"10.0.0.1:8080", 0, "10.0.0.2:8080", 1)
|
||||
topology.AssignTask("balance1")
|
||||
|
||||
// Try to get available disks for vacuum (conflicts with balance)
|
||||
availableDisks := topology.GetAvailableDisks(TaskTypeVacuum, "")
|
||||
|
||||
// Source disk should not be available due to conflict
|
||||
sourceDiskAvailable := false
|
||||
for _, disk := range availableDisks {
|
||||
if disk.NodeID == "10.0.0.1:8080" && disk.DiskID == 0 {
|
||||
sourceDiskAvailable = true
|
||||
break
|
||||
}
|
||||
}
|
||||
assert.False(t, sourceDiskAvailable, "Source disk should not be available due to task conflict")
|
||||
}
|
||||
|
||||
// TestPublicInterfaces tests the public interface methods
|
||||
func TestPublicInterfaces(t *testing.T) {
|
||||
topology := NewActiveTopology(10)
|
||||
topology.UpdateTopology(createSampleTopology())
|
||||
|
||||
// Test GetAllNodes
|
||||
nodes := topology.GetAllNodes()
|
||||
assert.Equal(t, 2, len(nodes))
|
||||
assert.Contains(t, nodes, "10.0.0.1:8080")
|
||||
assert.Contains(t, nodes, "10.0.0.2:8080")
|
||||
|
||||
// Test GetNodeDisks
|
||||
disks := topology.GetNodeDisks("10.0.0.1:8080")
|
||||
assert.Equal(t, 2, len(disks))
|
||||
|
||||
// Test with non-existent node
|
||||
disks = topology.GetNodeDisks("non-existent")
|
||||
assert.Nil(t, disks)
|
||||
}
|
||||
|
||||
// Helper functions to create test topologies
|
||||
|
||||
func createSampleTopology() *master_pb.TopologyInfo {
|
||||
return &master_pb.TopologyInfo{
|
||||
DataCenterInfos: []*master_pb.DataCenterInfo{
|
||||
{
|
||||
Id: "dc1",
|
||||
RackInfos: []*master_pb.RackInfo{
|
||||
{
|
||||
Id: "rack1",
|
||||
DataNodeInfos: []*master_pb.DataNodeInfo{
|
||||
{
|
||||
Id: "10.0.0.1:8080",
|
||||
DiskInfos: map[string]*master_pb.DiskInfo{
|
||||
"hdd": {DiskId: 0, VolumeCount: 10, MaxVolumeCount: 100},
|
||||
"ssd": {DiskId: 1, VolumeCount: 5, MaxVolumeCount: 50},
|
||||
},
|
||||
},
|
||||
{
|
||||
Id: "10.0.0.2:8080",
|
||||
DiskInfos: map[string]*master_pb.DiskInfo{
|
||||
"hdd": {DiskId: 0, VolumeCount: 8, MaxVolumeCount: 100},
|
||||
"ssd": {DiskId: 1, VolumeCount: 3, MaxVolumeCount: 50},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func createEmptyTopology() *master_pb.TopologyInfo {
|
||||
return &master_pb.TopologyInfo{
|
||||
DataCenterInfos: []*master_pb.DataCenterInfo{
|
||||
{
|
||||
Id: "dc1",
|
||||
RackInfos: []*master_pb.RackInfo{
|
||||
{
|
||||
Id: "rack1",
|
||||
DataNodeInfos: []*master_pb.DataNodeInfo{
|
||||
{
|
||||
Id: "10.0.0.1:8080",
|
||||
DiskInfos: map[string]*master_pb.DiskInfo{
|
||||
"hdd": {DiskId: 0, VolumeCount: 0, MaxVolumeCount: 100},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func createUnbalancedTopology() *master_pb.TopologyInfo {
|
||||
return &master_pb.TopologyInfo{
|
||||
DataCenterInfos: []*master_pb.DataCenterInfo{
|
||||
{
|
||||
Id: "dc1",
|
||||
RackInfos: []*master_pb.RackInfo{
|
||||
{
|
||||
Id: "rack1",
|
||||
DataNodeInfos: []*master_pb.DataNodeInfo{
|
||||
{
|
||||
Id: "10.0.0.1:8080",
|
||||
DiskInfos: map[string]*master_pb.DiskInfo{
|
||||
"hdd": {DiskId: 0, VolumeCount: 90, MaxVolumeCount: 100}, // Very loaded
|
||||
},
|
||||
},
|
||||
{
|
||||
Id: "10.0.0.2:8080",
|
||||
DiskInfos: map[string]*master_pb.DiskInfo{
|
||||
"hdd": {DiskId: 0, VolumeCount: 10, MaxVolumeCount: 100}, // Lightly loaded
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func createHighGarbageTopology() *master_pb.TopologyInfo {
|
||||
// In a real implementation, this would include volume-level garbage metrics
|
||||
return createSampleTopology()
|
||||
}
|
||||
|
||||
func createLargeVolumeTopology() *master_pb.TopologyInfo {
|
||||
// In a real implementation, this would include volume-level size metrics
|
||||
return createSampleTopology()
|
||||
}
|
||||
|
||||
func createTopologyWithLoad() *ActiveTopology {
|
||||
topology := NewActiveTopology(10)
|
||||
topology.UpdateTopology(createSampleTopology())
|
||||
|
||||
// Add some existing tasks to create load
|
||||
topology.AddPendingTask("existing1", TaskTypeVacuum, 2001,
|
||||
"10.0.0.1:8080", 0, "", 0)
|
||||
topology.AssignTask("existing1")
|
||||
|
||||
return topology
|
||||
}
|
||||
|
||||
func createTopologyForEC() *ActiveTopology {
|
||||
topology := NewActiveTopology(10)
|
||||
topology.UpdateTopology(createSampleTopology())
|
||||
return topology
|
||||
}
|
||||
|
||||
func createTopologyWithConflicts() *ActiveTopology {
|
||||
topology := NewActiveTopology(10)
|
||||
topology.UpdateTopology(createSampleTopology())
|
||||
|
||||
// Add conflicting tasks
|
||||
topology.AddPendingTask("balance1", TaskTypeBalance, 3001,
|
||||
"10.0.0.1:8080", 0, "10.0.0.2:8080", 0)
|
||||
topology.AssignTask("balance1")
|
||||
|
||||
topology.AddPendingTask("ec1", TaskTypeErasureCoding, 3002,
|
||||
"10.0.0.1:8080", 1, "", 0)
|
||||
topology.AssignTask("ec1")
|
||||
|
||||
return topology
|
||||
}
|
||||
|
||||
// TestDestinationPlanning tests destination planning functionality
|
||||
func TestDestinationPlanning(t *testing.T) {
|
||||
topology := NewActiveTopology(10)
|
||||
topology.UpdateTopology(createSampleTopology())
|
||||
|
||||
// Test balance destination planning
|
||||
t.Run("Balance destination planning", func(t *testing.T) {
|
||||
plan, err := topology.PlanBalanceDestination(1001, "10.0.0.1:8080", "rack1", "dc1", 1024*1024) // 1MB
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, plan)
|
||||
|
||||
// Should not target the source node
|
||||
assert.NotEqual(t, "10.0.0.1:8080", plan.TargetNode)
|
||||
assert.Equal(t, "10.0.0.2:8080", plan.TargetNode)
|
||||
assert.NotEmpty(t, plan.TargetRack)
|
||||
assert.NotEmpty(t, plan.TargetDC)
|
||||
assert.Greater(t, plan.PlacementScore, 0.0)
|
||||
})
|
||||
|
||||
// Test EC destination planning
|
||||
t.Run("EC destination planning", func(t *testing.T) {
|
||||
multiPlan, err := topology.PlanECDestinations(1002, "10.0.0.1:8080", "rack1", "dc1", 3) // Ask for 3 shards - source node can be included
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, multiPlan)
|
||||
assert.Greater(t, len(multiPlan.Plans), 0)
|
||||
assert.LessOrEqual(t, len(multiPlan.Plans), 3) // Should get at most 3 shards
|
||||
assert.Equal(t, len(multiPlan.Plans), multiPlan.TotalShards)
|
||||
|
||||
// Check that all plans have valid target nodes
|
||||
for _, plan := range multiPlan.Plans {
|
||||
assert.NotEmpty(t, plan.TargetNode)
|
||||
assert.NotEmpty(t, plan.TargetRack)
|
||||
assert.NotEmpty(t, plan.TargetDC)
|
||||
assert.GreaterOrEqual(t, plan.PlacementScore, 0.0)
|
||||
}
|
||||
|
||||
// Check diversity metrics
|
||||
assert.GreaterOrEqual(t, multiPlan.SuccessfulRack, 1)
|
||||
assert.GreaterOrEqual(t, multiPlan.SuccessfulDCs, 1)
|
||||
})
|
||||
|
||||
// Test destination planning with load
|
||||
t.Run("Destination planning considers load", func(t *testing.T) {
|
||||
// Add load to one disk
|
||||
topology.AddPendingTask("task1", TaskTypeBalance, 2001,
|
||||
"10.0.0.2:8080", 0, "", 0)
|
||||
|
||||
plan, err := topology.PlanBalanceDestination(1003, "10.0.0.1:8080", "rack1", "dc1", 1024*1024)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, plan)
|
||||
|
||||
// Should prefer less loaded disk (disk 1 over disk 0 on node2)
|
||||
assert.Equal(t, "10.0.0.2:8080", plan.TargetNode)
|
||||
assert.Equal(t, uint32(1), plan.TargetDisk) // Should prefer SSD (disk 1) which has no load
|
||||
})
|
||||
|
||||
// Test insufficient destinations
|
||||
t.Run("Handle insufficient destinations", func(t *testing.T) {
|
||||
// Try to plan for more EC shards than available disks
|
||||
multiPlan, err := topology.PlanECDestinations(1004, "10.0.0.1:8080", "rack1", "dc1", 100)
|
||||
|
||||
// Should get an error for insufficient disks
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, multiPlan)
|
||||
})
|
||||
}
|
||||
|
||||
// TestDestinationPlanningWithActiveTopology tests the integration between task detection and destination planning
|
||||
func TestDestinationPlanningWithActiveTopology(t *testing.T) {
|
||||
topology := NewActiveTopology(10)
|
||||
topology.UpdateTopology(createUnbalancedTopology())
|
||||
|
||||
// Test that tasks are created with destinations
|
||||
t.Run("Balance task with destination", func(t *testing.T) {
|
||||
// Simulate what the balance detector would create
|
||||
sourceNode := "10.0.0.1:8080" // Overloaded node
|
||||
volumeID := uint32(1001)
|
||||
|
||||
plan, err := topology.PlanBalanceDestination(volumeID, sourceNode, "rack1", "dc1", 1024*1024)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, plan)
|
||||
|
||||
// Verify the destination is different from source
|
||||
assert.NotEqual(t, sourceNode, plan.TargetNode)
|
||||
assert.Equal(t, "10.0.0.2:8080", plan.TargetNode) // Should be the lightly loaded node
|
||||
|
||||
// Verify placement quality
|
||||
assert.Greater(t, plan.PlacementScore, 0.0)
|
||||
assert.LessOrEqual(t, plan.PlacementScore, 1.0)
|
||||
})
|
||||
|
||||
// Test task state integration
|
||||
t.Run("Task state affects future planning", func(t *testing.T) {
|
||||
volumeID := uint32(1002)
|
||||
sourceNode := "10.0.0.1:8080"
|
||||
targetNode := "10.0.0.2:8080"
|
||||
|
||||
// Plan first destination
|
||||
plan1, err := topology.PlanBalanceDestination(volumeID, sourceNode, "rack1", "dc1", 1024*1024)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, plan1)
|
||||
|
||||
// Add a pending task to the target
|
||||
topology.AddPendingTask("task1", TaskTypeBalance, volumeID, sourceNode, 0, targetNode, 0)
|
||||
|
||||
// Plan another destination - should consider the pending task load
|
||||
plan2, err := topology.PlanBalanceDestination(1003, sourceNode, "rack1", "dc1", 1024*1024)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, plan2)
|
||||
|
||||
// The placement score should reflect the increased load
|
||||
// (This test might need adjustment based on the actual scoring algorithm)
|
||||
glog.V(1).Infof("Plan1 score: %.3f, Plan2 score: %.3f", plan1.PlacementScore, plan2.PlacementScore)
|
||||
})
|
||||
}
|
||||
|
||||
// TestECDestinationPlanningDetailed tests the EC destination planning with multiple shards
|
||||
func TestECDestinationPlanningDetailed(t *testing.T) {
|
||||
topology := NewActiveTopology(10)
|
||||
topology.UpdateTopology(createSampleTopology())
|
||||
|
||||
t.Run("EC multiple destinations", func(t *testing.T) {
|
||||
// Plan for 3 EC shards (now including source node, we have 4 disks total)
|
||||
multiPlan, err := topology.PlanECDestinations(1005, "10.0.0.1:8080", "rack1", "dc1", 3)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, multiPlan)
|
||||
|
||||
// Should get 3 destinations (can include source node's disks)
|
||||
assert.Equal(t, 3, len(multiPlan.Plans))
|
||||
assert.Equal(t, 3, multiPlan.TotalShards)
|
||||
|
||||
// Count node distribution - source node can now be included
|
||||
nodeCount := make(map[string]int)
|
||||
for _, plan := range multiPlan.Plans {
|
||||
nodeCount[plan.TargetNode]++
|
||||
}
|
||||
|
||||
// Should distribute across available nodes (both nodes can be used)
|
||||
assert.GreaterOrEqual(t, len(nodeCount), 1, "Should use at least 1 node")
|
||||
assert.LessOrEqual(t, len(nodeCount), 2, "Should use at most 2 nodes")
|
||||
glog.V(1).Infof("EC destinations node distribution: %v", nodeCount)
|
||||
|
||||
glog.V(1).Infof("EC destinations: %d plans across %d racks, %d DCs",
|
||||
multiPlan.TotalShards, multiPlan.SuccessfulRack, multiPlan.SuccessfulDCs)
|
||||
})
|
||||
|
||||
t.Run("EC destination planning with task conflicts", func(t *testing.T) {
|
||||
// Create a fresh topology for this test to avoid conflicts from previous test
|
||||
freshTopology := NewActiveTopology(10)
|
||||
freshTopology.UpdateTopology(createSampleTopology())
|
||||
|
||||
// Add tasks to create conflicts on some disks
|
||||
freshTopology.AddPendingTask("conflict1", TaskTypeVacuum, 2001, "10.0.0.2:8080", 0, "", 0)
|
||||
freshTopology.AddPendingTask("conflict2", TaskTypeBalance, 2002, "10.0.0.1:8080", 0, "", 0)
|
||||
freshTopology.AssignTask("conflict1")
|
||||
freshTopology.AssignTask("conflict2")
|
||||
|
||||
// Plan EC destinations - should still succeed using available disks
|
||||
multiPlan, err := freshTopology.PlanECDestinations(1006, "10.0.0.1:8080", "rack1", "dc1", 2)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, multiPlan)
|
||||
|
||||
// Should get destinations (using disks that don't have conflicts)
|
||||
assert.GreaterOrEqual(t, len(multiPlan.Plans), 1)
|
||||
assert.LessOrEqual(t, len(multiPlan.Plans), 2)
|
||||
|
||||
// Available disks should be: node1/disk1 and node2/disk1 (since disk0 on both nodes have conflicts)
|
||||
for _, plan := range multiPlan.Plans {
|
||||
assert.Equal(t, uint32(1), plan.TargetDisk, "Should prefer disk 1 which has no conflicts")
|
||||
}
|
||||
|
||||
glog.V(1).Infof("EC destination planning with conflicts: found %d destinations", len(multiPlan.Plans))
|
||||
})
|
||||
}
|
||||
Reference in New Issue
Block a user