test: add support for PXE nodes in qemu provision library

This isn't supposed to be used ever in Talos directly, but rather only
in integration tests for Sidero.

Signed-off-by: Andrey Smirnov <smirnov.andrey@gmail.com>
This commit is contained in:
Andrey Smirnov 2020-08-07 22:08:15 +03:00 committed by talos-bot
parent 83aa3bd3ab
commit d60adf9e3b
15 changed files with 129 additions and 23 deletions

View File

@ -115,7 +115,7 @@ func genV1Alpha1Config(args []string) error {
var configString string var configString string
switch t { switch t { //nolint: exhaustive
case machine.TypeInit: case machine.TypeInit:
configString, err = configBundle.Init().String() configString, err = configBundle.Init().String()
if err != nil { if err != nil {

View File

@ -41,6 +41,8 @@ func (cluster *clusterNodes) NodesByType(t machine.Type) []string {
return cluster.ControlPlaneNodes return cluster.ControlPlaneNodes
case machine.TypeJoin: case machine.TypeJoin:
return cluster.WorkerNodes return cluster.WorkerNodes
case machine.TypeUnknown:
return nil
default: default:
panic("unsupported machine type") panic("unsupported machine type")
} }

View File

@ -92,6 +92,8 @@ func (cluster *clusterState) NodesByType(t machine.Type) []string {
return cluster.controlPlaneNodes return cluster.controlPlaneNodes
case machine.TypeJoin: case machine.TypeJoin:
return cluster.workerNodes return cluster.workerNodes
case machine.TypeUnknown:
return nil
default: default:
panic("unsupported machine type") panic("unsupported machine type")
} }

View File

@ -641,6 +641,8 @@ func StartAllServices(seq runtime.Sequence, data interface{}) (runtime.TaskExecu
&services.Etcd{}, &services.Etcd{},
) )
case machine.TypeJoin: case machine.TypeJoin:
case machine.TypeUnknown:
return fmt.Errorf("unexpected machine type: %s", r.Config().Machine().Type())
} }
system.Services(r).StartAll() system.Services(r).StartAll()

View File

@ -49,7 +49,7 @@ func NewConfigBundle(opts ...Option) (*v1alpha1.ConfigBundle, error) {
return bundle, err return bundle, err
} }
switch configType { switch configType { //nolint: exhaustive
case machine.TypeInit: case machine.TypeInit:
bundle.InitCfg = unmarshalledConfig bundle.InitCfg = unmarshalledConfig
case machine.TypeControlPlane: case machine.TypeControlPlane:
@ -96,7 +96,7 @@ func NewConfigBundle(opts ...Option) (*v1alpha1.ConfigBundle, error) {
return bundle, err return bundle, err
} }
switch configType { switch configType { //nolint: exhaustive
case machine.TypeInit: case machine.TypeInit:
bundle.InitCfg = generatedConfig bundle.InitCfg = generatedConfig
case machine.TypeControlPlane: case machine.TypeControlPlane:

View File

@ -41,6 +41,8 @@ func Config(t machine.Type, in *Input) (c *v1alpha1.Config, err error) {
if c, err = workerUd(in); err != nil { if c, err = workerUd(in); err != nil {
return c, err return c, err
} }
case machine.TypeUnknown:
fallthrough
default: default:
return c, errors.New("failed to determine config type to generate") return c, errors.New("failed to determine config type to generate")
} }

View File

@ -10,8 +10,10 @@ import "fmt"
type Type int type Type int
const ( const (
// TypeUnknown represents undefined node type.
TypeUnknown Type = iota
// TypeInit represents a bootstrap node. // TypeInit represents a bootstrap node.
TypeInit Type = iota TypeInit
// TypeControlPlane represents a control plane node. // TypeControlPlane represents a control plane node.
TypeControlPlane TypeControlPlane
// TypeJoin represents a worker node. // TypeJoin represents a worker node.
@ -19,6 +21,7 @@ const (
) )
const ( const (
typeUnknown = "unknown"
typeInit = "init" typeInit = "init"
typeControlPlane = "controlplane" typeControlPlane = "controlplane"
typeJoin = "join" typeJoin = "join"
@ -26,12 +29,14 @@ const (
// String returns the string representation of Type. // String returns the string representation of Type.
func (t Type) String() string { func (t Type) String() string {
return [...]string{typeInit, typeControlPlane, typeJoin}[t] return [...]string{typeUnknown, typeInit, typeControlPlane, typeJoin}[t]
} }
// ParseType parses string constant as Type. // ParseType parses string constant as Type.
func ParseType(t string) (Type, error) { func ParseType(t string) (Type, error) {
switch t { switch t {
case typeUnknown:
return TypeUnknown, nil
case typeInit: case typeInit:
return TypeInit, nil return TypeInit, nil
case typeControlPlane: case typeControlPlane:

View File

@ -85,6 +85,17 @@ func (p *provisioner) Create(ctx context.Context, request provision.ClusterReque
return nil, err return nil, err
} }
var pxeNodeInfo []provision.NodeInfo
pxeNodes := request.Nodes.PXENodes()
if len(pxeNodes) > 0 {
fmt.Fprintln(options.LogWriter, "creating PXE nodes")
if pxeNodeInfo, err = p.createNodes(state, request, pxeNodes, &options); err != nil {
return nil, err
}
}
nodeInfo = append(nodeInfo, workerNodeInfo...) nodeInfo = append(nodeInfo, workerNodeInfo...)
state.ClusterInfo = provision.ClusterInfo{ state.ClusterInfo = provision.ClusterInfo{
@ -95,7 +106,8 @@ func (p *provisioner) Create(ctx context.Context, request provision.ClusterReque
GatewayAddr: request.Network.GatewayAddr, GatewayAddr: request.Network.GatewayAddr,
MTU: request.Network.MTU, MTU: request.Network.MTU,
}, },
Nodes: nodeInfo, Nodes: nodeInfo,
ExtraNodes: pxeNodeInfo,
} }
err = state.Save() err = state.Save()

View File

@ -42,6 +42,7 @@ type LaunchConfig struct {
MachineType string MachineType string
EnableKVM bool EnableKVM bool
BootloaderEnabled bool BootloaderEnabled bool
NodeUUID uuid.UUID
// Talos config // Talos config
Config string Config string
@ -56,6 +57,11 @@ type LaunchConfig struct {
MTU int MTU int
Nameservers []net.IP Nameservers []net.IP
// PXE
TFTPServer string
BootFilename string
IPXEBootFileName string
// filled by CNI invocation // filled by CNI invocation
tapName string tapName string
vmMAC string vmMAC string
@ -131,13 +137,15 @@ func withCNI(ctx context.Context, config *LaunchConfig, f func(config *LaunchCon
// dump node IP/mac/hostname for dhcp // dump node IP/mac/hostname for dhcp
if err = vm.DumpIPAMRecord(config.StatePath, vm.IPAMRecord{ if err = vm.DumpIPAMRecord(config.StatePath, vm.IPAMRecord{
IP: config.IP, IP: config.IP,
Netmask: config.CIDR.Mask, Netmask: config.CIDR.Mask,
MAC: vmIface.Mac, MAC: vmIface.Mac,
Hostname: config.Hostname, Hostname: config.Hostname,
Gateway: config.GatewayAddr, Gateway: config.GatewayAddr,
MTU: config.MTU, MTU: config.MTU,
Nameservers: config.Nameservers, Nameservers: config.Nameservers,
TFTPServer: config.TFTPServer,
IPXEBootFilename: config.IPXEBootFileName,
}); err != nil { }); err != nil {
return err return err
} }
@ -179,6 +187,8 @@ func launchVM(config *LaunchConfig) error {
"-device", fmt.Sprintf("virtio-net-pci,netdev=net0,mac=%s", config.vmMAC), "-device", fmt.Sprintf("virtio-net-pci,netdev=net0,mac=%s", config.vmMAC),
"-device", "virtio-rng-pci", "-device", "virtio-rng-pci",
"-no-reboot", "-no-reboot",
"-boot", "order=cn,reboot-timeout=5000",
"-smbios", fmt.Sprintf("type=1,uuid=%s", config.NodeUUID),
} }
machineArg := config.MachineType machineArg := config.MachineType
@ -203,7 +213,7 @@ func launchVM(config *LaunchConfig) error {
return err return err
} }
if !diskBootable || !config.BootloaderEnabled { if (!diskBootable || !config.BootloaderEnabled) && config.KernelImagePath != "" {
args = append(args, args = append(args,
"-kernel", config.KernelImagePath, "-kernel", config.KernelImagePath,
"-initrd", config.InitrdPath, "-initrd", config.InitrdPath,

View File

@ -16,8 +16,10 @@ import (
"strconv" "strconv"
"syscall" "syscall"
"github.com/google/uuid"
multierror "github.com/hashicorp/go-multierror" multierror "github.com/hashicorp/go-multierror"
"github.com/talos-systems/talos/pkg/machinery/config/types/v1alpha1/machine"
"github.com/talos-systems/talos/pkg/provision" "github.com/talos-systems/talos/pkg/provision"
"github.com/talos-systems/talos/pkg/provision/providers/vm" "github.com/talos-systems/talos/pkg/provision/providers/vm"
@ -61,23 +63,28 @@ func (p *provisioner) createNode(state *vm.State, clusterReq provision.ClusterRe
cmdline.Append("talos.platform", "metal") cmdline.Append("talos.platform", "metal")
cmdline.Append("talos.config", "{TALOS_CONFIG_URL}") // to be patched by launcher cmdline.Append("talos.config", "{TALOS_CONFIG_URL}") // to be patched by launcher
nodeConfig, err := nodeReq.Config.String() var nodeConfig string
if err != nil {
return provision.NodeInfo{}, err if nodeReq.Config != nil {
nodeConfig, err = nodeReq.Config.String()
if err != nil {
return provision.NodeInfo{}, err
}
} }
nodeUUID := uuid.New()
launchConfig := LaunchConfig{ launchConfig := LaunchConfig{
QemuExecutable: fmt.Sprintf("qemu-system-%s", arch.QemuArch()), QemuExecutable: fmt.Sprintf("qemu-system-%s", arch.QemuArch()),
DiskPath: diskPath, DiskPath: diskPath,
VCPUCount: vcpuCount, VCPUCount: vcpuCount,
MemSize: memSize, MemSize: memSize,
KernelImagePath: clusterReq.KernelPath,
KernelArgs: cmdline.String(), KernelArgs: cmdline.String(),
InitrdPath: clusterReq.InitramfsPath,
MachineType: arch.QemuMachine(), MachineType: arch.QemuMachine(),
PFlashImages: state.PFlashImages, PFlashImages: state.PFlashImages,
EnableKVM: opts.TargetArch == runtime.GOARCH, EnableKVM: opts.TargetArch == runtime.GOARCH,
BootloaderEnabled: opts.BootloaderEnabled, BootloaderEnabled: opts.BootloaderEnabled,
NodeUUID: nodeUUID,
Config: nodeConfig, Config: nodeConfig,
NetworkConfig: state.VMCNIConfig, NetworkConfig: state.VMCNIConfig,
CNI: clusterReq.Network.CNI, CNI: clusterReq.Network.CNI,
@ -87,6 +94,13 @@ func (p *provisioner) createNode(state *vm.State, clusterReq provision.ClusterRe
GatewayAddr: clusterReq.Network.GatewayAddr, GatewayAddr: clusterReq.Network.GatewayAddr,
MTU: clusterReq.Network.MTU, MTU: clusterReq.Network.MTU,
Nameservers: clusterReq.Network.Nameservers, Nameservers: clusterReq.Network.Nameservers,
TFTPServer: nodeReq.TFTPServer,
IPXEBootFileName: nodeReq.IPXEBootFilename,
}
if !nodeReq.PXEBooted {
launchConfig.KernelImagePath = clusterReq.KernelPath
launchConfig.InitrdPath = clusterReq.InitramfsPath
} }
launchConfig.StatePath, err = state.StatePath() launchConfig.StatePath, err = state.StatePath()
@ -127,10 +141,16 @@ func (p *provisioner) createNode(state *vm.State, clusterReq provision.ClusterRe
// no need to wait here, as cmd has all the Stdin/out/err via *os.File // no need to wait here, as cmd has all the Stdin/out/err via *os.File
nodeType := machine.TypeUnknown
if nodeReq.Config != nil {
nodeType = nodeReq.Config.Machine().Type()
}
nodeInfo := provision.NodeInfo{ nodeInfo := provision.NodeInfo{
ID: pidPath, ID: pidPath,
UUID: nodeUUID,
Name: nodeReq.Name, Name: nodeReq.Name,
Type: nodeReq.Config.Machine().Type(), Type: nodeType,
NanoCPUs: nodeReq.NanoCPUs, NanoCPUs: nodeReq.NanoCPUs,
Memory: nodeReq.Memory, Memory: nodeReq.Memory,

View File

@ -21,6 +21,7 @@ import (
"github.com/talos-systems/talos/pkg/provision" "github.com/talos-systems/talos/pkg/provision"
) )
//nolint: gocyclo
func handler(serverIP net.IP, statePath string) server4.Handler { func handler(serverIP net.IP, statePath string) server4.Handler {
return func(conn net.PacketConn, peer net.Addr, m *dhcpv4.DHCPv4) { return func(conn net.PacketConn, peer net.Addr, m *dhcpv4.DHCPv4) {
if m.OpCode != dhcpv4.OpcodeBootRequest { if m.OpCode != dhcpv4.OpcodeBootRequest {
@ -45,7 +46,6 @@ func handler(serverIP net.IP, statePath string) server4.Handler {
resp, err := dhcpv4.NewReplyFromRequest(m, resp, err := dhcpv4.NewReplyFromRequest(m,
dhcpv4.WithNetmask(match.Netmask), dhcpv4.WithNetmask(match.Netmask),
dhcpv4.WithServerIP(serverIP),
dhcpv4.WithYourIP(match.IP), dhcpv4.WithYourIP(match.IP),
dhcpv4.WithOption(dhcpv4.OptHostName(match.Hostname)), dhcpv4.WithOption(dhcpv4.OptHostName(match.Hostname)),
dhcpv4.WithOption(dhcpv4.OptDNS(match.Nameservers...)), dhcpv4.WithOption(dhcpv4.OptDNS(match.Nameservers...)),
@ -58,6 +58,18 @@ func handler(serverIP net.IP, statePath string) server4.Handler {
return return
} }
if m.IsOptionRequested(dhcpv4.OptionBootfileName) {
log.Printf("received PXE boot request from %s", m.ClientHWAddr)
if match.TFTPServer != "" {
log.Printf("sending PXE response to %s: %s/%s", m.ClientHWAddr, match.TFTPServer, match.IPXEBootFilename)
resp.ServerIPAddr = net.ParseIP(match.TFTPServer)
resp.UpdateOption(dhcpv4.OptTFTPServerName(match.TFTPServer))
resp.UpdateOption(dhcpv4.OptBootFileName(match.IPXEBootFilename))
}
}
resp.UpdateOption(dhcpv4.OptGeneric(dhcpv4.OptionInterfaceMTU, dhcpv4.Uint16(match.MTU).ToBytes())) resp.UpdateOption(dhcpv4.OptGeneric(dhcpv4.OptionInterfaceMTU, dhcpv4.Uint16(match.MTU).ToBytes()))
switch mt := m.MessageType(); mt { //nolint: exhaustive switch mt := m.MessageType(); mt { //nolint: exhaustive

View File

@ -22,6 +22,9 @@ type IPAMRecord struct {
Gateway net.IP Gateway net.IP
MTU int MTU int
Nameservers []net.IP Nameservers []net.IP
TFTPServer string
IPXEBootFilename string
} }
// IPAMDatabase is a mapping from MAC address to records. // IPAMDatabase is a mapping from MAC address to records.

View File

@ -16,7 +16,9 @@ import (
func (p *Provisioner) DestroyNodes(cluster provision.ClusterInfo, options *provision.Options) error { func (p *Provisioner) DestroyNodes(cluster provision.ClusterInfo, options *provision.Options) error {
errCh := make(chan error) errCh := make(chan error)
for _, node := range cluster.Nodes { nodes := append(cluster.Nodes, cluster.ExtraNodes...)
for _, node := range nodes {
go func(node provision.NodeInfo) { go func(node provision.NodeInfo) {
fmt.Fprintln(options.LogWriter, "stopping VM", node.Name) fmt.Fprintln(options.LogWriter, "stopping VM", node.Name)
@ -26,7 +28,7 @@ func (p *Provisioner) DestroyNodes(cluster provision.ClusterInfo, options *provi
var multiErr *multierror.Error var multiErr *multierror.Error
for range cluster.Nodes { for range nodes {
multiErr = multierror.Append(multiErr, <-errCh) multiErr = multierror.Append(multiErr, <-errCh)
} }

View File

@ -57,6 +57,10 @@ func (reqs NodeRequests) FindInitNode() (req NodeRequest, err error) {
found := false found := false
for i := range reqs { for i := range reqs {
if reqs[i].Config == nil {
continue
}
if reqs[i].Config.Machine().Type() == machine.TypeInit { if reqs[i].Config.Machine().Type() == machine.TypeInit {
if found { if found {
err = fmt.Errorf("duplicate init node in requests") err = fmt.Errorf("duplicate init node in requests")
@ -78,6 +82,10 @@ func (reqs NodeRequests) FindInitNode() (req NodeRequest, err error) {
// MasterNodes returns subset of nodes which are Init/ControlPlane type. // MasterNodes returns subset of nodes which are Init/ControlPlane type.
func (reqs NodeRequests) MasterNodes() (nodes []NodeRequest) { func (reqs NodeRequests) MasterNodes() (nodes []NodeRequest) {
for i := range reqs { for i := range reqs {
if reqs[i].Config == nil {
continue
}
if reqs[i].Config.Machine().Type() == machine.TypeInit || reqs[i].Config.Machine().Type() == machine.TypeControlPlane { if reqs[i].Config.Machine().Type() == machine.TypeInit || reqs[i].Config.Machine().Type() == machine.TypeControlPlane {
nodes = append(nodes, reqs[i]) nodes = append(nodes, reqs[i])
} }
@ -89,6 +97,10 @@ func (reqs NodeRequests) MasterNodes() (nodes []NodeRequest) {
// WorkerNodes returns subset of nodes which are Init/ControlPlane type. // WorkerNodes returns subset of nodes which are Init/ControlPlane type.
func (reqs NodeRequests) WorkerNodes() (nodes []NodeRequest) { func (reqs NodeRequests) WorkerNodes() (nodes []NodeRequest) {
for i := range reqs { for i := range reqs {
if reqs[i].Config == nil {
continue
}
if reqs[i].Config.Machine().Type() == machine.TypeJoin { if reqs[i].Config.Machine().Type() == machine.TypeJoin {
nodes = append(nodes, reqs[i]) nodes = append(nodes, reqs[i])
} }
@ -97,6 +109,17 @@ func (reqs NodeRequests) WorkerNodes() (nodes []NodeRequest) {
return return
} }
// PXENodes returns subset of nodes which are PXE booted.
func (reqs NodeRequests) PXENodes() (nodes []NodeRequest) {
for i := range reqs {
if reqs[i].PXEBooted {
nodes = append(nodes, reqs[i])
}
}
return
}
// NodeRequest describes a request for a node. // NodeRequest describes a request for a node.
type NodeRequest struct { type NodeRequest struct {
Name string Name string
@ -111,4 +134,9 @@ type NodeRequest struct {
DiskSize int64 DiskSize int64
// Ports // Ports
Ports []string Ports []string
// PXE-booted VMs
PXEBooted bool
TFTPServer string
IPXEBootFilename string
} }

View File

@ -7,6 +7,8 @@ package provision
import ( import (
"net" "net"
"github.com/google/uuid"
"github.com/talos-systems/talos/pkg/machinery/config/types/v1alpha1/machine" "github.com/talos-systems/talos/pkg/machinery/config/types/v1alpha1/machine"
) )
@ -26,6 +28,9 @@ type ClusterInfo struct {
Network NetworkInfo Network NetworkInfo
Nodes []NodeInfo Nodes []NodeInfo
// ExtraNodes are not part of the cluster.
ExtraNodes []NodeInfo
} }
// NetworkInfo describes cluster network. // NetworkInfo describes cluster network.
@ -39,6 +44,7 @@ type NetworkInfo struct {
// NodeInfo describes a node. // NodeInfo describes a node.
type NodeInfo struct { type NodeInfo struct {
ID string ID string
UUID uuid.UUID
Name string Name string
Type machine.Type Type machine.Type