mirror of
https://github.com/siderolabs/talos.git
synced 2025-10-27 14:31:11 +01:00
feat: show siderolink status on dashboard
Add a new resource, `SiderolinkStatus`, which combines the following info: - The Siderolink API endpoint without the query parameters or fragments (potentially sensitive info due to the join token) - The status of the Siderolink connection This resource is not set as sensitive, so it can be retrieved by the users with `os:operator` role (e.g., using `talosctl dashboard` through Omni). Make use of this resource in the dashboard to display the status of the Siderolink connection. Additionally, rework the status columns in the dashboard to: - Display a Linux terminal compatible "tick" or a "cross" prefix for statuses in addition to the red/green color coding. - Move and combine some statuses to save rows and make them more even. Closes siderolabs/talos#8643. Signed-off-by: Utku Ozdemir <utku.ozdemir@siderolabs.com>
This commit is contained in:
parent
6f6a5d1057
commit
5ffc3f14bd
@ -15,6 +15,12 @@ message ConfigSpec {
|
||||
bool tunnel = 5;
|
||||
}
|
||||
|
||||
// StatusSpec describes Siderolink status.
|
||||
message StatusSpec {
|
||||
string host = 1;
|
||||
bool connected = 2;
|
||||
}
|
||||
|
||||
// TunnelSpec describes Siderolink GRPC Tunnel configuration.
|
||||
message TunnelSpec {
|
||||
string api_endpoint = 1;
|
||||
|
||||
@ -118,7 +118,7 @@ func (ctrl *DiagnosticsController) Run(ctx context.Context, r controller.Runtime
|
||||
return nil
|
||||
}
|
||||
|
||||
return safe.WriterModify(ctx, r, runtime.NewDiagnstic(runtime.NamespaceName, checkDescription.ID), func(res *runtime.Diagnostic) error {
|
||||
return safe.WriterModify(ctx, r, runtime.NewDiagnostic(runtime.NamespaceName, checkDescription.ID), func(res *runtime.Diagnostic) error {
|
||||
*res.TypedSpec() = *warning
|
||||
|
||||
return nil
|
||||
|
||||
@ -138,8 +138,13 @@ func (ctrl *ManagerController) Run(ctx context.Context, r controller.Runtime, lo
|
||||
case <-ctx.Done():
|
||||
return nil
|
||||
case <-ticker.C:
|
||||
reconnect, err := ctrl.shouldReconnect(wgClient)
|
||||
reconnect, err := peerDown(wgClient)
|
||||
if err != nil {
|
||||
if errors.Is(err, os.ErrNotExist) {
|
||||
// no Wireguard device, so no need to reconnect
|
||||
continue
|
||||
}
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
@ -476,27 +481,6 @@ func (ctrl *ManagerController) cleanupAddressSpecs(ctx context.Context, r contro
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ctrl *ManagerController) shouldReconnect(wgClient *wgctrl.Client) (bool, error) {
|
||||
wgDevice, err := wgClient.Device(constants.SideroLinkName)
|
||||
if err != nil {
|
||||
if errors.Is(err, os.ErrNotExist) {
|
||||
// no Wireguard device, so no need to reconnect
|
||||
return false, nil
|
||||
}
|
||||
|
||||
return false, fmt.Errorf("error reading Wireguard device: %w", err)
|
||||
}
|
||||
|
||||
if len(wgDevice.Peers) != 1 {
|
||||
return false, fmt.Errorf("unexpected number of Wireguard peers: %d", len(wgDevice.Peers))
|
||||
}
|
||||
|
||||
peer := wgDevice.Peers[0]
|
||||
since := time.Since(peer.LastHandshakeTime)
|
||||
|
||||
return since >= wireguard.PeerDownInterval, nil
|
||||
}
|
||||
|
||||
func withTransportCredentials(insec bool) grpc.DialOption {
|
||||
var transportCredentials credentials.TransportCredentials
|
||||
|
||||
|
||||
@ -4,3 +4,35 @@
|
||||
|
||||
// Package siderolink provides controllers which manage file resources.
|
||||
package siderolink
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/siderolabs/siderolink/pkg/wireguard"
|
||||
"golang.zx2c4.com/wireguard/wgctrl/wgtypes"
|
||||
|
||||
"github.com/siderolabs/talos/pkg/machinery/constants"
|
||||
)
|
||||
|
||||
// WireguardClient allows mocking Wireguard client.
|
||||
type WireguardClient interface {
|
||||
Device(string) (*wgtypes.Device, error)
|
||||
Close() error
|
||||
}
|
||||
|
||||
func peerDown(wgClient WireguardClient) (bool, error) {
|
||||
wgDevice, err := wgClient.Device(constants.SideroLinkName)
|
||||
if err != nil {
|
||||
return false, fmt.Errorf("error reading Wireguard device: %w", err)
|
||||
}
|
||||
|
||||
if len(wgDevice.Peers) != 1 {
|
||||
return false, fmt.Errorf("unexpected number of Wireguard peers: %d", len(wgDevice.Peers))
|
||||
}
|
||||
|
||||
peer := wgDevice.Peers[0]
|
||||
since := time.Since(peer.LastHandshakeTime)
|
||||
|
||||
return since >= wireguard.PeerDownInterval, nil
|
||||
}
|
||||
|
||||
158
internal/app/machined/pkg/controllers/siderolink/status.go
Normal file
158
internal/app/machined/pkg/controllers/siderolink/status.go
Normal file
@ -0,0 +1,158 @@
|
||||
// This Source Code Form is subject to the terms of the Mozilla Public
|
||||
// License, v. 2.0. If a copy of the MPL was not distributed with this
|
||||
// file, You can obtain one at http://mozilla.org/MPL/2.0/.
|
||||
|
||||
package siderolink
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"net"
|
||||
"net/url"
|
||||
"os"
|
||||
"time"
|
||||
|
||||
"github.com/cosi-project/runtime/pkg/controller"
|
||||
"github.com/cosi-project/runtime/pkg/safe"
|
||||
"github.com/cosi-project/runtime/pkg/state"
|
||||
"github.com/siderolabs/gen/optional"
|
||||
"go.uber.org/zap"
|
||||
"golang.zx2c4.com/wireguard/wgctrl"
|
||||
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/config"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/siderolink"
|
||||
)
|
||||
|
||||
// DefaultStatusUpdateInterval is the default interval between status updates.
|
||||
const DefaultStatusUpdateInterval = 30 * time.Second
|
||||
|
||||
// StatusController reports siderolink status.
|
||||
type StatusController struct {
|
||||
// WGClientFunc is a function that returns a WireguardClient.
|
||||
//
|
||||
// When nil, it defaults to an actual Wireguard client.
|
||||
WGClientFunc func() (WireguardClient, error)
|
||||
|
||||
// Interval is the time between peer status checks.
|
||||
//
|
||||
// When zero, it defaults to DefaultStatusUpdateInterval.
|
||||
Interval time.Duration
|
||||
}
|
||||
|
||||
// Name implements controller.Controller interface.
|
||||
func (ctrl *StatusController) Name() string {
|
||||
return "siderolink.StatusController"
|
||||
}
|
||||
|
||||
// Inputs implements controller.Controller interface.
|
||||
func (ctrl *StatusController) Inputs() []controller.Input {
|
||||
return []controller.Input{
|
||||
{
|
||||
Namespace: config.NamespaceName,
|
||||
Type: siderolink.ConfigType,
|
||||
ID: optional.Some(siderolink.ConfigID),
|
||||
Kind: controller.InputWeak,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// Outputs implements controller.Controller interface.
|
||||
func (ctrl *StatusController) Outputs() []controller.Output {
|
||||
return []controller.Output{
|
||||
{
|
||||
Type: siderolink.StatusType,
|
||||
Kind: controller.OutputExclusive,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// Run implements controller.Controller interface.
|
||||
func (ctrl *StatusController) Run(ctx context.Context, r controller.Runtime, _ *zap.Logger) error {
|
||||
interval := ctrl.Interval
|
||||
if interval == 0 {
|
||||
interval = DefaultStatusUpdateInterval
|
||||
}
|
||||
|
||||
ticker := time.NewTicker(interval)
|
||||
defer ticker.Stop()
|
||||
|
||||
wgClientFunc := ctrl.WGClientFunc
|
||||
if wgClientFunc == nil {
|
||||
wgClientFunc = func() (WireguardClient, error) {
|
||||
return wgctrl.New()
|
||||
}
|
||||
}
|
||||
|
||||
wgClient, err := wgClientFunc()
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to create wireguard client: %w", err)
|
||||
}
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return nil
|
||||
case <-r.EventCh():
|
||||
case <-ticker.C:
|
||||
}
|
||||
|
||||
r.StartTrackingOutputs()
|
||||
|
||||
if err = ctrl.reconcileStatus(ctx, r, wgClient); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if err = safe.CleanupOutputs[*siderolink.Status](ctx, r); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
r.ResetRestartBackoff()
|
||||
}
|
||||
}
|
||||
|
||||
func (ctrl *StatusController) reconcileStatus(ctx context.Context, r controller.Runtime, wgClient WireguardClient) (err error) {
|
||||
cfg, err := safe.ReaderGetByID[*siderolink.Config](ctx, r, siderolink.ConfigID)
|
||||
if err != nil {
|
||||
if state.IsNotFoundError(err) {
|
||||
return nil
|
||||
}
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
if cfg.TypedSpec().APIEndpoint == "" {
|
||||
return nil
|
||||
}
|
||||
|
||||
var parsed *url.URL
|
||||
|
||||
if parsed, err = url.Parse(cfg.TypedSpec().APIEndpoint); err != nil {
|
||||
return fmt.Errorf("failed to parse siderolink API endpoint: %w", err)
|
||||
}
|
||||
|
||||
host, _, err := net.SplitHostPort(parsed.Host)
|
||||
if err != nil {
|
||||
host = parsed.Host
|
||||
}
|
||||
|
||||
down, err := peerDown(wgClient)
|
||||
if err != nil {
|
||||
if !errors.Is(err, os.ErrNotExist) {
|
||||
return err
|
||||
}
|
||||
|
||||
down = true // wireguard device does not exist, we mark it as down
|
||||
}
|
||||
|
||||
if err = safe.WriterModify(ctx, r, siderolink.NewStatus(), func(status *siderolink.Status) error {
|
||||
status.TypedSpec().Host = host
|
||||
status.TypedSpec().Connected = !down
|
||||
|
||||
return nil
|
||||
}); err != nil {
|
||||
return fmt.Errorf("failed to update status: %w", err)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
132
internal/app/machined/pkg/controllers/siderolink/status_test.go
Normal file
132
internal/app/machined/pkg/controllers/siderolink/status_test.go
Normal file
@ -0,0 +1,132 @@
|
||||
// This Source Code Form is subject to the terms of the Mozilla Public
|
||||
// License, v. 2.0. If a copy of the MPL was not distributed with this
|
||||
// file, You can obtain one at http://mozilla.org/MPL/2.0/.
|
||||
|
||||
package siderolink_test
|
||||
|
||||
import (
|
||||
"os"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/cosi-project/runtime/pkg/resource"
|
||||
"github.com/cosi-project/runtime/pkg/resource/rtestutils"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/suite"
|
||||
"golang.zx2c4.com/wireguard/wgctrl/wgtypes"
|
||||
|
||||
"github.com/siderolabs/talos/internal/app/machined/pkg/controllers/ctest"
|
||||
siderolinkctrl "github.com/siderolabs/talos/internal/app/machined/pkg/controllers/siderolink"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/config"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/siderolink"
|
||||
)
|
||||
|
||||
type StatusSuite struct {
|
||||
ctest.DefaultSuite
|
||||
}
|
||||
|
||||
func TestStatusSuite(t *testing.T) {
|
||||
suite.Run(t, &StatusSuite{
|
||||
DefaultSuite: ctest.DefaultSuite{
|
||||
Timeout: 3 * time.Second,
|
||||
},
|
||||
})
|
||||
}
|
||||
|
||||
func (suite *StatusSuite) TestStatus() {
|
||||
wgClient := &mockWgClient{
|
||||
device: &wgtypes.Device{
|
||||
Peers: []wgtypes.Peer{
|
||||
{
|
||||
LastHandshakeTime: time.Now().Add(-time.Minute),
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
suite.Require().NoError(suite.Runtime().RegisterController(&siderolinkctrl.StatusController{
|
||||
WGClientFunc: func() (siderolinkctrl.WireguardClient, error) {
|
||||
return wgClient, nil
|
||||
},
|
||||
Interval: 100 * time.Millisecond,
|
||||
}))
|
||||
|
||||
rtestutils.AssertNoResource[*siderolink.Status](suite.Ctx(), suite.T(), suite.State(), siderolink.StatusID)
|
||||
|
||||
siderolinkConfig := siderolink.NewConfig(config.NamespaceName, siderolink.ConfigID)
|
||||
|
||||
siderolinkConfig.TypedSpec().APIEndpoint = "https://siderolink.example.org:1234?jointoken=supersecret&foo=bar#some=fragment"
|
||||
|
||||
suite.Require().NoError(suite.State().Create(suite.Ctx(), siderolinkConfig))
|
||||
|
||||
suite.assertStatus("siderolink.example.org", true)
|
||||
|
||||
// disconnect the peer
|
||||
|
||||
wgClient.setDevice(&wgtypes.Device{
|
||||
Peers: []wgtypes.Peer{
|
||||
{LastHandshakeTime: time.Now().Add(-time.Hour)},
|
||||
},
|
||||
})
|
||||
|
||||
// no device
|
||||
wgClient.setDevice(nil)
|
||||
suite.assertStatus("siderolink.example.org", false)
|
||||
|
||||
// reconnect the peer
|
||||
wgClient.setDevice(&wgtypes.Device{
|
||||
Peers: []wgtypes.Peer{
|
||||
{LastHandshakeTime: time.Now().Add(-5 * time.Second)},
|
||||
},
|
||||
})
|
||||
|
||||
suite.assertStatus("siderolink.example.org", true)
|
||||
|
||||
// update API endpoint
|
||||
|
||||
siderolinkConfig.TypedSpec().APIEndpoint = "https://new.example.org?jointoken=supersecret"
|
||||
|
||||
suite.Require().NoError(suite.State().Update(suite.Ctx(), siderolinkConfig))
|
||||
suite.assertStatus("new.example.org", true)
|
||||
|
||||
// no config
|
||||
|
||||
suite.Require().NoError(suite.State().Destroy(suite.Ctx(), siderolinkConfig.Metadata()))
|
||||
rtestutils.AssertNoResource[*siderolink.Status](suite.Ctx(), suite.T(), suite.State(), siderolink.StatusID)
|
||||
}
|
||||
|
||||
func (suite *StatusSuite) assertStatus(endpoint string, connected bool) {
|
||||
rtestutils.AssertResources(suite.Ctx(), suite.T(), suite.State(), []resource.ID{siderolink.StatusID},
|
||||
func(c *siderolink.Status, assert *assert.Assertions) {
|
||||
assert.Equal(endpoint, c.TypedSpec().Host)
|
||||
assert.Equal(connected, c.TypedSpec().Connected)
|
||||
})
|
||||
}
|
||||
|
||||
type mockWgClient struct {
|
||||
mu sync.Mutex
|
||||
device *wgtypes.Device
|
||||
}
|
||||
|
||||
func (m *mockWgClient) setDevice(device *wgtypes.Device) {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
|
||||
m.device = device
|
||||
}
|
||||
|
||||
func (m *mockWgClient) Device(string) (*wgtypes.Device, error) {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
|
||||
if m.device == nil {
|
||||
return nil, os.ErrNotExist
|
||||
}
|
||||
|
||||
return m.device, nil
|
||||
}
|
||||
|
||||
func (m *mockWgClient) Close() error {
|
||||
return nil
|
||||
}
|
||||
@ -336,6 +336,7 @@ func (ctrl *Controller) Run(ctx context.Context, drainer *runtime.Drainer) error
|
||||
V1Alpha1Mode: ctrl.v1alpha1Runtime.State().Platform().Mode(),
|
||||
},
|
||||
&siderolink.ManagerController{},
|
||||
&siderolink.StatusController{},
|
||||
&siderolink.UserspaceWireguardController{
|
||||
RelayRetryTimeout: 10 * time.Second,
|
||||
},
|
||||
|
||||
@ -214,6 +214,7 @@ func NewState() (*State, error) {
|
||||
&secrets.OSRoot{},
|
||||
&secrets.Trustd{},
|
||||
&siderolink.Config{},
|
||||
&siderolink.Status{},
|
||||
&siderolink.Tunnel{},
|
||||
&time.AdjtimeStatus{},
|
||||
&time.Status{},
|
||||
|
||||
@ -75,14 +75,26 @@ func formatStatus(status any) string {
|
||||
|
||||
switch strings.ToLower(statusStr) {
|
||||
case "running", "healthy", "true":
|
||||
return fmt.Sprintf("[green]%s[-]", statusStr)
|
||||
return formatText(statusStr, true)
|
||||
case "stopped", "unhealthy", "false":
|
||||
return fmt.Sprintf("[red]%s[-]", statusStr)
|
||||
return formatText(statusStr, false)
|
||||
default:
|
||||
return statusStr
|
||||
}
|
||||
}
|
||||
|
||||
func formatText(text string, ok bool) string {
|
||||
if text == "" {
|
||||
return ""
|
||||
}
|
||||
|
||||
if ok {
|
||||
return fmt.Sprintf("[green]√ %s[-]", text)
|
||||
}
|
||||
|
||||
return fmt.Sprintf("[red]× %s[-]", text)
|
||||
}
|
||||
|
||||
// capitalizeFirst capitalizes the first character of string.
|
||||
func capitalizeFirst(s string) string {
|
||||
if s == "" {
|
||||
|
||||
@ -26,6 +26,7 @@ type staticPodStatuses struct {
|
||||
|
||||
type kubernetesInfoData struct {
|
||||
isControlPlane bool
|
||||
typ string
|
||||
kubernetesVersion string
|
||||
kubeletStatus string
|
||||
|
||||
@ -106,7 +107,13 @@ func (widget *KubernetesInfo) updateNodeData(data resourcedata.Data) {
|
||||
|
||||
nodeData.podStatuses = widget.staticPodStatuses(maps.Values(nodeData.staticPodStatusMap))
|
||||
case *config.MachineType:
|
||||
nodeData.isControlPlane = !data.Deleted && res.MachineType() == machine.TypeControlPlane
|
||||
if data.Deleted {
|
||||
nodeData.isControlPlane = false
|
||||
nodeData.typ = notAvailable
|
||||
} else {
|
||||
nodeData.isControlPlane = res.MachineType() == machine.TypeControlPlane
|
||||
nodeData.typ = res.MachineType().String()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -128,6 +135,7 @@ func (widget *KubernetesInfo) getOrCreateNodeData(node string) *kubernetesInfoDa
|
||||
nodeData, ok := widget.nodeMap[node]
|
||||
if !ok {
|
||||
nodeData = &kubernetesInfoData{
|
||||
typ: notAvailable,
|
||||
kubernetesVersion: notAvailable,
|
||||
kubeletStatus: notAvailable,
|
||||
podStatuses: staticPodStatuses{
|
||||
@ -150,6 +158,10 @@ func (widget *KubernetesInfo) redraw() {
|
||||
fieldList := make([]field, 0, 5)
|
||||
|
||||
fieldList = append(fieldList,
|
||||
field{
|
||||
Name: "TYPE",
|
||||
Value: data.typ,
|
||||
},
|
||||
field{
|
||||
Name: "KUBERNETES",
|
||||
Value: data.kubernetesVersion,
|
||||
|
||||
@ -266,8 +266,8 @@ func (widget *NetworkInfo) timeservers(status *network.TimeServerStatus) string
|
||||
|
||||
func (widget *NetworkInfo) connectivity(status *network.Status) string {
|
||||
if status.TypedSpec().ConnectivityReady {
|
||||
return "[green]OK[-]"
|
||||
return "[green]√ OK[-]"
|
||||
}
|
||||
|
||||
return "[red]FAILED[-]"
|
||||
return "[red]× FAILED[-]"
|
||||
}
|
||||
|
||||
@ -6,29 +6,24 @@ package components
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strconv"
|
||||
"strings"
|
||||
|
||||
"github.com/rivo/tview"
|
||||
|
||||
"github.com/siderolabs/talos/internal/pkg/dashboard/resourcedata"
|
||||
"github.com/siderolabs/talos/pkg/machinery/constants"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/cluster"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/config"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/hardware"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/runtime"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/siderolink"
|
||||
)
|
||||
|
||||
type talosInfoData struct {
|
||||
uuid string
|
||||
clusterName string
|
||||
stage string
|
||||
ready string
|
||||
typ string
|
||||
numMachinesText string
|
||||
secureBootState string
|
||||
statePartitionMountStatus string
|
||||
ephemeralPartitionMountStatus string
|
||||
uuid string
|
||||
clusterName string
|
||||
siderolink string
|
||||
stage string
|
||||
ready string
|
||||
numMachinesText string
|
||||
secureBootState string
|
||||
|
||||
machineIDSet map[string]struct{}
|
||||
}
|
||||
@ -91,6 +86,12 @@ func (widget *TalosInfo) updateNodeData(data resourcedata.Data) {
|
||||
} else {
|
||||
nodeData.clusterName = clusterName
|
||||
}
|
||||
case *siderolink.Status:
|
||||
if data.Deleted {
|
||||
nodeData.siderolink = notAvailable
|
||||
} else {
|
||||
nodeData.siderolink = formatText(res.TypedSpec().Host, res.TypedSpec().Connected)
|
||||
}
|
||||
case *runtime.MachineStatus:
|
||||
if data.Deleted {
|
||||
nodeData.stage = notAvailable
|
||||
@ -105,28 +106,6 @@ func (widget *TalosInfo) updateNodeData(data resourcedata.Data) {
|
||||
} else {
|
||||
nodeData.secureBootState = formatStatus(res.TypedSpec().SecureBoot)
|
||||
}
|
||||
case *runtime.MountStatus:
|
||||
switch res.Metadata().ID() {
|
||||
case constants.StatePartitionLabel:
|
||||
if data.Deleted {
|
||||
nodeData.statePartitionMountStatus = notAvailable
|
||||
} else {
|
||||
nodeData.statePartitionMountStatus = mountStatus(res.TypedSpec().Encrypted, res.TypedSpec().EncryptionProviders)
|
||||
}
|
||||
case constants.EphemeralPartitionLabel:
|
||||
if data.Deleted {
|
||||
nodeData.ephemeralPartitionMountStatus = notAvailable
|
||||
} else {
|
||||
nodeData.ephemeralPartitionMountStatus = mountStatus(res.TypedSpec().Encrypted, res.TypedSpec().EncryptionProviders)
|
||||
}
|
||||
}
|
||||
|
||||
case *config.MachineType:
|
||||
if data.Deleted {
|
||||
nodeData.typ = notAvailable
|
||||
} else {
|
||||
nodeData.typ = res.MachineType().String()
|
||||
}
|
||||
case *cluster.Member:
|
||||
if data.Deleted {
|
||||
delete(nodeData.machineIDSet, res.Metadata().ID())
|
||||
@ -134,7 +113,12 @@ func (widget *TalosInfo) updateNodeData(data resourcedata.Data) {
|
||||
nodeData.machineIDSet[res.Metadata().ID()] = struct{}{}
|
||||
}
|
||||
|
||||
nodeData.numMachinesText = strconv.Itoa(len(nodeData.machineIDSet))
|
||||
suffix := ""
|
||||
if len(nodeData.machineIDSet) != 1 {
|
||||
suffix = "s"
|
||||
}
|
||||
|
||||
nodeData.numMachinesText = fmt.Sprintf("(%d machine%s)", len(nodeData.machineIDSet), suffix)
|
||||
}
|
||||
}
|
||||
|
||||
@ -142,16 +126,14 @@ func (widget *TalosInfo) getOrCreateNodeData(node string) *talosInfoData {
|
||||
nodeData, ok := widget.nodeMap[node]
|
||||
if !ok {
|
||||
nodeData = &talosInfoData{
|
||||
uuid: notAvailable,
|
||||
clusterName: notAvailable,
|
||||
stage: notAvailable,
|
||||
ready: notAvailable,
|
||||
typ: notAvailable,
|
||||
numMachinesText: notAvailable,
|
||||
secureBootState: notAvailable,
|
||||
statePartitionMountStatus: notAvailable,
|
||||
ephemeralPartitionMountStatus: notAvailable,
|
||||
machineIDSet: make(map[string]struct{}),
|
||||
uuid: notAvailable,
|
||||
clusterName: notAvailable,
|
||||
siderolink: notAvailable,
|
||||
stage: notAvailable,
|
||||
ready: notAvailable,
|
||||
numMachinesText: notAvailable,
|
||||
secureBootState: notAvailable,
|
||||
machineIDSet: make(map[string]struct{}),
|
||||
}
|
||||
|
||||
widget.nodeMap[node] = nodeData
|
||||
@ -171,7 +153,11 @@ func (widget *TalosInfo) redraw() {
|
||||
},
|
||||
{
|
||||
Name: "CLUSTER",
|
||||
Value: data.clusterName,
|
||||
Value: data.clusterName + " " + data.numMachinesText,
|
||||
},
|
||||
{
|
||||
Name: "SIDEROLINK",
|
||||
Value: data.siderolink,
|
||||
},
|
||||
{
|
||||
Name: "STAGE",
|
||||
@ -181,36 +167,12 @@ func (widget *TalosInfo) redraw() {
|
||||
Name: "READY",
|
||||
Value: data.ready,
|
||||
},
|
||||
{
|
||||
Name: "TYPE",
|
||||
Value: data.typ,
|
||||
},
|
||||
{
|
||||
Name: "MACHINES",
|
||||
Value: data.numMachinesText,
|
||||
},
|
||||
{
|
||||
Name: "SECUREBOOT",
|
||||
Value: data.secureBootState,
|
||||
},
|
||||
{
|
||||
Name: "STATE",
|
||||
Value: data.statePartitionMountStatus,
|
||||
},
|
||||
{
|
||||
Name: "EPHEMERAL",
|
||||
Value: data.ephemeralPartitionMountStatus,
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
widget.SetText(fields.String())
|
||||
}
|
||||
|
||||
func mountStatus(encrypted bool, providers []string) string {
|
||||
if !encrypted {
|
||||
return "[green]OK[-]"
|
||||
}
|
||||
|
||||
return fmt.Sprintf("[green]OK - encrypted[-] (%s)", strings.Join(providers, ","))
|
||||
}
|
||||
|
||||
@ -20,13 +20,13 @@ import (
|
||||
|
||||
"github.com/siderolabs/talos/internal/pkg/dashboard/util"
|
||||
"github.com/siderolabs/talos/pkg/machinery/client"
|
||||
"github.com/siderolabs/talos/pkg/machinery/constants"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/cluster"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/config"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/hardware"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/k8s"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/network"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/runtime"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/siderolink"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/v1alpha1"
|
||||
)
|
||||
|
||||
@ -114,14 +114,6 @@ func (source *Source) runResourceWatch(ctx context.Context, node string) error {
|
||||
return err
|
||||
}
|
||||
|
||||
if err := source.COSI.Watch(ctx, runtime.NewMountStatus(v1alpha1.NamespaceName, constants.StatePartitionLabel).Metadata(), eventCh); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if err := source.COSI.Watch(ctx, runtime.NewMountStatus(v1alpha1.NamespaceName, constants.EphemeralPartitionLabel).Metadata(), eventCh); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if err := source.COSI.Watch(ctx, config.NewMachineType().Metadata(), eventCh); err != nil {
|
||||
return err
|
||||
}
|
||||
@ -178,7 +170,11 @@ func (source *Source) runResourceWatch(ctx context.Context, node string) error {
|
||||
return err
|
||||
}
|
||||
|
||||
if err := source.COSI.WatchKind(ctx, runtime.NewDiagnstic(runtime.NamespaceName, "").Metadata(), eventCh, state.WithBootstrapContents(true)); err != nil {
|
||||
if err := source.COSI.WatchKind(ctx, siderolink.NewStatus().Metadata(), eventCh, state.WithBootstrapContents(true)); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if err := source.COSI.WatchKind(ctx, runtime.NewDiagnostic(runtime.NamespaceName, "").Metadata(), eventCh, state.WithBootstrapContents(true)); err != nil {
|
||||
if client.StatusCode(err) != codes.PermissionDenied {
|
||||
// ignore permission denied, means resource is not supported yet
|
||||
return err
|
||||
|
||||
@ -30,7 +30,7 @@ type SummaryGrid struct {
|
||||
diagnosticsVisible bool
|
||||
}
|
||||
|
||||
const summaryTopFixedRows = 8
|
||||
const summaryTopFixedRows = 7
|
||||
|
||||
// NewSummaryGrid initializes SummaryGrid.
|
||||
func NewSummaryGrid(app *tview.Application) *SummaryGrid {
|
||||
|
||||
@ -103,6 +103,62 @@ func (x *ConfigSpec) GetTunnel() bool {
|
||||
return false
|
||||
}
|
||||
|
||||
// StatusSpec describes Siderolink status.
|
||||
type StatusSpec struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Host string `protobuf:"bytes,1,opt,name=host,proto3" json:"host,omitempty"`
|
||||
Connected bool `protobuf:"varint,2,opt,name=connected,proto3" json:"connected,omitempty"`
|
||||
}
|
||||
|
||||
func (x *StatusSpec) Reset() {
|
||||
*x = StatusSpec{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_resource_definitions_siderolink_siderolink_proto_msgTypes[1]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *StatusSpec) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*StatusSpec) ProtoMessage() {}
|
||||
|
||||
func (x *StatusSpec) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_resource_definitions_siderolink_siderolink_proto_msgTypes[1]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use StatusSpec.ProtoReflect.Descriptor instead.
|
||||
func (*StatusSpec) Descriptor() ([]byte, []int) {
|
||||
return file_resource_definitions_siderolink_siderolink_proto_rawDescGZIP(), []int{1}
|
||||
}
|
||||
|
||||
func (x *StatusSpec) GetHost() string {
|
||||
if x != nil {
|
||||
return x.Host
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *StatusSpec) GetConnected() bool {
|
||||
if x != nil {
|
||||
return x.Connected
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
// TunnelSpec describes Siderolink GRPC Tunnel configuration.
|
||||
type TunnelSpec struct {
|
||||
state protoimpl.MessageState
|
||||
@ -118,7 +174,7 @@ type TunnelSpec struct {
|
||||
func (x *TunnelSpec) Reset() {
|
||||
*x = TunnelSpec{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_resource_definitions_siderolink_siderolink_proto_msgTypes[1]
|
||||
mi := &file_resource_definitions_siderolink_siderolink_proto_msgTypes[2]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@ -131,7 +187,7 @@ func (x *TunnelSpec) String() string {
|
||||
func (*TunnelSpec) ProtoMessage() {}
|
||||
|
||||
func (x *TunnelSpec) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_resource_definitions_siderolink_siderolink_proto_msgTypes[1]
|
||||
mi := &file_resource_definitions_siderolink_siderolink_proto_msgTypes[2]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@ -144,7 +200,7 @@ func (x *TunnelSpec) ProtoReflect() protoreflect.Message {
|
||||
|
||||
// Deprecated: Use TunnelSpec.ProtoReflect.Descriptor instead.
|
||||
func (*TunnelSpec) Descriptor() ([]byte, []int) {
|
||||
return file_resource_definitions_siderolink_siderolink_proto_rawDescGZIP(), []int{1}
|
||||
return file_resource_definitions_siderolink_siderolink_proto_rawDescGZIP(), []int{2}
|
||||
}
|
||||
|
||||
func (x *TunnelSpec) GetApiEndpoint() string {
|
||||
@ -194,7 +250,11 @@ var file_resource_definitions_siderolink_siderolink_proto_rawDesc = []byte{
|
||||
0x6b, 0x65, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x69, 0x6e, 0x73, 0x65, 0x63, 0x75, 0x72, 0x65, 0x18,
|
||||
0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x6e, 0x73, 0x65, 0x63, 0x75, 0x72, 0x65, 0x12,
|
||||
0x16, 0x0a, 0x06, 0x74, 0x75, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52,
|
||||
0x06, 0x74, 0x75, 0x6e, 0x6e, 0x65, 0x6c, 0x22, 0x94, 0x01, 0x0a, 0x0a, 0x54, 0x75, 0x6e, 0x6e,
|
||||
0x06, 0x74, 0x75, 0x6e, 0x6e, 0x65, 0x6c, 0x22, 0x3e, 0x0a, 0x0a, 0x53, 0x74, 0x61, 0x74, 0x75,
|
||||
0x73, 0x53, 0x70, 0x65, 0x63, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x6f, 0x73, 0x74, 0x18, 0x01, 0x20,
|
||||
0x01, 0x28, 0x09, 0x52, 0x04, 0x68, 0x6f, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6f, 0x6e,
|
||||
0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x63, 0x6f,
|
||||
0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x22, 0x94, 0x01, 0x0a, 0x0a, 0x54, 0x75, 0x6e, 0x6e,
|
||||
0x65, 0x6c, 0x53, 0x70, 0x65, 0x63, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x70, 0x69, 0x5f, 0x65, 0x6e,
|
||||
0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x61, 0x70,
|
||||
0x69, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x6c, 0x69, 0x6e,
|
||||
@ -224,14 +284,15 @@ func file_resource_definitions_siderolink_siderolink_proto_rawDescGZIP() []byte
|
||||
return file_resource_definitions_siderolink_siderolink_proto_rawDescData
|
||||
}
|
||||
|
||||
var file_resource_definitions_siderolink_siderolink_proto_msgTypes = make([]protoimpl.MessageInfo, 2)
|
||||
var file_resource_definitions_siderolink_siderolink_proto_msgTypes = make([]protoimpl.MessageInfo, 3)
|
||||
var file_resource_definitions_siderolink_siderolink_proto_goTypes = []interface{}{
|
||||
(*ConfigSpec)(nil), // 0: talos.resource.definitions.siderolink.ConfigSpec
|
||||
(*TunnelSpec)(nil), // 1: talos.resource.definitions.siderolink.TunnelSpec
|
||||
(*common.NetIPPort)(nil), // 2: common.NetIPPort
|
||||
(*StatusSpec)(nil), // 1: talos.resource.definitions.siderolink.StatusSpec
|
||||
(*TunnelSpec)(nil), // 2: talos.resource.definitions.siderolink.TunnelSpec
|
||||
(*common.NetIPPort)(nil), // 3: common.NetIPPort
|
||||
}
|
||||
var file_resource_definitions_siderolink_siderolink_proto_depIdxs = []int32{
|
||||
2, // 0: talos.resource.definitions.siderolink.TunnelSpec.node_address:type_name -> common.NetIPPort
|
||||
3, // 0: talos.resource.definitions.siderolink.TunnelSpec.node_address:type_name -> common.NetIPPort
|
||||
1, // [1:1] is the sub-list for method output_type
|
||||
1, // [1:1] is the sub-list for method input_type
|
||||
1, // [1:1] is the sub-list for extension type_name
|
||||
@ -258,6 +319,18 @@ func file_resource_definitions_siderolink_siderolink_proto_init() {
|
||||
}
|
||||
}
|
||||
file_resource_definitions_siderolink_siderolink_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*StatusSpec); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_resource_definitions_siderolink_siderolink_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*TunnelSpec); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
@ -276,7 +349,7 @@ func file_resource_definitions_siderolink_siderolink_proto_init() {
|
||||
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
||||
RawDescriptor: file_resource_definitions_siderolink_siderolink_proto_rawDesc,
|
||||
NumEnums: 0,
|
||||
NumMessages: 2,
|
||||
NumMessages: 3,
|
||||
NumExtensions: 0,
|
||||
NumServices: 0,
|
||||
},
|
||||
|
||||
@ -96,6 +96,56 @@ func (m *ConfigSpec) MarshalToSizedBufferVT(dAtA []byte) (int, error) {
|
||||
return len(dAtA) - i, nil
|
||||
}
|
||||
|
||||
func (m *StatusSpec) MarshalVT() (dAtA []byte, err error) {
|
||||
if m == nil {
|
||||
return nil, nil
|
||||
}
|
||||
size := m.SizeVT()
|
||||
dAtA = make([]byte, size)
|
||||
n, err := m.MarshalToSizedBufferVT(dAtA[:size])
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return dAtA[:n], nil
|
||||
}
|
||||
|
||||
func (m *StatusSpec) MarshalToVT(dAtA []byte) (int, error) {
|
||||
size := m.SizeVT()
|
||||
return m.MarshalToSizedBufferVT(dAtA[:size])
|
||||
}
|
||||
|
||||
func (m *StatusSpec) MarshalToSizedBufferVT(dAtA []byte) (int, error) {
|
||||
if m == nil {
|
||||
return 0, nil
|
||||
}
|
||||
i := len(dAtA)
|
||||
_ = i
|
||||
var l int
|
||||
_ = l
|
||||
if m.unknownFields != nil {
|
||||
i -= len(m.unknownFields)
|
||||
copy(dAtA[i:], m.unknownFields)
|
||||
}
|
||||
if m.Connected {
|
||||
i--
|
||||
if m.Connected {
|
||||
dAtA[i] = 1
|
||||
} else {
|
||||
dAtA[i] = 0
|
||||
}
|
||||
i--
|
||||
dAtA[i] = 0x10
|
||||
}
|
||||
if len(m.Host) > 0 {
|
||||
i -= len(m.Host)
|
||||
copy(dAtA[i:], m.Host)
|
||||
i = protohelpers.EncodeVarint(dAtA, i, uint64(len(m.Host)))
|
||||
i--
|
||||
dAtA[i] = 0xa
|
||||
}
|
||||
return len(dAtA) - i, nil
|
||||
}
|
||||
|
||||
func (m *TunnelSpec) MarshalVT() (dAtA []byte, err error) {
|
||||
if m == nil {
|
||||
return nil, nil
|
||||
@ -198,6 +248,23 @@ func (m *ConfigSpec) SizeVT() (n int) {
|
||||
return n
|
||||
}
|
||||
|
||||
func (m *StatusSpec) SizeVT() (n int) {
|
||||
if m == nil {
|
||||
return 0
|
||||
}
|
||||
var l int
|
||||
_ = l
|
||||
l = len(m.Host)
|
||||
if l > 0 {
|
||||
n += 1 + l + protohelpers.SizeOfVarint(uint64(l))
|
||||
}
|
||||
if m.Connected {
|
||||
n += 2
|
||||
}
|
||||
n += len(m.unknownFields)
|
||||
return n
|
||||
}
|
||||
|
||||
func (m *TunnelSpec) SizeVT() (n int) {
|
||||
if m == nil {
|
||||
return 0
|
||||
@ -416,6 +483,109 @@ func (m *ConfigSpec) UnmarshalVT(dAtA []byte) error {
|
||||
}
|
||||
return nil
|
||||
}
|
||||
func (m *StatusSpec) UnmarshalVT(dAtA []byte) error {
|
||||
l := len(dAtA)
|
||||
iNdEx := 0
|
||||
for iNdEx < l {
|
||||
preIndex := iNdEx
|
||||
var wire uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return protohelpers.ErrIntOverflow
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
wire |= uint64(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
fieldNum := int32(wire >> 3)
|
||||
wireType := int(wire & 0x7)
|
||||
if wireType == 4 {
|
||||
return fmt.Errorf("proto: StatusSpec: wiretype end group for non-group")
|
||||
}
|
||||
if fieldNum <= 0 {
|
||||
return fmt.Errorf("proto: StatusSpec: illegal tag %d (wire type %d)", fieldNum, wire)
|
||||
}
|
||||
switch fieldNum {
|
||||
case 1:
|
||||
if wireType != 2 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field Host", wireType)
|
||||
}
|
||||
var stringLen uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return protohelpers.ErrIntOverflow
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
stringLen |= uint64(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
intStringLen := int(stringLen)
|
||||
if intStringLen < 0 {
|
||||
return protohelpers.ErrInvalidLength
|
||||
}
|
||||
postIndex := iNdEx + intStringLen
|
||||
if postIndex < 0 {
|
||||
return protohelpers.ErrInvalidLength
|
||||
}
|
||||
if postIndex > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
m.Host = string(dAtA[iNdEx:postIndex])
|
||||
iNdEx = postIndex
|
||||
case 2:
|
||||
if wireType != 0 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field Connected", wireType)
|
||||
}
|
||||
var v int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return protohelpers.ErrIntOverflow
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
v |= int(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
m.Connected = bool(v != 0)
|
||||
default:
|
||||
iNdEx = preIndex
|
||||
skippy, err := protohelpers.Skip(dAtA[iNdEx:])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if (skippy < 0) || (iNdEx+skippy) < 0 {
|
||||
return protohelpers.ErrInvalidLength
|
||||
}
|
||||
if (iNdEx + skippy) > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...)
|
||||
iNdEx += skippy
|
||||
}
|
||||
}
|
||||
|
||||
if iNdEx > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
return nil
|
||||
}
|
||||
func (m *TunnelSpec) UnmarshalVT(dAtA []byte) error {
|
||||
l := len(dAtA)
|
||||
iNdEx := 0
|
||||
|
||||
@ -34,8 +34,8 @@ func (spec *DiagnosticSpec) DocumentationURL(id string) string {
|
||||
return "https://talos.dev/diagnostic/" + id
|
||||
}
|
||||
|
||||
// NewDiagnstic initializes a Diagnostic resource.
|
||||
func NewDiagnstic(namespace resource.Namespace, id resource.ID) *Diagnostic {
|
||||
// NewDiagnostic initializes a Diagnostic resource.
|
||||
func NewDiagnostic(namespace resource.Namespace, id resource.ID) *Diagnostic {
|
||||
return typed.NewResource[DiagnosticSpec, DiagnosticExtension](
|
||||
resource.NewMetadata(namespace, DiagnosticType, id, resource.VersionUndefined),
|
||||
DiagnosticSpec{},
|
||||
|
||||
@ -2,7 +2,7 @@
|
||||
// License, v. 2.0. If a copy of the MPL was not distributed with this
|
||||
// file, You can obtain one at http://mozilla.org/MPL/2.0/.
|
||||
|
||||
// Code generated by "deep-copy -type ConfigSpec -type TunnelSpec -header-file ../../../../hack/boilerplate.txt -o deep_copy.generated.go ."; DO NOT EDIT.
|
||||
// Code generated by "deep-copy -type ConfigSpec -type StatusSpec -type TunnelSpec -header-file ../../../../hack/boilerplate.txt -o deep_copy.generated.go ."; DO NOT EDIT.
|
||||
|
||||
package siderolink
|
||||
|
||||
@ -12,6 +12,12 @@ func (o ConfigSpec) DeepCopy() ConfigSpec {
|
||||
return cp
|
||||
}
|
||||
|
||||
// DeepCopy generates a deep copy of StatusSpec.
|
||||
func (o StatusSpec) DeepCopy() StatusSpec {
|
||||
var cp StatusSpec = o
|
||||
return cp
|
||||
}
|
||||
|
||||
// DeepCopy generates a deep copy of TunnelSpec.
|
||||
func (o TunnelSpec) DeepCopy() TunnelSpec {
|
||||
var cp TunnelSpec = o
|
||||
|
||||
@ -15,7 +15,7 @@ import (
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/config"
|
||||
)
|
||||
|
||||
//go:generate deep-copy -type ConfigSpec -type TunnelSpec -header-file ../../../../hack/boilerplate.txt -o deep_copy.generated.go .
|
||||
//go:generate deep-copy -type ConfigSpec -type StatusSpec -type TunnelSpec -header-file ../../../../hack/boilerplate.txt -o deep_copy.generated.go .
|
||||
|
||||
// ConfigType is type of Config resource.
|
||||
const ConfigType = resource.Type("SiderolinkConfigs.siderolink.talos.dev")
|
||||
|
||||
73
pkg/machinery/resources/siderolink/siderolink_status.go
Normal file
73
pkg/machinery/resources/siderolink/siderolink_status.go
Normal file
@ -0,0 +1,73 @@
|
||||
// This Source Code Form is subject to the terms of the Mozilla Public
|
||||
// License, v. 2.0. If a copy of the MPL was not distributed with this
|
||||
// file, You can obtain one at http://mozilla.org/MPL/2.0/.
|
||||
|
||||
package siderolink
|
||||
|
||||
import (
|
||||
"github.com/cosi-project/runtime/pkg/resource"
|
||||
"github.com/cosi-project/runtime/pkg/resource/meta"
|
||||
"github.com/cosi-project/runtime/pkg/resource/protobuf"
|
||||
"github.com/cosi-project/runtime/pkg/resource/typed"
|
||||
|
||||
"github.com/siderolabs/talos/pkg/machinery/proto"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/config"
|
||||
)
|
||||
|
||||
// StatusType is the type of Status resource.
|
||||
const StatusType = resource.Type("SiderolinkStatuses.siderolink.talos.dev")
|
||||
|
||||
// StatusID the singleton status resource ID.
|
||||
const StatusID = resource.ID("siderolink-status")
|
||||
|
||||
// Status resource holds Siderolink status.
|
||||
type Status = typed.Resource[StatusSpec, StatusExtension]
|
||||
|
||||
// StatusSpec describes Siderolink status.
|
||||
//
|
||||
//gotagsrewrite:gen
|
||||
type StatusSpec struct {
|
||||
// Host is the Siderolink target host.
|
||||
Host string `yaml:"host" protobuf:"1"`
|
||||
// Connected is the status of the Siderolink GRPC connection.
|
||||
Connected bool `yaml:"connected" protobuf:"2"`
|
||||
}
|
||||
|
||||
// NewStatus initializes a Status resource.
|
||||
func NewStatus() *Status {
|
||||
return typed.NewResource[StatusSpec, StatusExtension](
|
||||
resource.NewMetadata(config.NamespaceName, StatusType, StatusID, resource.VersionUndefined),
|
||||
StatusSpec{},
|
||||
)
|
||||
}
|
||||
|
||||
// StatusExtension provides auxiliary methods for Status.
|
||||
type StatusExtension struct{}
|
||||
|
||||
// ResourceDefinition implements [typed.Extension] interface.
|
||||
func (StatusExtension) ResourceDefinition() meta.ResourceDefinitionSpec {
|
||||
return meta.ResourceDefinitionSpec{
|
||||
Type: StatusType,
|
||||
Aliases: []resource.Type{},
|
||||
DefaultNamespace: config.NamespaceName,
|
||||
PrintColumns: []meta.PrintColumn{
|
||||
{
|
||||
Name: "Host",
|
||||
JSONPath: `{.host}`,
|
||||
},
|
||||
{
|
||||
Name: "Connected",
|
||||
JSONPath: `{.connected}`,
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func init() {
|
||||
proto.RegisterDefaultTypes()
|
||||
|
||||
err := protobuf.RegisterDynamic[StatusSpec](StatusType, &Status{})
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
@ -255,6 +255,7 @@ description: Talos gRPC API reference.
|
||||
|
||||
- [resource/definitions/siderolink/siderolink.proto](#resource/definitions/siderolink/siderolink.proto)
|
||||
- [ConfigSpec](#talos.resource.definitions.siderolink.ConfigSpec)
|
||||
- [StatusSpec](#talos.resource.definitions.siderolink.StatusSpec)
|
||||
- [TunnelSpec](#talos.resource.definitions.siderolink.TunnelSpec)
|
||||
|
||||
- [resource/definitions/time/time.proto](#resource/definitions/time/time.proto)
|
||||
@ -4612,6 +4613,22 @@ ConfigSpec describes Siderolink configuration.
|
||||
|
||||
|
||||
|
||||
<a name="talos.resource.definitions.siderolink.StatusSpec"></a>
|
||||
|
||||
### StatusSpec
|
||||
StatusSpec describes Siderolink status.
|
||||
|
||||
|
||||
| Field | Type | Label | Description |
|
||||
| ----- | ---- | ----- | ----------- |
|
||||
| host | [string](#string) | | |
|
||||
| connected | [bool](#bool) | | |
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
<a name="talos.resource.definitions.siderolink.TunnelSpec"></a>
|
||||
|
||||
### TunnelSpec
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user