mirror of
https://github.com/siderolabs/talos.git
synced 2025-10-05 20:51:15 +02:00
feat: introduce network probes
Network probes are configured with the specs, and provide their output as a status. At the moment only platform code can configure network probes. If any network probes are configured, they affect network.Status 'Connectivity' flag. Example, create the probe: ``` talosctl -n 172.20.0.3 meta write 0xa '{"probes": [{"interval": "1s", "tcp": {"endpoint": "google.com:80", "timeout": "10s"}}]}' ``` Watch probe status: ``` $ talosctl -n 172.20.0.3 get probe NODE NAMESPACE TYPE ID VERSION SUCCESS 172.20.0.3 network ProbeStatus tcp:google.com:80 5 true ``` With failing probes: ``` $ talosctl -n 172.20.0.3 get probe NODE NAMESPACE TYPE ID VERSION SUCCESS 172.20.0.3 network ProbeStatus tcp:google.com:80 4 true 172.20.0.3 network ProbeStatus tcp:google.com:81 1 false $ talosctl -n 172.20.0.3 get networkstatus NODE NAMESPACE TYPE ID VERSION ADDRESS CONNECTIVITY HOSTNAME ETC 172.20.0.3 network NetworkStatus status 5 true true true true ``` Signed-off-by: Andrey Smirnov <andrey.smirnov@talos-systems.com>
This commit is contained in:
parent
9dc1150e3a
commit
aa14993539
@ -188,6 +188,20 @@ message OperatorSpecSpec {
|
||||
talos.resource.definitions.enums.NetworkConfigLayer config_layer = 7;
|
||||
}
|
||||
|
||||
// ProbeSpecSpec describes the Probe.
|
||||
message ProbeSpecSpec {
|
||||
google.protobuf.Duration interval = 1;
|
||||
int64 failure_threshold = 2;
|
||||
TCPProbeSpec tcp = 3;
|
||||
talos.resource.definitions.enums.NetworkConfigLayer config_layer = 4;
|
||||
}
|
||||
|
||||
// ProbeStatusSpec describes the Probe.
|
||||
message ProbeStatusSpec {
|
||||
bool success = 1;
|
||||
string last_error = 2;
|
||||
}
|
||||
|
||||
// ResolverSpecSpec describes DNS resolvers.
|
||||
message ResolverSpecSpec {
|
||||
repeated common.NetIP dns_servers = 1;
|
||||
@ -246,6 +260,12 @@ message StatusSpec {
|
||||
bool etc_files_ready = 4;
|
||||
}
|
||||
|
||||
// TCPProbeSpec describes the TCP Probe.
|
||||
message TCPProbeSpec {
|
||||
string endpoint = 1;
|
||||
google.protobuf.Duration timeout = 2;
|
||||
}
|
||||
|
||||
// TimeServerSpecSpec describes NTP servers.
|
||||
message TimeServerSpecSpec {
|
||||
repeated string ntp_servers = 1;
|
||||
|
@ -39,11 +39,16 @@ type DefaultSuite struct { //nolint:govet
|
||||
|
||||
AfterSetup func(suite *DefaultSuite)
|
||||
AfterTearDown func(suite *DefaultSuite)
|
||||
Timeout time.Duration
|
||||
}
|
||||
|
||||
// SetupTest is a function for setting up a test.
|
||||
func (suite *DefaultSuite) SetupTest() {
|
||||
suite.ctx, suite.ctxCancel = context.WithTimeout(context.Background(), 3*time.Minute)
|
||||
if suite.Timeout == 0 {
|
||||
suite.Timeout = 3 * time.Minute
|
||||
}
|
||||
|
||||
suite.ctx, suite.ctxCancel = context.WithTimeout(context.Background(), suite.Timeout)
|
||||
|
||||
suite.state = state.WrapCore(namespaced.NewState(inmem.Build))
|
||||
|
||||
@ -103,7 +108,7 @@ func (suite *DefaultSuite) TearDownTest() {
|
||||
}
|
||||
}
|
||||
|
||||
// Suite is a type which dectibes the suite type.
|
||||
// Suite is a type which describes the suite type.
|
||||
type Suite interface {
|
||||
T() *testing.T
|
||||
Require() *require.Assertions
|
||||
|
@ -0,0 +1,159 @@
|
||||
// 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 probe contains implementation of the network probe runners.
|
||||
package probe
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"net"
|
||||
"sync"
|
||||
"syscall"
|
||||
|
||||
"github.com/benbjohnson/clock"
|
||||
"github.com/siderolabs/gen/channel"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/network"
|
||||
)
|
||||
|
||||
// Runner describes a state of running probe.
|
||||
type Runner struct {
|
||||
ID string
|
||||
Spec network.ProbeSpecSpec
|
||||
Clock clock.Clock
|
||||
|
||||
cancel context.CancelFunc
|
||||
wg sync.WaitGroup
|
||||
}
|
||||
|
||||
// Notification of a runner status.
|
||||
type Notification struct {
|
||||
ID string
|
||||
Status network.ProbeStatusSpec
|
||||
}
|
||||
|
||||
// Start a runner with a given context.
|
||||
func (runner *Runner) Start(ctx context.Context, notifyCh chan<- Notification, logger *zap.Logger) {
|
||||
runner.wg.Add(1)
|
||||
|
||||
ctx, runner.cancel = context.WithCancel(ctx)
|
||||
|
||||
go func() {
|
||||
defer runner.wg.Done()
|
||||
|
||||
runner.run(ctx, notifyCh, logger)
|
||||
}()
|
||||
}
|
||||
|
||||
// Stop a runner.
|
||||
func (runner *Runner) Stop() {
|
||||
runner.cancel()
|
||||
|
||||
runner.wg.Wait()
|
||||
}
|
||||
|
||||
// run a probe.
|
||||
//
|
||||
//nolint:gocyclo
|
||||
func (runner *Runner) run(ctx context.Context, notifyCh chan<- Notification, logger *zap.Logger) {
|
||||
logger = logger.With(zap.String("probe", runner.ID))
|
||||
|
||||
if runner.Clock == nil {
|
||||
runner.Clock = clock.New()
|
||||
}
|
||||
|
||||
ticker := runner.Clock.Ticker(runner.Spec.Interval)
|
||||
defer ticker.Stop()
|
||||
|
||||
consecutiveFailures := 0
|
||||
firstIteration := true
|
||||
|
||||
for {
|
||||
if !firstIteration {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-ticker.C:
|
||||
}
|
||||
} else {
|
||||
firstIteration = false
|
||||
}
|
||||
|
||||
err := runner.probe(ctx)
|
||||
if err == nil {
|
||||
if consecutiveFailures > 0 {
|
||||
logger.Info("probe succeeded")
|
||||
}
|
||||
|
||||
consecutiveFailures = 0
|
||||
|
||||
if !channel.SendWithContext(ctx, notifyCh, Notification{
|
||||
ID: runner.ID,
|
||||
Status: network.ProbeStatusSpec{
|
||||
Success: true,
|
||||
},
|
||||
}) {
|
||||
return
|
||||
}
|
||||
|
||||
continue
|
||||
}
|
||||
|
||||
if consecutiveFailures == runner.Spec.FailureThreshold {
|
||||
logger.Error("probe failed", zap.Error(err))
|
||||
}
|
||||
|
||||
consecutiveFailures++
|
||||
|
||||
if consecutiveFailures < runner.Spec.FailureThreshold {
|
||||
continue
|
||||
}
|
||||
|
||||
if !channel.SendWithContext(ctx, notifyCh, Notification{
|
||||
ID: runner.ID,
|
||||
Status: network.ProbeStatusSpec{
|
||||
Success: false,
|
||||
LastError: err.Error(),
|
||||
},
|
||||
}) {
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// probe runs a probe.
|
||||
func (runner *Runner) probe(ctx context.Context) error {
|
||||
var zeroTCP network.TCPProbeSpec
|
||||
|
||||
switch {
|
||||
case runner.Spec.TCP != zeroTCP:
|
||||
return runner.probeTCP(ctx)
|
||||
default:
|
||||
return fmt.Errorf("no probe type specified")
|
||||
}
|
||||
}
|
||||
|
||||
// probeTCP runs a TCP probe.
|
||||
func (runner *Runner) probeTCP(ctx context.Context) error {
|
||||
dialer := &net.Dialer{
|
||||
// The dialer reduces the TIME-WAIT period to 1 seconds instead of the OS default of 60 seconds.
|
||||
Control: func(network, address string, c syscall.RawConn) error {
|
||||
return c.Control(func(fd uintptr) {
|
||||
syscall.SetsockoptLinger(int(fd), syscall.SOL_SOCKET, syscall.SO_LINGER, &syscall.Linger{Onoff: 1, Linger: 1}) //nolint: errcheck
|
||||
})
|
||||
},
|
||||
}
|
||||
|
||||
ctx, cancel := context.WithTimeout(ctx, runner.Spec.TCP.Timeout)
|
||||
defer cancel()
|
||||
|
||||
conn, err := dialer.DialContext(ctx, "tcp", runner.Spec.TCP.Endpoint)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return conn.Close()
|
||||
}
|
@ -0,0 +1,152 @@
|
||||
// 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 probe_test
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net/http"
|
||||
"net/http/httptest"
|
||||
"net/url"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/benbjohnson/clock"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
"go.uber.org/zap/zaptest"
|
||||
|
||||
"github.com/siderolabs/talos/internal/app/machined/pkg/controllers/network/internal/probe"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/network"
|
||||
)
|
||||
|
||||
func TestProbeHTTP(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
server := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
|
||||
w.WriteHeader(http.StatusOK)
|
||||
}))
|
||||
t.Cleanup(server.Close)
|
||||
|
||||
u, err := url.Parse(server.URL)
|
||||
require.NoError(t, err)
|
||||
|
||||
p := probe.Runner{
|
||||
ID: "test",
|
||||
Spec: network.ProbeSpecSpec{
|
||||
Interval: 10 * time.Millisecond,
|
||||
TCP: network.TCPProbeSpec{
|
||||
Endpoint: u.Host,
|
||||
Timeout: time.Second,
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second)
|
||||
t.Cleanup(cancel)
|
||||
|
||||
notifyCh := make(chan probe.Notification)
|
||||
|
||||
p.Start(ctx, notifyCh, zaptest.NewLogger(t))
|
||||
t.Cleanup(p.Stop)
|
||||
|
||||
// probe should always succeed
|
||||
for i := 0; i < 3; i++ {
|
||||
assert.Equal(t, probe.Notification{
|
||||
ID: "test",
|
||||
Status: network.ProbeStatusSpec{
|
||||
Success: true,
|
||||
},
|
||||
}, <-notifyCh)
|
||||
}
|
||||
|
||||
// stop the test server, probe should fail
|
||||
server.Close()
|
||||
|
||||
for {
|
||||
notification := <-notifyCh
|
||||
|
||||
if notification.Status.Success {
|
||||
continue
|
||||
}
|
||||
|
||||
assert.Equal(t, "test", notification.ID)
|
||||
assert.False(t, notification.Status.Success)
|
||||
assert.Contains(t, notification.Status.LastError, "connection refused")
|
||||
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
func TestProbeConsecutiveFailures(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
server := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
|
||||
w.WriteHeader(http.StatusOK)
|
||||
}))
|
||||
t.Cleanup(server.Close)
|
||||
|
||||
u, err := url.Parse(server.URL)
|
||||
require.NoError(t, err)
|
||||
|
||||
mockClock := clock.NewMock()
|
||||
|
||||
p := probe.Runner{
|
||||
ID: "consecutive-failures",
|
||||
Spec: network.ProbeSpecSpec{
|
||||
Interval: 10 * time.Millisecond,
|
||||
FailureThreshold: 3,
|
||||
TCP: network.TCPProbeSpec{
|
||||
Endpoint: u.Host,
|
||||
Timeout: time.Second,
|
||||
},
|
||||
},
|
||||
Clock: mockClock,
|
||||
}
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second)
|
||||
t.Cleanup(cancel)
|
||||
|
||||
notifyCh := make(chan probe.Notification)
|
||||
|
||||
p.Start(ctx, notifyCh, zaptest.NewLogger(t))
|
||||
t.Cleanup(p.Stop)
|
||||
|
||||
// first iteration should succeed
|
||||
assert.Equal(t, probe.Notification{
|
||||
ID: "consecutive-failures",
|
||||
Status: network.ProbeStatusSpec{
|
||||
Success: true,
|
||||
},
|
||||
}, <-notifyCh)
|
||||
|
||||
// stop the test server, probe should fail
|
||||
server.Close()
|
||||
|
||||
for i := 0; i < p.Spec.FailureThreshold-1; i++ {
|
||||
// probe should fail, but no notification should be sent yet (failure threshold not reached)
|
||||
mockClock.Add(p.Spec.Interval)
|
||||
|
||||
select {
|
||||
case ev := <-notifyCh:
|
||||
require.Fail(t, "unexpected notification", "got: %v", ev)
|
||||
case <-time.After(100 * time.Millisecond):
|
||||
}
|
||||
}
|
||||
|
||||
// advance clock to trigger another failure(s)
|
||||
mockClock.Add(p.Spec.Interval)
|
||||
|
||||
notify := <-notifyCh
|
||||
assert.Equal(t, "consecutive-failures", notify.ID)
|
||||
assert.False(t, notify.Status.Success)
|
||||
assert.Contains(t, notify.Status.LastError, "connection refused")
|
||||
|
||||
// advance clock to trigger another failure(s)
|
||||
mockClock.Add(p.Spec.Interval)
|
||||
|
||||
notify = <-notifyCh
|
||||
assert.Equal(t, "consecutive-failures", notify.ID)
|
||||
assert.False(t, notify.Status.Success)
|
||||
}
|
@ -92,6 +92,10 @@ func (ctrl *PlatformConfigController) Outputs() []controller.Output {
|
||||
Type: network.OperatorSpecType,
|
||||
Kind: controller.OutputShared,
|
||||
},
|
||||
{
|
||||
Type: network.ProbeSpecType,
|
||||
Kind: controller.OutputShared,
|
||||
},
|
||||
{
|
||||
Type: runtimeres.PlatformMetadataType,
|
||||
Kind: controller.OutputExclusive,
|
||||
@ -214,7 +218,7 @@ func (ctrl *PlatformConfigController) apply(ctx context.Context, r controller.Ru
|
||||
for _, specType := range []struct {
|
||||
length int
|
||||
getter func(i int) interface{}
|
||||
idBuilder func(spec interface{}) resource.ID
|
||||
idBuilder func(spec interface{}) (resource.ID, error)
|
||||
resourceBuilder func(id string) resource.Resource
|
||||
resourceModifier func(newSpec interface{}) func(r resource.Resource) error
|
||||
}{
|
||||
@ -224,10 +228,10 @@ func (ctrl *PlatformConfigController) apply(ctx context.Context, r controller.Ru
|
||||
getter: func(i int) interface{} {
|
||||
return networkConfig.Addresses[i]
|
||||
},
|
||||
idBuilder: func(spec interface{}) resource.ID {
|
||||
idBuilder: func(spec interface{}) (resource.ID, error) {
|
||||
addressSpec := spec.(network.AddressSpecSpec) //nolint:errcheck,forcetypeassert
|
||||
|
||||
return network.LayeredID(network.ConfigPlatform, network.AddressID(addressSpec.LinkName, addressSpec.Address))
|
||||
return network.LayeredID(network.ConfigPlatform, network.AddressID(addressSpec.LinkName, addressSpec.Address)), nil
|
||||
},
|
||||
resourceBuilder: func(id string) resource.Resource {
|
||||
return network.NewAddressSpec(network.ConfigNamespaceName, id)
|
||||
@ -249,10 +253,10 @@ func (ctrl *PlatformConfigController) apply(ctx context.Context, r controller.Ru
|
||||
getter: func(i int) interface{} {
|
||||
return networkConfig.Links[i]
|
||||
},
|
||||
idBuilder: func(spec interface{}) resource.ID {
|
||||
idBuilder: func(spec interface{}) (resource.ID, error) {
|
||||
linkSpec := spec.(network.LinkSpecSpec) //nolint:errcheck,forcetypeassert
|
||||
|
||||
return network.LayeredID(network.ConfigPlatform, network.LinkID(linkSpec.Name))
|
||||
return network.LayeredID(network.ConfigPlatform, network.LinkID(linkSpec.Name)), nil
|
||||
},
|
||||
resourceBuilder: func(id string) resource.Resource {
|
||||
return network.NewLinkSpec(network.ConfigNamespaceName, id)
|
||||
@ -274,10 +278,10 @@ func (ctrl *PlatformConfigController) apply(ctx context.Context, r controller.Ru
|
||||
getter: func(i int) interface{} {
|
||||
return networkConfig.Routes[i]
|
||||
},
|
||||
idBuilder: func(spec interface{}) resource.ID {
|
||||
idBuilder: func(spec interface{}) (resource.ID, error) {
|
||||
routeSpec := spec.(network.RouteSpecSpec) //nolint:errcheck,forcetypeassert
|
||||
|
||||
return network.LayeredID(network.ConfigPlatform, network.RouteID(routeSpec.Table, routeSpec.Family, routeSpec.Destination, routeSpec.Gateway, routeSpec.Priority))
|
||||
return network.LayeredID(network.ConfigPlatform, network.RouteID(routeSpec.Table, routeSpec.Family, routeSpec.Destination, routeSpec.Gateway, routeSpec.Priority)), nil
|
||||
},
|
||||
resourceBuilder: func(id string) resource.Resource {
|
||||
return network.NewRouteSpec(network.ConfigNamespaceName, id)
|
||||
@ -299,8 +303,8 @@ func (ctrl *PlatformConfigController) apply(ctx context.Context, r controller.Ru
|
||||
getter: func(i int) interface{} {
|
||||
return networkConfig.Hostnames[i]
|
||||
},
|
||||
idBuilder: func(spec interface{}) resource.ID {
|
||||
return network.LayeredID(network.ConfigPlatform, network.HostnameID)
|
||||
idBuilder: func(spec interface{}) (resource.ID, error) {
|
||||
return network.LayeredID(network.ConfigPlatform, network.HostnameID), nil
|
||||
},
|
||||
resourceBuilder: func(id string) resource.Resource {
|
||||
return network.NewHostnameSpec(network.ConfigNamespaceName, id)
|
||||
@ -322,8 +326,8 @@ func (ctrl *PlatformConfigController) apply(ctx context.Context, r controller.Ru
|
||||
getter: func(i int) interface{} {
|
||||
return networkConfig.Resolvers[i]
|
||||
},
|
||||
idBuilder: func(spec interface{}) resource.ID {
|
||||
return network.LayeredID(network.ConfigPlatform, network.ResolverID)
|
||||
idBuilder: func(spec interface{}) (resource.ID, error) {
|
||||
return network.LayeredID(network.ConfigPlatform, network.ResolverID), nil
|
||||
},
|
||||
resourceBuilder: func(id string) resource.Resource {
|
||||
return network.NewResolverSpec(network.ConfigNamespaceName, id)
|
||||
@ -345,8 +349,8 @@ func (ctrl *PlatformConfigController) apply(ctx context.Context, r controller.Ru
|
||||
getter: func(i int) interface{} {
|
||||
return networkConfig.TimeServers[i]
|
||||
},
|
||||
idBuilder: func(spec interface{}) resource.ID {
|
||||
return network.LayeredID(network.ConfigPlatform, network.TimeServerID)
|
||||
idBuilder: func(spec interface{}) (resource.ID, error) {
|
||||
return network.LayeredID(network.ConfigPlatform, network.TimeServerID), nil
|
||||
},
|
||||
resourceBuilder: func(id string) resource.Resource {
|
||||
return network.NewTimeServerSpec(network.ConfigNamespaceName, id)
|
||||
@ -368,10 +372,10 @@ func (ctrl *PlatformConfigController) apply(ctx context.Context, r controller.Ru
|
||||
getter: func(i int) interface{} {
|
||||
return networkConfig.Operators[i]
|
||||
},
|
||||
idBuilder: func(spec interface{}) resource.ID {
|
||||
idBuilder: func(spec interface{}) (resource.ID, error) {
|
||||
operatorSpec := spec.(network.OperatorSpecSpec) //nolint:errcheck,forcetypeassert
|
||||
|
||||
return network.LayeredID(network.ConfigPlatform, network.OperatorID(operatorSpec.Operator, operatorSpec.LinkName))
|
||||
return network.LayeredID(network.ConfigPlatform, network.OperatorID(operatorSpec.Operator, operatorSpec.LinkName)), nil
|
||||
},
|
||||
resourceBuilder: func(id string) resource.Resource {
|
||||
return network.NewOperatorSpec(network.ConfigNamespaceName, id)
|
||||
@ -393,11 +397,11 @@ func (ctrl *PlatformConfigController) apply(ctx context.Context, r controller.Ru
|
||||
getter: func(i int) interface{} {
|
||||
return networkConfig.ExternalIPs[i]
|
||||
},
|
||||
idBuilder: func(spec interface{}) resource.ID {
|
||||
idBuilder: func(spec interface{}) (resource.ID, error) {
|
||||
ipAddr := spec.(netip.Addr) //nolint:errcheck,forcetypeassert
|
||||
ipPrefix := netip.PrefixFrom(ipAddr, ipAddr.BitLen())
|
||||
|
||||
return network.AddressID(externalLink, ipPrefix)
|
||||
return network.AddressID(externalLink, ipPrefix), nil
|
||||
},
|
||||
resourceBuilder: func(id string) resource.Resource {
|
||||
return network.NewAddressStatus(network.NamespaceName, id)
|
||||
@ -424,14 +428,39 @@ func (ctrl *PlatformConfigController) apply(ctx context.Context, r controller.Ru
|
||||
}
|
||||
},
|
||||
},
|
||||
// ProbeSpec
|
||||
{
|
||||
length: len(networkConfig.Probes),
|
||||
getter: func(i int) interface{} {
|
||||
return networkConfig.Probes[i]
|
||||
},
|
||||
idBuilder: func(spec interface{}) (resource.ID, error) {
|
||||
probeSpec := spec.(network.ProbeSpecSpec) //nolint:errcheck,forcetypeassert
|
||||
|
||||
return probeSpec.ID()
|
||||
},
|
||||
resourceBuilder: func(id string) resource.Resource {
|
||||
return network.NewProbeSpec(network.NamespaceName, id)
|
||||
},
|
||||
resourceModifier: func(newSpec interface{}) func(r resource.Resource) error {
|
||||
return func(r resource.Resource) error {
|
||||
spec := r.(*network.ProbeSpec).TypedSpec()
|
||||
|
||||
*spec = newSpec.(network.ProbeSpecSpec) //nolint:errcheck,forcetypeassert
|
||||
spec.ConfigLayer = network.ConfigPlatform
|
||||
|
||||
return nil
|
||||
}
|
||||
},
|
||||
},
|
||||
// Platform metadata
|
||||
{
|
||||
length: metadataLength,
|
||||
getter: func(i int) interface{} {
|
||||
return networkConfig.Metadata
|
||||
},
|
||||
idBuilder: func(spec interface{}) resource.ID {
|
||||
return runtimeres.PlatformMetadataID
|
||||
idBuilder: func(spec interface{}) (resource.ID, error) {
|
||||
return runtimeres.PlatformMetadataID, nil
|
||||
},
|
||||
resourceBuilder: func(id string) resource.Resource {
|
||||
return runtimeres.NewPlatformMetadataSpec(runtimeres.NamespaceName, id)
|
||||
@ -455,9 +484,13 @@ func (ctrl *PlatformConfigController) apply(ctx context.Context, r controller.Ru
|
||||
|
||||
for i := 0; i < specType.length; i++ {
|
||||
spec := specType.getter(i)
|
||||
id := specType.idBuilder(spec)
|
||||
|
||||
if err := r.Modify(ctx, specType.resourceBuilder(id), specType.resourceModifier(spec)); err != nil {
|
||||
id, err := specType.idBuilder(spec)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error building resource %s ID: %w", resourceType, err)
|
||||
}
|
||||
|
||||
if err = r.Modify(ctx, specType.resourceBuilder(id), specType.resourceModifier(spec)); err != nil {
|
||||
return fmt.Errorf("error modifying resource %s: %w", resourceType, err)
|
||||
}
|
||||
|
||||
|
@ -432,6 +432,42 @@ func (suite *PlatformConfigSuite) TestPlatformMockTimeServers() {
|
||||
)
|
||||
}
|
||||
|
||||
func (suite *PlatformConfigSuite) TestPlatformMockProbes() {
|
||||
suite.Require().NoError(
|
||||
suite.runtime.RegisterController(
|
||||
&netctrl.PlatformConfigController{
|
||||
V1alpha1Platform: &platformMock{
|
||||
tcpProbes: []string{"example.com:80", "example.com:443"},
|
||||
},
|
||||
StatePath: suite.statePath,
|
||||
PlatformState: suite.state,
|
||||
},
|
||||
),
|
||||
)
|
||||
|
||||
suite.startRuntime()
|
||||
|
||||
suite.Assert().NoError(
|
||||
retry.Constant(3*time.Second, retry.WithUnits(100*time.Millisecond)).Retry(
|
||||
func() error {
|
||||
return suite.assertResources(
|
||||
network.NamespaceName, network.ProbeSpecType, []string{
|
||||
"tcp:example.com:80",
|
||||
"tcp:example.com:443",
|
||||
}, func(r resource.Resource) error {
|
||||
spec := r.(*network.ProbeSpec).TypedSpec()
|
||||
|
||||
suite.Assert().Equal(time.Second, spec.Interval)
|
||||
suite.Assert().Equal(network.ConfigPlatform, spec.ConfigLayer)
|
||||
|
||||
return nil
|
||||
},
|
||||
)
|
||||
},
|
||||
),
|
||||
)
|
||||
}
|
||||
|
||||
func (suite *PlatformConfigSuite) TestPlatformMockExternalIPs() {
|
||||
suite.Require().NoError(
|
||||
suite.runtime.RegisterController(
|
||||
@ -658,6 +694,7 @@ type platformMock struct {
|
||||
resolvers []netip.Addr
|
||||
timeServers []string
|
||||
dhcp4Links []string
|
||||
tcpProbes []string
|
||||
|
||||
metadata *runtimeres.PlatformMetadataSpec
|
||||
}
|
||||
@ -785,6 +822,18 @@ func (mock *platformMock) NetworkConfiguration(
|
||||
)
|
||||
}
|
||||
|
||||
for _, endpoint := range mock.tcpProbes {
|
||||
networkConfig.Probes = append(
|
||||
networkConfig.Probes, network.ProbeSpecSpec{
|
||||
Interval: time.Second,
|
||||
TCP: network.TCPProbeSpec{
|
||||
Endpoint: endpoint,
|
||||
Timeout: time.Second,
|
||||
},
|
||||
ConfigLayer: network.ConfigPlatform,
|
||||
})
|
||||
}
|
||||
|
||||
networkConfig.Metadata = mock.metadata
|
||||
|
||||
select {
|
||||
|
156
internal/app/machined/pkg/controllers/network/probe.go
Normal file
156
internal/app/machined/pkg/controllers/network/probe.go
Normal file
@ -0,0 +1,156 @@
|
||||
// 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 network
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
|
||||
"github.com/cosi-project/runtime/pkg/controller"
|
||||
"github.com/cosi-project/runtime/pkg/resource"
|
||||
"github.com/cosi-project/runtime/pkg/safe"
|
||||
"github.com/cosi-project/runtime/pkg/state"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/siderolabs/talos/internal/app/machined/pkg/controllers/network/internal/probe"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/network"
|
||||
)
|
||||
|
||||
// ProbeController runs network probes configured with ProbeSpecs and outputs ProbeStatuses.
|
||||
type ProbeController struct {
|
||||
runners map[string]*probe.Runner
|
||||
}
|
||||
|
||||
// Name implements controller.Controller interface.
|
||||
func (ctrl *ProbeController) Name() string {
|
||||
return "network.ProbeController"
|
||||
}
|
||||
|
||||
// Inputs implements controller.Controller interface.
|
||||
func (ctrl *ProbeController) Inputs() []controller.Input {
|
||||
return []controller.Input{
|
||||
{
|
||||
Namespace: network.NamespaceName,
|
||||
Type: network.ProbeSpecType,
|
||||
Kind: controller.InputWeak,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// Outputs implements controller.Controller interface.
|
||||
func (ctrl *ProbeController) Outputs() []controller.Output {
|
||||
return []controller.Output{
|
||||
{
|
||||
Type: network.ProbeStatusType,
|
||||
Kind: controller.OutputExclusive,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// Run implements controller.Controller interface.
|
||||
func (ctrl *ProbeController) Run(ctx context.Context, r controller.Runtime, logger *zap.Logger) error {
|
||||
notifyCh := make(chan probe.Notification)
|
||||
|
||||
ctrl.runners = make(map[string]*probe.Runner)
|
||||
|
||||
defer func() {
|
||||
for _, runner := range ctrl.runners {
|
||||
runner.Stop()
|
||||
}
|
||||
}()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return nil
|
||||
case <-r.EventCh():
|
||||
if err := ctrl.reconcileRunners(ctx, r, logger, notifyCh); err != nil {
|
||||
return err
|
||||
}
|
||||
case ev := <-notifyCh:
|
||||
if err := ctrl.reconcileOutputs(ctx, r, ev); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
r.ResetRestartBackoff()
|
||||
}
|
||||
}
|
||||
|
||||
//nolint:gocyclo
|
||||
func (ctrl *ProbeController) reconcileRunners(ctx context.Context, r controller.Runtime, logger *zap.Logger, notifyCh chan<- probe.Notification) error {
|
||||
specList, err := safe.ReaderList[*network.ProbeSpec](ctx, r, resource.NewMetadata(network.NamespaceName, network.ProbeSpecType, "", resource.VersionUndefined))
|
||||
if err != nil {
|
||||
return fmt.Errorf("error listing probe specs: %w", err)
|
||||
}
|
||||
|
||||
// figure out which operators should run
|
||||
shouldRun := make(map[string]network.ProbeSpecSpec)
|
||||
|
||||
for iter := safe.IteratorFromList(specList); iter.Next(); {
|
||||
shouldRun[iter.Value().Metadata().ID()] = *iter.Value().TypedSpec()
|
||||
}
|
||||
|
||||
// stop running probes which shouldn't run
|
||||
for id := range ctrl.runners {
|
||||
if _, exists := shouldRun[id]; !exists {
|
||||
logger.Debug("stopping probe", zap.String("probe", id))
|
||||
|
||||
ctrl.runners[id].Stop()
|
||||
delete(ctrl.runners, id)
|
||||
} else if !shouldRun[id].Equal(ctrl.runners[id].Spec) {
|
||||
logger.Debug("replacing probe", zap.String("probe", id))
|
||||
|
||||
ctrl.runners[id].Stop()
|
||||
delete(ctrl.runners, id)
|
||||
}
|
||||
}
|
||||
|
||||
// start probes which aren't running
|
||||
for id := range shouldRun {
|
||||
if _, exists := ctrl.runners[id]; !exists {
|
||||
ctrl.runners[id] = &probe.Runner{
|
||||
ID: id,
|
||||
Spec: shouldRun[id],
|
||||
}
|
||||
|
||||
logger.Debug("starting probe", zap.String("probe", id))
|
||||
ctrl.runners[id].Start(ctx, notifyCh, logger)
|
||||
}
|
||||
}
|
||||
|
||||
// clean up statuses which should no longer exist
|
||||
statusList, err := safe.ReaderList[*network.ProbeStatus](ctx, r, resource.NewMetadata(network.NamespaceName, network.ProbeStatusType, "", resource.VersionUndefined))
|
||||
if err != nil {
|
||||
return fmt.Errorf("error listing probe statuses: %w", err)
|
||||
}
|
||||
|
||||
for iter := safe.IteratorFromList(statusList); iter.Next(); {
|
||||
if _, exists := shouldRun[iter.Value().Metadata().ID()]; exists {
|
||||
continue
|
||||
}
|
||||
|
||||
if err = r.Destroy(ctx, iter.Value().Metadata()); err != nil && !state.IsNotFoundError(err) {
|
||||
return fmt.Errorf("error destroying probe status: %w", err)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
//nolint:gocyclo,cyclop
|
||||
func (ctrl *ProbeController) reconcileOutputs(ctx context.Context, r controller.Runtime, ev probe.Notification) error {
|
||||
if _, exists := ctrl.runners[ev.ID]; !exists {
|
||||
// probe was already removed, late notification, ignore it
|
||||
return nil
|
||||
}
|
||||
|
||||
return safe.WriterModify(ctx, r, network.NewProbeStatus(network.NamespaceName, ev.ID),
|
||||
func(status *network.ProbeStatus) error {
|
||||
*status.TypedSpec() = ev.Status
|
||||
|
||||
return nil
|
||||
})
|
||||
}
|
91
internal/app/machined/pkg/controllers/network/probe_test.go
Normal file
91
internal/app/machined/pkg/controllers/network/probe_test.go
Normal file
@ -0,0 +1,91 @@
|
||||
// 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 network_test
|
||||
|
||||
import (
|
||||
"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"
|
||||
|
||||
"github.com/siderolabs/talos/internal/app/machined/pkg/controllers/ctest"
|
||||
networkctrl "github.com/siderolabs/talos/internal/app/machined/pkg/controllers/network"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/network"
|
||||
)
|
||||
|
||||
type ProbeSuite struct {
|
||||
ctest.DefaultSuite
|
||||
}
|
||||
|
||||
func (suite *ProbeSuite) TestReconcile() {
|
||||
googleProbeSpec := network.ProbeSpecSpec{
|
||||
Interval: 100 * time.Millisecond,
|
||||
TCP: network.TCPProbeSpec{
|
||||
Endpoint: "google.com:80",
|
||||
Timeout: 5 * time.Second,
|
||||
},
|
||||
}
|
||||
googleProbeSpecID, err := googleProbeSpec.ID()
|
||||
suite.Require().NoError(err)
|
||||
|
||||
probeGoogle := network.NewProbeSpec(network.NamespaceName, googleProbeSpecID)
|
||||
*probeGoogle.TypedSpec() = googleProbeSpec
|
||||
suite.Require().NoError(suite.State().Create(suite.Ctx(), probeGoogle))
|
||||
|
||||
rtestutils.AssertResources(suite.Ctx(), suite.T(), suite.State(), []resource.ID{googleProbeSpecID}, func(r *network.ProbeStatus, assert *assert.Assertions) {
|
||||
assert.Equal(network.ProbeStatusSpec{
|
||||
Success: true,
|
||||
}, *r.TypedSpec())
|
||||
})
|
||||
|
||||
failingProbeSpec := network.ProbeSpecSpec{
|
||||
Interval: 100 * time.Millisecond,
|
||||
FailureThreshold: 1,
|
||||
TCP: network.TCPProbeSpec{
|
||||
Endpoint: "google.com:81",
|
||||
Timeout: time.Second,
|
||||
},
|
||||
}
|
||||
failingProbeSpecID, err := failingProbeSpec.ID()
|
||||
suite.Require().NoError(err)
|
||||
|
||||
probeFailing := network.NewProbeSpec(network.NamespaceName, failingProbeSpecID)
|
||||
*probeFailing.TypedSpec() = failingProbeSpec
|
||||
suite.Require().NoError(suite.State().Create(suite.Ctx(), probeFailing))
|
||||
|
||||
rtestutils.AssertResources(suite.Ctx(), suite.T(), suite.State(), []resource.ID{failingProbeSpecID}, func(r *network.ProbeStatus, assert *assert.Assertions) {
|
||||
assert.False(r.TypedSpec().Success)
|
||||
})
|
||||
|
||||
probeFailing.TypedSpec().TCP.Endpoint = "google.com:443"
|
||||
suite.Require().NoError(suite.State().Update(suite.Ctx(), probeFailing))
|
||||
|
||||
rtestutils.AssertResources(suite.Ctx(), suite.T(), suite.State(), []resource.ID{failingProbeSpecID}, func(r *network.ProbeStatus, assert *assert.Assertions) {
|
||||
assert.Equal(network.ProbeStatusSpec{
|
||||
Success: true,
|
||||
}, *r.TypedSpec())
|
||||
})
|
||||
|
||||
suite.Require().NoError(suite.State().Destroy(suite.Ctx(), probeFailing.Metadata()))
|
||||
suite.Require().NoError(suite.State().Destroy(suite.Ctx(), probeGoogle.Metadata()))
|
||||
|
||||
rtestutils.AssertNoResource[*network.ProbeStatus](suite.Ctx(), suite.T(), suite.State(), failingProbeSpecID)
|
||||
rtestutils.AssertNoResource[*network.ProbeStatus](suite.Ctx(), suite.T(), suite.State(), googleProbeSpecID)
|
||||
}
|
||||
|
||||
// TestProbeSuite runs the ProbeSuite.
|
||||
func TestProbeSuite(t *testing.T) {
|
||||
suite.Run(t, &ProbeSuite{
|
||||
DefaultSuite: ctest.DefaultSuite{
|
||||
Timeout: 20 * time.Second,
|
||||
AfterSetup: func(suite *ctest.DefaultSuite) {
|
||||
suite.Require().NoError(suite.Runtime().RegisterController(&networkctrl.ProbeController{}))
|
||||
},
|
||||
},
|
||||
})
|
||||
}
|
@ -10,6 +10,7 @@ import (
|
||||
|
||||
"github.com/cosi-project/runtime/pkg/controller"
|
||||
"github.com/cosi-project/runtime/pkg/resource"
|
||||
"github.com/cosi-project/runtime/pkg/safe"
|
||||
"github.com/cosi-project/runtime/pkg/state"
|
||||
"github.com/siderolabs/gen/value"
|
||||
"github.com/siderolabs/go-pointer"
|
||||
@ -51,6 +52,11 @@ func (ctrl *StatusController) Inputs() []controller.Input {
|
||||
Type: files.EtcFileStatusType,
|
||||
Kind: controller.InputWeak,
|
||||
},
|
||||
{
|
||||
Namespace: network.NamespaceName,
|
||||
Type: network.ProbeStatusType,
|
||||
Kind: controller.InputWeak,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
@ -66,7 +72,7 @@ func (ctrl *StatusController) Outputs() []controller.Output {
|
||||
|
||||
// Run implements controller.Controller interface.
|
||||
//
|
||||
//nolint:gocyclo
|
||||
//nolint:gocyclo,cyclop
|
||||
func (ctrl *StatusController) Run(ctx context.Context, r controller.Runtime, logger *zap.Logger) error {
|
||||
for {
|
||||
select {
|
||||
@ -78,29 +84,51 @@ func (ctrl *StatusController) Run(ctx context.Context, r controller.Runtime, log
|
||||
result := network.StatusSpec{}
|
||||
|
||||
// addresses
|
||||
currentAddresses, err := r.Get(ctx, resource.NewMetadata(network.NamespaceName, network.NodeAddressType, network.NodeAddressCurrentID, resource.VersionUndefined))
|
||||
currentAddresses, err := safe.ReaderGet[*network.NodeAddress](ctx, r, resource.NewMetadata(network.NamespaceName, network.NodeAddressType, network.NodeAddressCurrentID, resource.VersionUndefined))
|
||||
if err != nil {
|
||||
if !state.IsNotFoundError(err) {
|
||||
return fmt.Errorf("error getting resource: %w", err)
|
||||
}
|
||||
} else {
|
||||
result.AddressReady = len(currentAddresses.(*network.NodeAddress).TypedSpec().Addresses) > 0
|
||||
result.AddressReady = len(currentAddresses.TypedSpec().Addresses) > 0
|
||||
}
|
||||
|
||||
// connectivity
|
||||
list, err := r.List(ctx, resource.NewMetadata(network.NamespaceName, network.RouteStatusType, "", resource.VersionUndefined))
|
||||
// if any probes are defined, use their status, otherwise rely on presence of the default gateway
|
||||
probeStatuses, err := safe.ReaderList[*network.ProbeStatus](ctx, r, resource.NewMetadata(network.NamespaceName, network.ProbeStatusType, "", resource.VersionUndefined))
|
||||
if err != nil {
|
||||
return fmt.Errorf("error getting routes: %w", err)
|
||||
return fmt.Errorf("error getting probe statuses: %w", err)
|
||||
}
|
||||
|
||||
for _, item := range list.Items {
|
||||
if value.IsZero(item.(*network.RouteStatus).TypedSpec().Destination) {
|
||||
result.ConnectivityReady = true
|
||||
allProbesSuccess := true
|
||||
|
||||
for iter := safe.IteratorFromList(probeStatuses); iter.Next(); {
|
||||
if !iter.Value().TypedSpec().Success {
|
||||
allProbesSuccess = false
|
||||
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
if probeStatuses.Len() > 0 && allProbesSuccess {
|
||||
result.ConnectivityReady = true
|
||||
} else if probeStatuses.Len() == 0 {
|
||||
var routes safe.List[*network.RouteStatus]
|
||||
|
||||
routes, err = safe.ReaderList[*network.RouteStatus](ctx, r, resource.NewMetadata(network.NamespaceName, network.RouteStatusType, "", resource.VersionUndefined))
|
||||
if err != nil {
|
||||
return fmt.Errorf("error getting routes: %w", err)
|
||||
}
|
||||
|
||||
for iter := safe.IteratorFromList(routes); iter.Next(); {
|
||||
if value.IsZero(iter.Value().TypedSpec().Destination) {
|
||||
result.ConnectivityReady = true
|
||||
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// hostname
|
||||
_, err = r.Get(ctx, resource.NewMetadata(network.NamespaceName, network.HostnameStatusType, network.HostnameID, resource.VersionUndefined))
|
||||
if err != nil {
|
||||
@ -128,9 +156,9 @@ func (ctrl *StatusController) Run(ctx context.Context, r controller.Runtime, log
|
||||
}
|
||||
|
||||
// update output status
|
||||
if err = r.Modify(ctx, network.NewStatus(network.NamespaceName, network.StatusID),
|
||||
func(r resource.Resource) error {
|
||||
*r.(*network.Status).TypedSpec() = result
|
||||
if err = safe.WriterModify(ctx, r, network.NewStatus(network.NamespaceName, network.StatusID),
|
||||
func(r *network.Status) error {
|
||||
*r.TypedSpec() = result
|
||||
|
||||
return nil
|
||||
}); err != nil {
|
||||
|
@ -6,184 +6,106 @@
|
||||
package network_test
|
||||
|
||||
import (
|
||||
"context"
|
||||
"log"
|
||||
"net/netip"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/cosi-project/runtime/pkg/controller/runtime"
|
||||
"github.com/cosi-project/runtime/pkg/resource"
|
||||
"github.com/cosi-project/runtime/pkg/state"
|
||||
"github.com/cosi-project/runtime/pkg/state/impl/inmem"
|
||||
"github.com/cosi-project/runtime/pkg/state/impl/namespaced"
|
||||
"github.com/siderolabs/go-retry/retry"
|
||||
"github.com/cosi-project/runtime/pkg/resource/rtestutils"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/suite"
|
||||
|
||||
"github.com/siderolabs/talos/internal/app/machined/pkg/controllers/ctest"
|
||||
netctrl "github.com/siderolabs/talos/internal/app/machined/pkg/controllers/network"
|
||||
"github.com/siderolabs/talos/pkg/logging"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/files"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/network"
|
||||
)
|
||||
|
||||
type StatusSuite struct {
|
||||
suite.Suite
|
||||
|
||||
state state.State
|
||||
|
||||
runtime *runtime.Runtime
|
||||
wg sync.WaitGroup
|
||||
|
||||
ctx context.Context //nolint:containedctx
|
||||
ctxCancel context.CancelFunc
|
||||
}
|
||||
|
||||
func (suite *StatusSuite) SetupTest() {
|
||||
suite.ctx, suite.ctxCancel = context.WithTimeout(context.Background(), 3*time.Minute)
|
||||
|
||||
suite.state = state.WrapCore(namespaced.NewState(inmem.Build))
|
||||
|
||||
var err error
|
||||
|
||||
suite.runtime, err = runtime.NewRuntime(suite.state, logging.Wrap(log.Writer()))
|
||||
suite.Require().NoError(err)
|
||||
|
||||
suite.Require().NoError(suite.runtime.RegisterController(&netctrl.StatusController{}))
|
||||
|
||||
suite.startRuntime()
|
||||
}
|
||||
|
||||
func (suite *StatusSuite) startRuntime() {
|
||||
suite.wg.Add(1)
|
||||
|
||||
go func() {
|
||||
defer suite.wg.Done()
|
||||
|
||||
suite.Assert().NoError(suite.runtime.Run(suite.ctx))
|
||||
}()
|
||||
}
|
||||
|
||||
func (suite *StatusSuite) assertStatus(expected network.StatusSpec) error {
|
||||
status, err := suite.state.Get(
|
||||
suite.ctx,
|
||||
resource.NewMetadata(network.NamespaceName, network.StatusType, network.StatusID, resource.VersionUndefined),
|
||||
)
|
||||
if err != nil {
|
||||
if !state.IsNotFoundError(err) {
|
||||
suite.Require().NoError(err)
|
||||
}
|
||||
|
||||
return retry.ExpectedError(err)
|
||||
}
|
||||
|
||||
if *status.(*network.Status).TypedSpec() != expected {
|
||||
return retry.ExpectedErrorf("status %+v != expected %+v", *status.(*network.Status).TypedSpec(), expected)
|
||||
}
|
||||
|
||||
return nil
|
||||
ctest.DefaultSuite
|
||||
}
|
||||
|
||||
func (suite *StatusSuite) TestNone() {
|
||||
suite.Assert().NoError(
|
||||
retry.Constant(3*time.Second, retry.WithUnits(100*time.Millisecond)).Retry(
|
||||
func() error {
|
||||
return suite.assertStatus(network.StatusSpec{})
|
||||
},
|
||||
),
|
||||
)
|
||||
rtestutils.AssertResources(suite.Ctx(), suite.T(), suite.State(), []resource.ID{network.StatusID}, func(r *network.Status, assert *assert.Assertions) {
|
||||
assert.Equal(network.StatusSpec{}, *r.TypedSpec())
|
||||
})
|
||||
}
|
||||
|
||||
func (suite *StatusSuite) TestAddresses() {
|
||||
nodeAddress := network.NewNodeAddress(network.NamespaceName, network.NodeAddressCurrentID)
|
||||
nodeAddress.TypedSpec().Addresses = []netip.Prefix{netip.MustParsePrefix("10.0.0.1/24")}
|
||||
|
||||
suite.Require().NoError(suite.state.Create(suite.ctx, nodeAddress))
|
||||
suite.Require().NoError(suite.State().Create(suite.Ctx(), nodeAddress))
|
||||
|
||||
suite.Assert().NoError(
|
||||
retry.Constant(3*time.Second, retry.WithUnits(100*time.Millisecond)).Retry(
|
||||
func() error {
|
||||
return suite.assertStatus(network.StatusSpec{AddressReady: true})
|
||||
},
|
||||
),
|
||||
)
|
||||
rtestutils.AssertResources(suite.Ctx(), suite.T(), suite.State(), []resource.ID{network.StatusID}, func(r *network.Status, assert *assert.Assertions) {
|
||||
assert.Equal(network.StatusSpec{AddressReady: true}, *r.TypedSpec())
|
||||
})
|
||||
}
|
||||
|
||||
func (suite *StatusSuite) TestRoutes() {
|
||||
route := network.NewRouteStatus(network.NamespaceName, "foo")
|
||||
route.TypedSpec().Gateway = netip.MustParseAddr("10.0.0.1")
|
||||
|
||||
suite.Require().NoError(suite.state.Create(suite.ctx, route))
|
||||
suite.Require().NoError(suite.State().Create(suite.Ctx(), route))
|
||||
|
||||
suite.Assert().NoError(
|
||||
retry.Constant(3*time.Second, retry.WithUnits(100*time.Millisecond)).Retry(
|
||||
func() error {
|
||||
return suite.assertStatus(network.StatusSpec{ConnectivityReady: true})
|
||||
},
|
||||
),
|
||||
)
|
||||
rtestutils.AssertResources(suite.Ctx(), suite.T(), suite.State(), []resource.ID{network.StatusID}, func(r *network.Status, assert *assert.Assertions) {
|
||||
assert.Equal(network.StatusSpec{ConnectivityReady: true}, *r.TypedSpec())
|
||||
})
|
||||
}
|
||||
|
||||
func (suite *StatusSuite) TestProbeStatuses() {
|
||||
probeStatus := network.NewProbeStatus(network.NamespaceName, "foo")
|
||||
probeStatus.TypedSpec().Success = true
|
||||
suite.Require().NoError(suite.State().Create(suite.Ctx(), probeStatus))
|
||||
|
||||
rtestutils.AssertResources(suite.Ctx(), suite.T(), suite.State(), []resource.ID{network.StatusID}, func(r *network.Status, assert *assert.Assertions) {
|
||||
assert.Equal(network.StatusSpec{ConnectivityReady: true}, *r.TypedSpec())
|
||||
})
|
||||
|
||||
// failing probe make status not ready
|
||||
route := network.NewRouteStatus(network.NamespaceName, "foo")
|
||||
route.TypedSpec().Gateway = netip.MustParseAddr("10.0.0.1")
|
||||
|
||||
suite.Require().NoError(suite.State().Create(suite.Ctx(), route))
|
||||
|
||||
probeStatusFail := network.NewProbeStatus(network.NamespaceName, "failing")
|
||||
suite.Require().NoError(suite.State().Create(suite.Ctx(), probeStatusFail))
|
||||
|
||||
rtestutils.AssertResources(suite.Ctx(), suite.T(), suite.State(), []resource.ID{network.StatusID}, func(r *network.Status, assert *assert.Assertions) {
|
||||
assert.Equal(network.StatusSpec{}, *r.TypedSpec())
|
||||
})
|
||||
}
|
||||
|
||||
func (suite *StatusSuite) TestHostname() {
|
||||
hostname := network.NewHostnameStatus(network.NamespaceName, network.HostnameID)
|
||||
hostname.TypedSpec().Hostname = "foo"
|
||||
|
||||
suite.Require().NoError(suite.state.Create(suite.ctx, hostname))
|
||||
suite.Require().NoError(suite.State().Create(suite.Ctx(), hostname))
|
||||
|
||||
suite.Assert().NoError(
|
||||
retry.Constant(3*time.Second, retry.WithUnits(100*time.Millisecond)).Retry(
|
||||
func() error {
|
||||
return suite.assertStatus(network.StatusSpec{HostnameReady: true})
|
||||
},
|
||||
),
|
||||
)
|
||||
rtestutils.AssertResources(suite.Ctx(), suite.T(), suite.State(), []resource.ID{network.StatusID}, func(r *network.Status, assert *assert.Assertions) {
|
||||
assert.Equal(network.StatusSpec{HostnameReady: true}, *r.TypedSpec())
|
||||
})
|
||||
}
|
||||
|
||||
func (suite *StatusSuite) TestEtcFiles() {
|
||||
hosts := files.NewEtcFileStatus(files.NamespaceName, "hosts")
|
||||
resolv := files.NewEtcFileStatus(files.NamespaceName, "resolv.conf")
|
||||
|
||||
suite.Require().NoError(suite.state.Create(suite.ctx, hosts))
|
||||
suite.Require().NoError(suite.state.Create(suite.ctx, resolv))
|
||||
suite.Require().NoError(suite.State().Create(suite.Ctx(), hosts))
|
||||
suite.Require().NoError(suite.State().Create(suite.Ctx(), resolv))
|
||||
|
||||
suite.Assert().NoError(
|
||||
retry.Constant(3*time.Second, retry.WithUnits(100*time.Millisecond)).Retry(
|
||||
func() error {
|
||||
return suite.assertStatus(network.StatusSpec{EtcFilesReady: true})
|
||||
},
|
||||
),
|
||||
)
|
||||
}
|
||||
|
||||
func (suite *StatusSuite) TearDownTest() {
|
||||
suite.T().Log("tear down")
|
||||
|
||||
suite.ctxCancel()
|
||||
|
||||
suite.wg.Wait()
|
||||
|
||||
// trigger updates in resources to stop watch loops
|
||||
suite.Assert().NoError(
|
||||
suite.state.Create(
|
||||
context.Background(),
|
||||
network.NewNodeAddress(network.NamespaceName, "bar"),
|
||||
),
|
||||
)
|
||||
suite.Assert().NoError(
|
||||
suite.state.Create(
|
||||
context.Background(),
|
||||
network.NewResolverStatus(network.NamespaceName, "bar"),
|
||||
),
|
||||
)
|
||||
suite.Assert().NoError(
|
||||
suite.state.Create(
|
||||
context.Background(),
|
||||
network.NewHostnameStatus(network.NamespaceName, "bar"),
|
||||
),
|
||||
)
|
||||
suite.Assert().NoError(suite.state.Create(context.Background(), files.NewEtcFileStatus(files.NamespaceName, "bar")))
|
||||
rtestutils.AssertResources(suite.Ctx(), suite.T(), suite.State(), []resource.ID{network.StatusID}, func(r *network.Status, assert *assert.Assertions) {
|
||||
assert.Equal(network.StatusSpec{EtcFilesReady: true}, *r.TypedSpec())
|
||||
})
|
||||
}
|
||||
|
||||
func TestStatusSuite(t *testing.T) {
|
||||
suite.Run(t, new(StatusSuite))
|
||||
suite.Run(t, &StatusSuite{
|
||||
DefaultSuite: ctest.DefaultSuite{
|
||||
Timeout: 3 * time.Second,
|
||||
AfterSetup: func(suite *ctest.DefaultSuite) {
|
||||
suite.Require().NoError(suite.Runtime().RegisterController(&netctrl.StatusController{}))
|
||||
},
|
||||
},
|
||||
})
|
||||
}
|
||||
|
@ -57,5 +57,7 @@ type PlatformNetworkConfig struct {
|
||||
|
||||
ExternalIPs []netip.Addr `yaml:"externalIPs"`
|
||||
|
||||
Probes []network.ProbeSpecSpec `yaml:"probes,omitempty"`
|
||||
|
||||
Metadata *runtime.PlatformMetadataSpec `yaml:"metadata,omitempty"`
|
||||
}
|
||||
|
@ -193,6 +193,7 @@ func (ctrl *Controller) Run(ctx context.Context, drainer *runtime.Drainer) error
|
||||
V1alpha1Platform: ctrl.v1alpha1Runtime.State().Platform(),
|
||||
PlatformState: ctrl.v1alpha1Runtime.State().V1Alpha2().Resources(),
|
||||
},
|
||||
&network.ProbeController{},
|
||||
&network.ResolverConfigController{
|
||||
Cmdline: procfs.ProcCmdline(),
|
||||
},
|
||||
|
@ -151,6 +151,8 @@ func NewState() (*State, error) {
|
||||
&network.NodeAddress{},
|
||||
&network.NodeAddressFilter{},
|
||||
&network.OperatorSpec{},
|
||||
&network.ProbeSpec{},
|
||||
&network.ProbeStatus{},
|
||||
&network.ResolverStatus{},
|
||||
&network.ResolverSpec{},
|
||||
&network.RouteStatus{},
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -1528,6 +1528,131 @@ func (m *OperatorSpecSpec) MarshalToSizedBufferVT(dAtA []byte) (int, error) {
|
||||
return len(dAtA) - i, nil
|
||||
}
|
||||
|
||||
func (m *ProbeSpecSpec) 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 *ProbeSpecSpec) MarshalToVT(dAtA []byte) (int, error) {
|
||||
size := m.SizeVT()
|
||||
return m.MarshalToSizedBufferVT(dAtA[:size])
|
||||
}
|
||||
|
||||
func (m *ProbeSpecSpec) 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.ConfigLayer != 0 {
|
||||
i = encodeVarint(dAtA, i, uint64(m.ConfigLayer))
|
||||
i--
|
||||
dAtA[i] = 0x20
|
||||
}
|
||||
if m.Tcp != nil {
|
||||
size, err := m.Tcp.MarshalToSizedBufferVT(dAtA[:i])
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
i -= size
|
||||
i = encodeVarint(dAtA, i, uint64(size))
|
||||
i--
|
||||
dAtA[i] = 0x1a
|
||||
}
|
||||
if m.FailureThreshold != 0 {
|
||||
i = encodeVarint(dAtA, i, uint64(m.FailureThreshold))
|
||||
i--
|
||||
dAtA[i] = 0x10
|
||||
}
|
||||
if m.Interval != nil {
|
||||
if vtmsg, ok := interface{}(m.Interval).(interface {
|
||||
MarshalToSizedBufferVT([]byte) (int, error)
|
||||
}); ok {
|
||||
size, err := vtmsg.MarshalToSizedBufferVT(dAtA[:i])
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
i -= size
|
||||
i = encodeVarint(dAtA, i, uint64(size))
|
||||
} else {
|
||||
encoded, err := proto.Marshal(m.Interval)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
i -= len(encoded)
|
||||
copy(dAtA[i:], encoded)
|
||||
i = encodeVarint(dAtA, i, uint64(len(encoded)))
|
||||
}
|
||||
i--
|
||||
dAtA[i] = 0xa
|
||||
}
|
||||
return len(dAtA) - i, nil
|
||||
}
|
||||
|
||||
func (m *ProbeStatusSpec) 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 *ProbeStatusSpec) MarshalToVT(dAtA []byte) (int, error) {
|
||||
size := m.SizeVT()
|
||||
return m.MarshalToSizedBufferVT(dAtA[:size])
|
||||
}
|
||||
|
||||
func (m *ProbeStatusSpec) 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 len(m.LastError) > 0 {
|
||||
i -= len(m.LastError)
|
||||
copy(dAtA[i:], m.LastError)
|
||||
i = encodeVarint(dAtA, i, uint64(len(m.LastError)))
|
||||
i--
|
||||
dAtA[i] = 0x12
|
||||
}
|
||||
if m.Success {
|
||||
i--
|
||||
if m.Success {
|
||||
dAtA[i] = 1
|
||||
} else {
|
||||
dAtA[i] = 0
|
||||
}
|
||||
i--
|
||||
dAtA[i] = 0x8
|
||||
}
|
||||
return len(dAtA) - i, nil
|
||||
}
|
||||
|
||||
func (m *ResolverSpecSpec) MarshalVT() (dAtA []byte, err error) {
|
||||
if m == nil {
|
||||
return nil, nil
|
||||
@ -2065,6 +2190,68 @@ func (m *StatusSpec) MarshalToSizedBufferVT(dAtA []byte) (int, error) {
|
||||
return len(dAtA) - i, nil
|
||||
}
|
||||
|
||||
func (m *TCPProbeSpec) 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 *TCPProbeSpec) MarshalToVT(dAtA []byte) (int, error) {
|
||||
size := m.SizeVT()
|
||||
return m.MarshalToSizedBufferVT(dAtA[:size])
|
||||
}
|
||||
|
||||
func (m *TCPProbeSpec) 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.Timeout != nil {
|
||||
if vtmsg, ok := interface{}(m.Timeout).(interface {
|
||||
MarshalToSizedBufferVT([]byte) (int, error)
|
||||
}); ok {
|
||||
size, err := vtmsg.MarshalToSizedBufferVT(dAtA[:i])
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
i -= size
|
||||
i = encodeVarint(dAtA, i, uint64(size))
|
||||
} else {
|
||||
encoded, err := proto.Marshal(m.Timeout)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
i -= len(encoded)
|
||||
copy(dAtA[i:], encoded)
|
||||
i = encodeVarint(dAtA, i, uint64(len(encoded)))
|
||||
}
|
||||
i--
|
||||
dAtA[i] = 0x12
|
||||
}
|
||||
if len(m.Endpoint) > 0 {
|
||||
i -= len(m.Endpoint)
|
||||
copy(dAtA[i:], m.Endpoint)
|
||||
i = encodeVarint(dAtA, i, uint64(len(m.Endpoint)))
|
||||
i--
|
||||
dAtA[i] = 0xa
|
||||
}
|
||||
return len(dAtA) - i, nil
|
||||
}
|
||||
|
||||
func (m *TimeServerSpecSpec) MarshalVT() (dAtA []byte, err error) {
|
||||
if m == nil {
|
||||
return nil, nil
|
||||
@ -3186,6 +3373,53 @@ func (m *OperatorSpecSpec) SizeVT() (n int) {
|
||||
return n
|
||||
}
|
||||
|
||||
func (m *ProbeSpecSpec) SizeVT() (n int) {
|
||||
if m == nil {
|
||||
return 0
|
||||
}
|
||||
var l int
|
||||
_ = l
|
||||
if m.Interval != nil {
|
||||
if size, ok := interface{}(m.Interval).(interface {
|
||||
SizeVT() int
|
||||
}); ok {
|
||||
l = size.SizeVT()
|
||||
} else {
|
||||
l = proto.Size(m.Interval)
|
||||
}
|
||||
n += 1 + l + sov(uint64(l))
|
||||
}
|
||||
if m.FailureThreshold != 0 {
|
||||
n += 1 + sov(uint64(m.FailureThreshold))
|
||||
}
|
||||
if m.Tcp != nil {
|
||||
l = m.Tcp.SizeVT()
|
||||
n += 1 + l + sov(uint64(l))
|
||||
}
|
||||
if m.ConfigLayer != 0 {
|
||||
n += 1 + sov(uint64(m.ConfigLayer))
|
||||
}
|
||||
n += len(m.unknownFields)
|
||||
return n
|
||||
}
|
||||
|
||||
func (m *ProbeStatusSpec) SizeVT() (n int) {
|
||||
if m == nil {
|
||||
return 0
|
||||
}
|
||||
var l int
|
||||
_ = l
|
||||
if m.Success {
|
||||
n += 2
|
||||
}
|
||||
l = len(m.LastError)
|
||||
if l > 0 {
|
||||
n += 1 + l + sov(uint64(l))
|
||||
}
|
||||
n += len(m.unknownFields)
|
||||
return n
|
||||
}
|
||||
|
||||
func (m *ResolverSpecSpec) SizeVT() (n int) {
|
||||
if m == nil {
|
||||
return 0
|
||||
@ -3410,6 +3644,30 @@ func (m *StatusSpec) SizeVT() (n int) {
|
||||
return n
|
||||
}
|
||||
|
||||
func (m *TCPProbeSpec) SizeVT() (n int) {
|
||||
if m == nil {
|
||||
return 0
|
||||
}
|
||||
var l int
|
||||
_ = l
|
||||
l = len(m.Endpoint)
|
||||
if l > 0 {
|
||||
n += 1 + l + sov(uint64(l))
|
||||
}
|
||||
if m.Timeout != nil {
|
||||
if size, ok := interface{}(m.Timeout).(interface {
|
||||
SizeVT() int
|
||||
}); ok {
|
||||
l = size.SizeVT()
|
||||
} else {
|
||||
l = proto.Size(m.Timeout)
|
||||
}
|
||||
n += 1 + l + sov(uint64(l))
|
||||
}
|
||||
n += len(m.unknownFields)
|
||||
return n
|
||||
}
|
||||
|
||||
func (m *TimeServerSpecSpec) SizeVT() (n int) {
|
||||
if m == nil {
|
||||
return 0
|
||||
@ -7427,6 +7685,278 @@ func (m *OperatorSpecSpec) UnmarshalVT(dAtA []byte) error {
|
||||
}
|
||||
return nil
|
||||
}
|
||||
func (m *ProbeSpecSpec) 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 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: ProbeSpecSpec: wiretype end group for non-group")
|
||||
}
|
||||
if fieldNum <= 0 {
|
||||
return fmt.Errorf("proto: ProbeSpecSpec: illegal tag %d (wire type %d)", fieldNum, wire)
|
||||
}
|
||||
switch fieldNum {
|
||||
case 1:
|
||||
if wireType != 2 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field Interval", wireType)
|
||||
}
|
||||
var msglen int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflow
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
msglen |= int(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
if msglen < 0 {
|
||||
return ErrInvalidLength
|
||||
}
|
||||
postIndex := iNdEx + msglen
|
||||
if postIndex < 0 {
|
||||
return ErrInvalidLength
|
||||
}
|
||||
if postIndex > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
if m.Interval == nil {
|
||||
m.Interval = &durationpb.Duration{}
|
||||
}
|
||||
if unmarshal, ok := interface{}(m.Interval).(interface {
|
||||
UnmarshalVT([]byte) error
|
||||
}); ok {
|
||||
if err := unmarshal.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil {
|
||||
return err
|
||||
}
|
||||
} else {
|
||||
if err := proto.Unmarshal(dAtA[iNdEx:postIndex], m.Interval); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
iNdEx = postIndex
|
||||
case 2:
|
||||
if wireType != 0 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field FailureThreshold", wireType)
|
||||
}
|
||||
m.FailureThreshold = 0
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflow
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
m.FailureThreshold |= int64(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
case 3:
|
||||
if wireType != 2 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field Tcp", wireType)
|
||||
}
|
||||
var msglen int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflow
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
msglen |= int(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
if msglen < 0 {
|
||||
return ErrInvalidLength
|
||||
}
|
||||
postIndex := iNdEx + msglen
|
||||
if postIndex < 0 {
|
||||
return ErrInvalidLength
|
||||
}
|
||||
if postIndex > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
if m.Tcp == nil {
|
||||
m.Tcp = &TCPProbeSpec{}
|
||||
}
|
||||
if err := m.Tcp.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil {
|
||||
return err
|
||||
}
|
||||
iNdEx = postIndex
|
||||
case 4:
|
||||
if wireType != 0 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field ConfigLayer", wireType)
|
||||
}
|
||||
m.ConfigLayer = 0
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflow
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
m.ConfigLayer |= enums.NetworkConfigLayer(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
default:
|
||||
iNdEx = preIndex
|
||||
skippy, err := skip(dAtA[iNdEx:])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if (skippy < 0) || (iNdEx+skippy) < 0 {
|
||||
return 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 *ProbeStatusSpec) 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 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: ProbeStatusSpec: wiretype end group for non-group")
|
||||
}
|
||||
if fieldNum <= 0 {
|
||||
return fmt.Errorf("proto: ProbeStatusSpec: illegal tag %d (wire type %d)", fieldNum, wire)
|
||||
}
|
||||
switch fieldNum {
|
||||
case 1:
|
||||
if wireType != 0 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field Success", wireType)
|
||||
}
|
||||
var v int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflow
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
v |= int(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
m.Success = bool(v != 0)
|
||||
case 2:
|
||||
if wireType != 2 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field LastError", wireType)
|
||||
}
|
||||
var stringLen uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return 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 ErrInvalidLength
|
||||
}
|
||||
postIndex := iNdEx + intStringLen
|
||||
if postIndex < 0 {
|
||||
return ErrInvalidLength
|
||||
}
|
||||
if postIndex > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
m.LastError = string(dAtA[iNdEx:postIndex])
|
||||
iNdEx = postIndex
|
||||
default:
|
||||
iNdEx = preIndex
|
||||
skippy, err := skip(dAtA[iNdEx:])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if (skippy < 0) || (iNdEx+skippy) < 0 {
|
||||
return 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 *ResolverSpecSpec) UnmarshalVT(dAtA []byte) error {
|
||||
l := len(dAtA)
|
||||
iNdEx := 0
|
||||
@ -8606,6 +9136,133 @@ func (m *StatusSpec) UnmarshalVT(dAtA []byte) error {
|
||||
}
|
||||
return nil
|
||||
}
|
||||
func (m *TCPProbeSpec) 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 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: TCPProbeSpec: wiretype end group for non-group")
|
||||
}
|
||||
if fieldNum <= 0 {
|
||||
return fmt.Errorf("proto: TCPProbeSpec: illegal tag %d (wire type %d)", fieldNum, wire)
|
||||
}
|
||||
switch fieldNum {
|
||||
case 1:
|
||||
if wireType != 2 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field Endpoint", wireType)
|
||||
}
|
||||
var stringLen uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return 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 ErrInvalidLength
|
||||
}
|
||||
postIndex := iNdEx + intStringLen
|
||||
if postIndex < 0 {
|
||||
return ErrInvalidLength
|
||||
}
|
||||
if postIndex > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
m.Endpoint = string(dAtA[iNdEx:postIndex])
|
||||
iNdEx = postIndex
|
||||
case 2:
|
||||
if wireType != 2 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field Timeout", wireType)
|
||||
}
|
||||
var msglen int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflow
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
msglen |= int(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
if msglen < 0 {
|
||||
return ErrInvalidLength
|
||||
}
|
||||
postIndex := iNdEx + msglen
|
||||
if postIndex < 0 {
|
||||
return ErrInvalidLength
|
||||
}
|
||||
if postIndex > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
if m.Timeout == nil {
|
||||
m.Timeout = &durationpb.Duration{}
|
||||
}
|
||||
if unmarshal, ok := interface{}(m.Timeout).(interface {
|
||||
UnmarshalVT([]byte) error
|
||||
}); ok {
|
||||
if err := unmarshal.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil {
|
||||
return err
|
||||
}
|
||||
} else {
|
||||
if err := proto.Unmarshal(dAtA[iNdEx:postIndex], m.Timeout); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
iNdEx = postIndex
|
||||
default:
|
||||
iNdEx = preIndex
|
||||
skippy, err := skip(dAtA[iNdEx:])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if (skippy < 0) || (iNdEx+skippy) < 0 {
|
||||
return 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 *TimeServerSpecSpec) UnmarshalVT(dAtA []byte) error {
|
||||
l := len(dAtA)
|
||||
iNdEx := 0
|
||||
|
@ -17,7 +17,7 @@ import (
|
||||
)
|
||||
|
||||
//nolint:lll
|
||||
//go:generate deep-copy -type AddressSpecSpec -type AddressStatusSpec -type HardwareAddrSpec -type HostnameSpecSpec -type HostnameStatusSpec -type LinkRefreshSpec -type LinkSpecSpec -type LinkStatusSpec -type NodeAddressSpec -type NodeAddressFilterSpec -type OperatorSpecSpec -type ResolverSpecSpec -type ResolverStatusSpec -type RouteSpecSpec -type RouteStatusSpec -type StatusSpec -type TimeServerSpecSpec -type TimeServerStatusSpec -header-file ../../../../hack/boilerplate.txt -o deep_copy.generated.go .
|
||||
//go:generate deep-copy -type AddressSpecSpec -type AddressStatusSpec -type HardwareAddrSpec -type HostnameSpecSpec -type HostnameStatusSpec -type LinkRefreshSpec -type LinkSpecSpec -type LinkStatusSpec -type NodeAddressSpec -type NodeAddressFilterSpec -type OperatorSpecSpec -type ProbeSpecSpec -type ProbeStatusSpec -type ResolverSpecSpec -type ResolverStatusSpec -type RouteSpecSpec -type RouteStatusSpec -type StatusSpec -type TimeServerSpecSpec -type TimeServerStatusSpec -header-file ../../../../hack/boilerplate.txt -o deep_copy.generated.go .
|
||||
|
||||
// AddressSpecType is type of AddressSpec resource.
|
||||
const AddressSpecType = resource.Type("AddressSpecs.net.talos.dev")
|
||||
|
@ -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 AddressSpecSpec -type AddressStatusSpec -type HardwareAddrSpec -type HostnameSpecSpec -type HostnameStatusSpec -type LinkRefreshSpec -type LinkSpecSpec -type LinkStatusSpec -type NodeAddressSpec -type NodeAddressFilterSpec -type OperatorSpecSpec -type ResolverSpecSpec -type ResolverStatusSpec -type RouteSpecSpec -type RouteStatusSpec -type StatusSpec -type TimeServerSpecSpec -type TimeServerStatusSpec -header-file ../../../../hack/boilerplate.txt -o deep_copy.generated.go ."; DO NOT EDIT.
|
||||
// Code generated by "deep-copy -type AddressSpecSpec -type AddressStatusSpec -type HardwareAddrSpec -type HostnameSpecSpec -type HostnameStatusSpec -type LinkRefreshSpec -type LinkSpecSpec -type LinkStatusSpec -type NodeAddressSpec -type NodeAddressFilterSpec -type OperatorSpecSpec -type ProbeSpecSpec -type ProbeStatusSpec -type ResolverSpecSpec -type ResolverStatusSpec -type RouteSpecSpec -type RouteStatusSpec -type StatusSpec -type TimeServerSpecSpec -type TimeServerStatusSpec -header-file ../../../../hack/boilerplate.txt -o deep_copy.generated.go ."; DO NOT EDIT.
|
||||
|
||||
package network
|
||||
|
||||
@ -124,6 +124,18 @@ func (o OperatorSpecSpec) DeepCopy() OperatorSpecSpec {
|
||||
return cp
|
||||
}
|
||||
|
||||
// DeepCopy generates a deep copy of ProbeSpecSpec.
|
||||
func (o ProbeSpecSpec) DeepCopy() ProbeSpecSpec {
|
||||
var cp ProbeSpecSpec = o
|
||||
return cp
|
||||
}
|
||||
|
||||
// DeepCopy generates a deep copy of ProbeStatusSpec.
|
||||
func (o ProbeStatusSpec) DeepCopy() ProbeStatusSpec {
|
||||
var cp ProbeStatusSpec = o
|
||||
return cp
|
||||
}
|
||||
|
||||
// DeepCopy generates a deep copy of ResolverSpecSpec.
|
||||
func (o ResolverSpecSpec) DeepCopy() ResolverSpecSpec {
|
||||
var cp ResolverSpecSpec = o
|
||||
|
@ -36,6 +36,7 @@ func TestRegisterResource(t *testing.T) {
|
||||
&network.NodeAddress{},
|
||||
&network.NodeAddressFilter{},
|
||||
&network.OperatorSpec{},
|
||||
&network.ProbeSpec{},
|
||||
&network.ResolverStatus{},
|
||||
&network.ResolverSpec{},
|
||||
&network.RouteStatus{},
|
||||
|
93
pkg/machinery/resources/network/probe_spec.go
Normal file
93
pkg/machinery/resources/network/probe_spec.go
Normal file
@ -0,0 +1,93 @@
|
||||
// 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 network
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"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"
|
||||
)
|
||||
|
||||
// ProbeSpecType is type of ProbeSpec resource.
|
||||
const ProbeSpecType = resource.Type("ProbeSpecs.net.talos.dev")
|
||||
|
||||
// ProbeSpec resource holds Probe specification to be run.
|
||||
type ProbeSpec = typed.Resource[ProbeSpecSpec, ProbeSpecExtension]
|
||||
|
||||
// ProbeSpecSpec describes the Probe.
|
||||
//
|
||||
//gotagsrewrite:gen
|
||||
type ProbeSpecSpec struct {
|
||||
// Interval between the probes.
|
||||
Interval time.Duration `yaml:"interval" protobuf:"1"`
|
||||
// FailureThreshold is the number of consecutive failures for the probe to be considered failed after having succeeded.
|
||||
FailureThreshold int `yaml:"failureThreshold" protobuf:"2"`
|
||||
// One of the probe types should be specified, for now it's only TCP.
|
||||
TCP TCPProbeSpec `yaml:"tcp,omitempty" protobuf:"3"`
|
||||
// Configuration layer.
|
||||
ConfigLayer ConfigLayer `yaml:"layer" protobuf:"4"`
|
||||
}
|
||||
|
||||
// ID returns the ID of the resource based on the spec.
|
||||
func (spec *ProbeSpecSpec) ID() (resource.ID, error) {
|
||||
var zeroTCP TCPProbeSpec
|
||||
|
||||
if spec.TCP == zeroTCP {
|
||||
return "", fmt.Errorf("no probe type specified")
|
||||
}
|
||||
|
||||
return fmt.Sprintf("tcp:%s", spec.TCP.Endpoint), nil
|
||||
}
|
||||
|
||||
// Equal returns true if the specs are equal.
|
||||
func (spec ProbeSpecSpec) Equal(other ProbeSpecSpec) bool {
|
||||
return spec == other
|
||||
}
|
||||
|
||||
// TCPProbeSpec describes the TCP Probe.
|
||||
//
|
||||
//gotagsrewrite:gen
|
||||
type TCPProbeSpec struct {
|
||||
// Endpoint to probe: host:port.
|
||||
Endpoint string `yaml:"endpoint" protobuf:"1"`
|
||||
// Timeout for the probe.
|
||||
Timeout time.Duration `yaml:"timeout" protobuf:"2"`
|
||||
}
|
||||
|
||||
// NewProbeSpec initializes a ProbeSpec resource.
|
||||
func NewProbeSpec(namespace resource.Namespace, id resource.ID) *ProbeSpec {
|
||||
return typed.NewResource[ProbeSpecSpec, ProbeSpecExtension](
|
||||
resource.NewMetadata(namespace, ProbeSpecType, id, resource.VersionUndefined),
|
||||
ProbeSpecSpec{},
|
||||
)
|
||||
}
|
||||
|
||||
// ProbeSpecExtension provides auxiliary methods for ProbeSpec.
|
||||
type ProbeSpecExtension struct{}
|
||||
|
||||
// ResourceDefinition implements [typed.Extension] interface.
|
||||
func (ProbeSpecExtension) ResourceDefinition() meta.ResourceDefinitionSpec {
|
||||
return meta.ResourceDefinitionSpec{
|
||||
Type: ProbeSpecType,
|
||||
Aliases: []resource.Type{},
|
||||
DefaultNamespace: NamespaceName,
|
||||
PrintColumns: []meta.PrintColumn{},
|
||||
}
|
||||
}
|
||||
|
||||
func init() {
|
||||
proto.RegisterDefaultTypes()
|
||||
|
||||
err := protobuf.RegisterDynamic[ProbeSpecSpec](ProbeSpecType, &ProbeSpec{})
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
65
pkg/machinery/resources/network/probe_status.go
Normal file
65
pkg/machinery/resources/network/probe_status.go
Normal file
@ -0,0 +1,65 @@
|
||||
// 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 network
|
||||
|
||||
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"
|
||||
)
|
||||
|
||||
// ProbeStatusType is type of ProbeStatus resource.
|
||||
const ProbeStatusType = resource.Type("ProbeStatuses.net.talos.dev")
|
||||
|
||||
// ProbeStatus resource holds Probe result.
|
||||
type ProbeStatus = typed.Resource[ProbeStatusSpec, ProbeStatusExtension]
|
||||
|
||||
// ProbeStatusSpec describes the Probe.
|
||||
//
|
||||
//gotagsrewrite:gen
|
||||
type ProbeStatusSpec struct {
|
||||
// Success of the check.
|
||||
Success bool `yaml:"success" protobuf:"1"`
|
||||
// Last error of the probe.
|
||||
LastError string `yaml:"lastError" protobuf:"2"`
|
||||
}
|
||||
|
||||
// NewProbeStatus initializes a ProbeStatus resource.
|
||||
func NewProbeStatus(namespace resource.Namespace, id resource.ID) *ProbeStatus {
|
||||
return typed.NewResource[ProbeStatusSpec, ProbeStatusExtension](
|
||||
resource.NewMetadata(namespace, ProbeStatusType, id, resource.VersionUndefined),
|
||||
ProbeStatusSpec{},
|
||||
)
|
||||
}
|
||||
|
||||
// ProbeStatusExtension provides auxiliary methods for ProbeStatus.
|
||||
type ProbeStatusExtension struct{}
|
||||
|
||||
// ResourceDefinition implements [typed.Extension] interface.
|
||||
func (ProbeStatusExtension) ResourceDefinition() meta.ResourceDefinitionSpec {
|
||||
return meta.ResourceDefinitionSpec{
|
||||
Type: ProbeStatusType,
|
||||
Aliases: []resource.Type{"probe", "probes"},
|
||||
DefaultNamespace: NamespaceName,
|
||||
PrintColumns: []meta.PrintColumn{
|
||||
{
|
||||
Name: "Success",
|
||||
JSONPath: "{.success}",
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func init() {
|
||||
proto.RegisterDefaultTypes()
|
||||
|
||||
err := protobuf.RegisterDynamic[ProbeStatusSpec](ProbeStatusType, &ProbeStatus{})
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
@ -49,7 +49,24 @@ func (StatusExtension) ResourceDefinition() meta.ResourceDefinitionSpec {
|
||||
Type: StatusType,
|
||||
Aliases: []resource.Type{"netstatus", "netstatuses"},
|
||||
DefaultNamespace: NamespaceName,
|
||||
PrintColumns: []meta.PrintColumn{},
|
||||
PrintColumns: []meta.PrintColumn{
|
||||
{
|
||||
Name: "Address",
|
||||
JSONPath: "{.addressReady}",
|
||||
},
|
||||
{
|
||||
Name: "Connectivity",
|
||||
JSONPath: "{.connectivityReady}",
|
||||
},
|
||||
{
|
||||
Name: "Hostname",
|
||||
JSONPath: "{.hostnameReady}",
|
||||
},
|
||||
{
|
||||
Name: "Etc",
|
||||
JSONPath: "{.etcFilesReady}",
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -148,12 +148,15 @@ description: Talos gRPC API reference.
|
||||
- [NodeAddressFilterSpec](#talos.resource.definitions.network.NodeAddressFilterSpec)
|
||||
- [NodeAddressSpec](#talos.resource.definitions.network.NodeAddressSpec)
|
||||
- [OperatorSpecSpec](#talos.resource.definitions.network.OperatorSpecSpec)
|
||||
- [ProbeSpecSpec](#talos.resource.definitions.network.ProbeSpecSpec)
|
||||
- [ProbeStatusSpec](#talos.resource.definitions.network.ProbeStatusSpec)
|
||||
- [ResolverSpecSpec](#talos.resource.definitions.network.ResolverSpecSpec)
|
||||
- [ResolverStatusSpec](#talos.resource.definitions.network.ResolverStatusSpec)
|
||||
- [RouteSpecSpec](#talos.resource.definitions.network.RouteSpecSpec)
|
||||
- [RouteStatusSpec](#talos.resource.definitions.network.RouteStatusSpec)
|
||||
- [STPSpec](#talos.resource.definitions.network.STPSpec)
|
||||
- [StatusSpec](#talos.resource.definitions.network.StatusSpec)
|
||||
- [TCPProbeSpec](#talos.resource.definitions.network.TCPProbeSpec)
|
||||
- [TimeServerSpecSpec](#talos.resource.definitions.network.TimeServerSpecSpec)
|
||||
- [TimeServerStatusSpec](#talos.resource.definitions.network.TimeServerStatusSpec)
|
||||
- [VIPEquinixMetalSpec](#talos.resource.definitions.network.VIPEquinixMetalSpec)
|
||||
@ -2737,6 +2740,40 @@ OperatorSpecSpec describes DNS resolvers.
|
||||
|
||||
|
||||
|
||||
<a name="talos.resource.definitions.network.ProbeSpecSpec"></a>
|
||||
|
||||
### ProbeSpecSpec
|
||||
ProbeSpecSpec describes the Probe.
|
||||
|
||||
|
||||
| Field | Type | Label | Description |
|
||||
| ----- | ---- | ----- | ----------- |
|
||||
| interval | [google.protobuf.Duration](#google.protobuf.Duration) | | |
|
||||
| failure_threshold | [int64](#int64) | | |
|
||||
| tcp | [TCPProbeSpec](#talos.resource.definitions.network.TCPProbeSpec) | | |
|
||||
| config_layer | [talos.resource.definitions.enums.NetworkConfigLayer](#talos.resource.definitions.enums.NetworkConfigLayer) | | |
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
<a name="talos.resource.definitions.network.ProbeStatusSpec"></a>
|
||||
|
||||
### ProbeStatusSpec
|
||||
ProbeStatusSpec describes the Probe.
|
||||
|
||||
|
||||
| Field | Type | Label | Description |
|
||||
| ----- | ---- | ----- | ----------- |
|
||||
| success | [bool](#bool) | | |
|
||||
| last_error | [string](#string) | | |
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
<a name="talos.resource.definitions.network.ResolverSpecSpec"></a>
|
||||
|
||||
### ResolverSpecSpec
|
||||
@ -2855,6 +2892,22 @@ StatusSpec describes network state.
|
||||
|
||||
|
||||
|
||||
<a name="talos.resource.definitions.network.TCPProbeSpec"></a>
|
||||
|
||||
### TCPProbeSpec
|
||||
TCPProbeSpec describes the TCP Probe.
|
||||
|
||||
|
||||
| Field | Type | Label | Description |
|
||||
| ----- | ---- | ----- | ----------- |
|
||||
| endpoint | [string](#string) | | |
|
||||
| timeout | [google.protobuf.Duration](#google.protobuf.Duration) | | |
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
<a name="talos.resource.definitions.network.TimeServerSpecSpec"></a>
|
||||
|
||||
### TimeServerSpecSpec
|
||||
|
Loading…
x
Reference in New Issue
Block a user