mirror of
				https://github.com/siderolabs/talos.git
				synced 2025-10-27 06:21: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