diff --git a/api/resource/definitions/network/network.proto b/api/resource/definitions/network/network.proto
index b9575e5d5..7f152a9a2 100755
--- a/api/resource/definitions/network/network.proto
+++ b/api/resource/definitions/network/network.proto
@@ -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;
diff --git a/internal/app/machined/pkg/controllers/ctest/ctest.go b/internal/app/machined/pkg/controllers/ctest/ctest.go
index 0a2ff0528..884b555b3 100644
--- a/internal/app/machined/pkg/controllers/ctest/ctest.go
+++ b/internal/app/machined/pkg/controllers/ctest/ctest.go
@@ -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
diff --git a/internal/app/machined/pkg/controllers/network/internal/probe/probe.go b/internal/app/machined/pkg/controllers/network/internal/probe/probe.go
new file mode 100644
index 000000000..a76175364
--- /dev/null
+++ b/internal/app/machined/pkg/controllers/network/internal/probe/probe.go
@@ -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()
+}
diff --git a/internal/app/machined/pkg/controllers/network/internal/probe/probe_test.go b/internal/app/machined/pkg/controllers/network/internal/probe/probe_test.go
new file mode 100644
index 000000000..fa6756869
--- /dev/null
+++ b/internal/app/machined/pkg/controllers/network/internal/probe/probe_test.go
@@ -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)
+}
diff --git a/internal/app/machined/pkg/controllers/network/platform_config.go b/internal/app/machined/pkg/controllers/network/platform_config.go
index 2264ecd65..19d112640 100644
--- a/internal/app/machined/pkg/controllers/network/platform_config.go
+++ b/internal/app/machined/pkg/controllers/network/platform_config.go
@@ -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)
}
diff --git a/internal/app/machined/pkg/controllers/network/platform_config_test.go b/internal/app/machined/pkg/controllers/network/platform_config_test.go
index c9914152c..a172d87dc 100644
--- a/internal/app/machined/pkg/controllers/network/platform_config_test.go
+++ b/internal/app/machined/pkg/controllers/network/platform_config_test.go
@@ -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 {
diff --git a/internal/app/machined/pkg/controllers/network/probe.go b/internal/app/machined/pkg/controllers/network/probe.go
new file mode 100644
index 000000000..2b87f5400
--- /dev/null
+++ b/internal/app/machined/pkg/controllers/network/probe.go
@@ -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
+ })
+}
diff --git a/internal/app/machined/pkg/controllers/network/probe_test.go b/internal/app/machined/pkg/controllers/network/probe_test.go
new file mode 100644
index 000000000..1061bb1b0
--- /dev/null
+++ b/internal/app/machined/pkg/controllers/network/probe_test.go
@@ -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{}))
+ },
+ },
+ })
+}
diff --git a/internal/app/machined/pkg/controllers/network/status.go b/internal/app/machined/pkg/controllers/network/status.go
index f8567d966..3ec9b70ab 100644
--- a/internal/app/machined/pkg/controllers/network/status.go
+++ b/internal/app/machined/pkg/controllers/network/status.go
@@ -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 {
diff --git a/internal/app/machined/pkg/controllers/network/status_test.go b/internal/app/machined/pkg/controllers/network/status_test.go
index 5bf2f0324..cdb2a8838 100644
--- a/internal/app/machined/pkg/controllers/network/status_test.go
+++ b/internal/app/machined/pkg/controllers/network/status_test.go
@@ -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{}))
+ },
+ },
+ })
}
diff --git a/internal/app/machined/pkg/runtime/platform.go b/internal/app/machined/pkg/runtime/platform.go
index 4665651f3..853708fa1 100644
--- a/internal/app/machined/pkg/runtime/platform.go
+++ b/internal/app/machined/pkg/runtime/platform.go
@@ -57,5 +57,7 @@ type PlatformNetworkConfig struct {
ExternalIPs []netip.Addr `yaml:"externalIPs"`
+ Probes []network.ProbeSpecSpec `yaml:"probes,omitempty"`
+
Metadata *runtime.PlatformMetadataSpec `yaml:"metadata,omitempty"`
}
diff --git a/internal/app/machined/pkg/runtime/v1alpha2/v1alpha2_controller.go b/internal/app/machined/pkg/runtime/v1alpha2/v1alpha2_controller.go
index 60383264f..45bff211e 100644
--- a/internal/app/machined/pkg/runtime/v1alpha2/v1alpha2_controller.go
+++ b/internal/app/machined/pkg/runtime/v1alpha2/v1alpha2_controller.go
@@ -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(),
},
diff --git a/internal/app/machined/pkg/runtime/v1alpha2/v1alpha2_state.go b/internal/app/machined/pkg/runtime/v1alpha2/v1alpha2_state.go
index b7db70185..184304577 100644
--- a/internal/app/machined/pkg/runtime/v1alpha2/v1alpha2_state.go
+++ b/internal/app/machined/pkg/runtime/v1alpha2/v1alpha2_state.go
@@ -151,6 +151,8 @@ func NewState() (*State, error) {
&network.NodeAddress{},
&network.NodeAddressFilter{},
&network.OperatorSpec{},
+ &network.ProbeSpec{},
+ &network.ProbeStatus{},
&network.ResolverStatus{},
&network.ResolverSpec{},
&network.RouteStatus{},
diff --git a/pkg/machinery/api/resource/definitions/network/network.pb.go b/pkg/machinery/api/resource/definitions/network/network.pb.go
index 43f5e3d90..d5fffe518 100644
--- a/pkg/machinery/api/resource/definitions/network/network.pb.go
+++ b/pkg/machinery/api/resource/definitions/network/network.pb.go
@@ -1601,6 +1601,134 @@ func (x *OperatorSpecSpec) GetConfigLayer() enums.NetworkConfigLayer {
return enums.NetworkConfigLayer(0)
}
+// ProbeSpecSpec describes the Probe.
+type ProbeSpecSpec struct {
+ state protoimpl.MessageState
+ sizeCache protoimpl.SizeCache
+ unknownFields protoimpl.UnknownFields
+
+ Interval *durationpb.Duration `protobuf:"bytes,1,opt,name=interval,proto3" json:"interval,omitempty"`
+ FailureThreshold int64 `protobuf:"varint,2,opt,name=failure_threshold,json=failureThreshold,proto3" json:"failure_threshold,omitempty"`
+ Tcp *TCPProbeSpec `protobuf:"bytes,3,opt,name=tcp,proto3" json:"tcp,omitempty"`
+ ConfigLayer enums.NetworkConfigLayer `protobuf:"varint,4,opt,name=config_layer,json=configLayer,proto3,enum=talos.resource.definitions.enums.NetworkConfigLayer" json:"config_layer,omitempty"`
+}
+
+func (x *ProbeSpecSpec) Reset() {
+ *x = ProbeSpecSpec{}
+ if protoimpl.UnsafeEnabled {
+ mi := &file_resource_definitions_network_network_proto_msgTypes[17]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+ }
+}
+
+func (x *ProbeSpecSpec) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ProbeSpecSpec) ProtoMessage() {}
+
+func (x *ProbeSpecSpec) ProtoReflect() protoreflect.Message {
+ mi := &file_resource_definitions_network_network_proto_msgTypes[17]
+ 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 ProbeSpecSpec.ProtoReflect.Descriptor instead.
+func (*ProbeSpecSpec) Descriptor() ([]byte, []int) {
+ return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{17}
+}
+
+func (x *ProbeSpecSpec) GetInterval() *durationpb.Duration {
+ if x != nil {
+ return x.Interval
+ }
+ return nil
+}
+
+func (x *ProbeSpecSpec) GetFailureThreshold() int64 {
+ if x != nil {
+ return x.FailureThreshold
+ }
+ return 0
+}
+
+func (x *ProbeSpecSpec) GetTcp() *TCPProbeSpec {
+ if x != nil {
+ return x.Tcp
+ }
+ return nil
+}
+
+func (x *ProbeSpecSpec) GetConfigLayer() enums.NetworkConfigLayer {
+ if x != nil {
+ return x.ConfigLayer
+ }
+ return enums.NetworkConfigLayer(0)
+}
+
+// ProbeStatusSpec describes the Probe.
+type ProbeStatusSpec struct {
+ state protoimpl.MessageState
+ sizeCache protoimpl.SizeCache
+ unknownFields protoimpl.UnknownFields
+
+ Success bool `protobuf:"varint,1,opt,name=success,proto3" json:"success,omitempty"`
+ LastError string `protobuf:"bytes,2,opt,name=last_error,json=lastError,proto3" json:"last_error,omitempty"`
+}
+
+func (x *ProbeStatusSpec) Reset() {
+ *x = ProbeStatusSpec{}
+ if protoimpl.UnsafeEnabled {
+ mi := &file_resource_definitions_network_network_proto_msgTypes[18]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+ }
+}
+
+func (x *ProbeStatusSpec) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ProbeStatusSpec) ProtoMessage() {}
+
+func (x *ProbeStatusSpec) ProtoReflect() protoreflect.Message {
+ mi := &file_resource_definitions_network_network_proto_msgTypes[18]
+ 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 ProbeStatusSpec.ProtoReflect.Descriptor instead.
+func (*ProbeStatusSpec) Descriptor() ([]byte, []int) {
+ return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{18}
+}
+
+func (x *ProbeStatusSpec) GetSuccess() bool {
+ if x != nil {
+ return x.Success
+ }
+ return false
+}
+
+func (x *ProbeStatusSpec) GetLastError() string {
+ if x != nil {
+ return x.LastError
+ }
+ return ""
+}
+
// ResolverSpecSpec describes DNS resolvers.
type ResolverSpecSpec struct {
state protoimpl.MessageState
@@ -1614,7 +1742,7 @@ type ResolverSpecSpec struct {
func (x *ResolverSpecSpec) Reset() {
*x = ResolverSpecSpec{}
if protoimpl.UnsafeEnabled {
- mi := &file_resource_definitions_network_network_proto_msgTypes[17]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[19]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1627,7 +1755,7 @@ func (x *ResolverSpecSpec) String() string {
func (*ResolverSpecSpec) ProtoMessage() {}
func (x *ResolverSpecSpec) ProtoReflect() protoreflect.Message {
- mi := &file_resource_definitions_network_network_proto_msgTypes[17]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[19]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1640,7 +1768,7 @@ func (x *ResolverSpecSpec) ProtoReflect() protoreflect.Message {
// Deprecated: Use ResolverSpecSpec.ProtoReflect.Descriptor instead.
func (*ResolverSpecSpec) Descriptor() ([]byte, []int) {
- return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{17}
+ return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{19}
}
func (x *ResolverSpecSpec) GetDnsServers() []*common.NetIP {
@@ -1669,7 +1797,7 @@ type ResolverStatusSpec struct {
func (x *ResolverStatusSpec) Reset() {
*x = ResolverStatusSpec{}
if protoimpl.UnsafeEnabled {
- mi := &file_resource_definitions_network_network_proto_msgTypes[18]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[20]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1682,7 +1810,7 @@ func (x *ResolverStatusSpec) String() string {
func (*ResolverStatusSpec) ProtoMessage() {}
func (x *ResolverStatusSpec) ProtoReflect() protoreflect.Message {
- mi := &file_resource_definitions_network_network_proto_msgTypes[18]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[20]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1695,7 +1823,7 @@ func (x *ResolverStatusSpec) ProtoReflect() protoreflect.Message {
// Deprecated: Use ResolverStatusSpec.ProtoReflect.Descriptor instead.
func (*ResolverStatusSpec) Descriptor() ([]byte, []int) {
- return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{18}
+ return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{20}
}
func (x *ResolverStatusSpec) GetDnsServers() []*common.NetIP {
@@ -1729,7 +1857,7 @@ type RouteSpecSpec struct {
func (x *RouteSpecSpec) Reset() {
*x = RouteSpecSpec{}
if protoimpl.UnsafeEnabled {
- mi := &file_resource_definitions_network_network_proto_msgTypes[19]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[21]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1742,7 +1870,7 @@ func (x *RouteSpecSpec) String() string {
func (*RouteSpecSpec) ProtoMessage() {}
func (x *RouteSpecSpec) ProtoReflect() protoreflect.Message {
- mi := &file_resource_definitions_network_network_proto_msgTypes[19]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[21]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1755,7 +1883,7 @@ func (x *RouteSpecSpec) ProtoReflect() protoreflect.Message {
// Deprecated: Use RouteSpecSpec.ProtoReflect.Descriptor instead.
func (*RouteSpecSpec) Descriptor() ([]byte, []int) {
- return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{19}
+ return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{21}
}
func (x *RouteSpecSpec) GetFamily() enums.NethelpersFamily {
@@ -1873,7 +2001,7 @@ type RouteStatusSpec struct {
func (x *RouteStatusSpec) Reset() {
*x = RouteStatusSpec{}
if protoimpl.UnsafeEnabled {
- mi := &file_resource_definitions_network_network_proto_msgTypes[20]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[22]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1886,7 +2014,7 @@ func (x *RouteStatusSpec) String() string {
func (*RouteStatusSpec) ProtoMessage() {}
func (x *RouteStatusSpec) ProtoReflect() protoreflect.Message {
- mi := &file_resource_definitions_network_network_proto_msgTypes[20]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[22]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1899,7 +2027,7 @@ func (x *RouteStatusSpec) ProtoReflect() protoreflect.Message {
// Deprecated: Use RouteStatusSpec.ProtoReflect.Descriptor instead.
func (*RouteStatusSpec) Descriptor() ([]byte, []int) {
- return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{20}
+ return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{22}
}
func (x *RouteStatusSpec) GetFamily() enums.NethelpersFamily {
@@ -2005,7 +2133,7 @@ type STPSpec struct {
func (x *STPSpec) Reset() {
*x = STPSpec{}
if protoimpl.UnsafeEnabled {
- mi := &file_resource_definitions_network_network_proto_msgTypes[21]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[23]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2018,7 +2146,7 @@ func (x *STPSpec) String() string {
func (*STPSpec) ProtoMessage() {}
func (x *STPSpec) ProtoReflect() protoreflect.Message {
- mi := &file_resource_definitions_network_network_proto_msgTypes[21]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[23]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2031,7 +2159,7 @@ func (x *STPSpec) ProtoReflect() protoreflect.Message {
// Deprecated: Use STPSpec.ProtoReflect.Descriptor instead.
func (*STPSpec) Descriptor() ([]byte, []int) {
- return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{21}
+ return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{23}
}
func (x *STPSpec) GetEnabled() bool {
@@ -2056,7 +2184,7 @@ type StatusSpec struct {
func (x *StatusSpec) Reset() {
*x = StatusSpec{}
if protoimpl.UnsafeEnabled {
- mi := &file_resource_definitions_network_network_proto_msgTypes[22]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[24]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2069,7 +2197,7 @@ func (x *StatusSpec) String() string {
func (*StatusSpec) ProtoMessage() {}
func (x *StatusSpec) ProtoReflect() protoreflect.Message {
- mi := &file_resource_definitions_network_network_proto_msgTypes[22]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[24]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2082,7 +2210,7 @@ func (x *StatusSpec) ProtoReflect() protoreflect.Message {
// Deprecated: Use StatusSpec.ProtoReflect.Descriptor instead.
func (*StatusSpec) Descriptor() ([]byte, []int) {
- return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{22}
+ return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{24}
}
func (x *StatusSpec) GetAddressReady() bool {
@@ -2113,6 +2241,62 @@ func (x *StatusSpec) GetEtcFilesReady() bool {
return false
}
+// TCPProbeSpec describes the TCP Probe.
+type TCPProbeSpec struct {
+ state protoimpl.MessageState
+ sizeCache protoimpl.SizeCache
+ unknownFields protoimpl.UnknownFields
+
+ Endpoint string `protobuf:"bytes,1,opt,name=endpoint,proto3" json:"endpoint,omitempty"`
+ Timeout *durationpb.Duration `protobuf:"bytes,2,opt,name=timeout,proto3" json:"timeout,omitempty"`
+}
+
+func (x *TCPProbeSpec) Reset() {
+ *x = TCPProbeSpec{}
+ if protoimpl.UnsafeEnabled {
+ mi := &file_resource_definitions_network_network_proto_msgTypes[25]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+ }
+}
+
+func (x *TCPProbeSpec) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*TCPProbeSpec) ProtoMessage() {}
+
+func (x *TCPProbeSpec) ProtoReflect() protoreflect.Message {
+ mi := &file_resource_definitions_network_network_proto_msgTypes[25]
+ 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 TCPProbeSpec.ProtoReflect.Descriptor instead.
+func (*TCPProbeSpec) Descriptor() ([]byte, []int) {
+ return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{25}
+}
+
+func (x *TCPProbeSpec) GetEndpoint() string {
+ if x != nil {
+ return x.Endpoint
+ }
+ return ""
+}
+
+func (x *TCPProbeSpec) GetTimeout() *durationpb.Duration {
+ if x != nil {
+ return x.Timeout
+ }
+ return nil
+}
+
// TimeServerSpecSpec describes NTP servers.
type TimeServerSpecSpec struct {
state protoimpl.MessageState
@@ -2126,7 +2310,7 @@ type TimeServerSpecSpec struct {
func (x *TimeServerSpecSpec) Reset() {
*x = TimeServerSpecSpec{}
if protoimpl.UnsafeEnabled {
- mi := &file_resource_definitions_network_network_proto_msgTypes[23]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[26]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2139,7 +2323,7 @@ func (x *TimeServerSpecSpec) String() string {
func (*TimeServerSpecSpec) ProtoMessage() {}
func (x *TimeServerSpecSpec) ProtoReflect() protoreflect.Message {
- mi := &file_resource_definitions_network_network_proto_msgTypes[23]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[26]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2152,7 +2336,7 @@ func (x *TimeServerSpecSpec) ProtoReflect() protoreflect.Message {
// Deprecated: Use TimeServerSpecSpec.ProtoReflect.Descriptor instead.
func (*TimeServerSpecSpec) Descriptor() ([]byte, []int) {
- return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{23}
+ return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{26}
}
func (x *TimeServerSpecSpec) GetNtpServers() []string {
@@ -2181,7 +2365,7 @@ type TimeServerStatusSpec struct {
func (x *TimeServerStatusSpec) Reset() {
*x = TimeServerStatusSpec{}
if protoimpl.UnsafeEnabled {
- mi := &file_resource_definitions_network_network_proto_msgTypes[24]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[27]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2194,7 +2378,7 @@ func (x *TimeServerStatusSpec) String() string {
func (*TimeServerStatusSpec) ProtoMessage() {}
func (x *TimeServerStatusSpec) ProtoReflect() protoreflect.Message {
- mi := &file_resource_definitions_network_network_proto_msgTypes[24]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[27]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2207,7 +2391,7 @@ func (x *TimeServerStatusSpec) ProtoReflect() protoreflect.Message {
// Deprecated: Use TimeServerStatusSpec.ProtoReflect.Descriptor instead.
func (*TimeServerStatusSpec) Descriptor() ([]byte, []int) {
- return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{24}
+ return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{27}
}
func (x *TimeServerStatusSpec) GetNtpServers() []string {
@@ -2231,7 +2415,7 @@ type VIPEquinixMetalSpec struct {
func (x *VIPEquinixMetalSpec) Reset() {
*x = VIPEquinixMetalSpec{}
if protoimpl.UnsafeEnabled {
- mi := &file_resource_definitions_network_network_proto_msgTypes[25]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[28]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2244,7 +2428,7 @@ func (x *VIPEquinixMetalSpec) String() string {
func (*VIPEquinixMetalSpec) ProtoMessage() {}
func (x *VIPEquinixMetalSpec) ProtoReflect() protoreflect.Message {
- mi := &file_resource_definitions_network_network_proto_msgTypes[25]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[28]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2257,7 +2441,7 @@ func (x *VIPEquinixMetalSpec) ProtoReflect() protoreflect.Message {
// Deprecated: Use VIPEquinixMetalSpec.ProtoReflect.Descriptor instead.
func (*VIPEquinixMetalSpec) Descriptor() ([]byte, []int) {
- return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{25}
+ return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{28}
}
func (x *VIPEquinixMetalSpec) GetProjectId() string {
@@ -2295,7 +2479,7 @@ type VIPHCloudSpec struct {
func (x *VIPHCloudSpec) Reset() {
*x = VIPHCloudSpec{}
if protoimpl.UnsafeEnabled {
- mi := &file_resource_definitions_network_network_proto_msgTypes[26]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[29]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2308,7 +2492,7 @@ func (x *VIPHCloudSpec) String() string {
func (*VIPHCloudSpec) ProtoMessage() {}
func (x *VIPHCloudSpec) ProtoReflect() protoreflect.Message {
- mi := &file_resource_definitions_network_network_proto_msgTypes[26]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[29]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2321,7 +2505,7 @@ func (x *VIPHCloudSpec) ProtoReflect() protoreflect.Message {
// Deprecated: Use VIPHCloudSpec.ProtoReflect.Descriptor instead.
func (*VIPHCloudSpec) Descriptor() ([]byte, []int) {
- return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{26}
+ return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{29}
}
func (x *VIPHCloudSpec) GetDeviceId() int64 {
@@ -2360,7 +2544,7 @@ type VIPOperatorSpec struct {
func (x *VIPOperatorSpec) Reset() {
*x = VIPOperatorSpec{}
if protoimpl.UnsafeEnabled {
- mi := &file_resource_definitions_network_network_proto_msgTypes[27]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[30]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2373,7 +2557,7 @@ func (x *VIPOperatorSpec) String() string {
func (*VIPOperatorSpec) ProtoMessage() {}
func (x *VIPOperatorSpec) ProtoReflect() protoreflect.Message {
- mi := &file_resource_definitions_network_network_proto_msgTypes[27]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[30]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2386,7 +2570,7 @@ func (x *VIPOperatorSpec) ProtoReflect() protoreflect.Message {
// Deprecated: Use VIPOperatorSpec.ProtoReflect.Descriptor instead.
func (*VIPOperatorSpec) Descriptor() ([]byte, []int) {
- return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{27}
+ return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{30}
}
func (x *VIPOperatorSpec) GetIp() *common.NetIP {
@@ -2430,7 +2614,7 @@ type VLANSpec struct {
func (x *VLANSpec) Reset() {
*x = VLANSpec{}
if protoimpl.UnsafeEnabled {
- mi := &file_resource_definitions_network_network_proto_msgTypes[28]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[31]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2443,7 +2627,7 @@ func (x *VLANSpec) String() string {
func (*VLANSpec) ProtoMessage() {}
func (x *VLANSpec) ProtoReflect() protoreflect.Message {
- mi := &file_resource_definitions_network_network_proto_msgTypes[28]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[31]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2456,7 +2640,7 @@ func (x *VLANSpec) ProtoReflect() protoreflect.Message {
// Deprecated: Use VLANSpec.ProtoReflect.Descriptor instead.
func (*VLANSpec) Descriptor() ([]byte, []int) {
- return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{28}
+ return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{31}
}
func (x *VLANSpec) GetVid() uint32 {
@@ -2489,7 +2673,7 @@ type WireguardPeer struct {
func (x *WireguardPeer) Reset() {
*x = WireguardPeer{}
if protoimpl.UnsafeEnabled {
- mi := &file_resource_definitions_network_network_proto_msgTypes[29]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[32]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2502,7 +2686,7 @@ func (x *WireguardPeer) String() string {
func (*WireguardPeer) ProtoMessage() {}
func (x *WireguardPeer) ProtoReflect() protoreflect.Message {
- mi := &file_resource_definitions_network_network_proto_msgTypes[29]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[32]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2515,7 +2699,7 @@ func (x *WireguardPeer) ProtoReflect() protoreflect.Message {
// Deprecated: Use WireguardPeer.ProtoReflect.Descriptor instead.
func (*WireguardPeer) Descriptor() ([]byte, []int) {
- return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{29}
+ return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{32}
}
func (x *WireguardPeer) GetPublicKey() string {
@@ -2569,7 +2753,7 @@ type WireguardSpec struct {
func (x *WireguardSpec) Reset() {
*x = WireguardSpec{}
if protoimpl.UnsafeEnabled {
- mi := &file_resource_definitions_network_network_proto_msgTypes[30]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[33]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2582,7 +2766,7 @@ func (x *WireguardSpec) String() string {
func (*WireguardSpec) ProtoMessage() {}
func (x *WireguardSpec) ProtoReflect() protoreflect.Message {
- mi := &file_resource_definitions_network_network_proto_msgTypes[30]
+ mi := &file_resource_definitions_network_network_proto_msgTypes[33]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2595,7 +2779,7 @@ func (x *WireguardSpec) ProtoReflect() protoreflect.Message {
// Deprecated: Use WireguardSpec.ProtoReflect.Descriptor instead.
func (*WireguardSpec) Descriptor() ([]byte, []int) {
- return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{30}
+ return file_resource_definitions_network_network_proto_rawDescGZIP(), []int{33}
}
func (x *WireguardSpec) GetPrivateKey() string {
@@ -3011,124 +3195,152 @@ var file_resource_definitions_network_network_proto_rawDesc = []byte{
0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e,
0x73, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x43,
0x6f, 0x6e, 0x66, 0x69, 0x67, 0x4c, 0x61, 0x79, 0x65, 0x72, 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x66,
- 0x69, 0x67, 0x4c, 0x61, 0x79, 0x65, 0x72, 0x22, 0x9b, 0x01, 0x0a, 0x10, 0x52, 0x65, 0x73, 0x6f,
- 0x6c, 0x76, 0x65, 0x72, 0x53, 0x70, 0x65, 0x63, 0x53, 0x70, 0x65, 0x63, 0x12, 0x2e, 0x0a, 0x0b,
- 0x64, 0x6e, 0x73, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28,
- 0x0b, 0x32, 0x0d, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4e, 0x65, 0x74, 0x49, 0x50,
- 0x52, 0x0a, 0x64, 0x6e, 0x73, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x73, 0x12, 0x57, 0x0a, 0x0c,
- 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x5f, 0x6c, 0x61, 0x79, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01,
- 0x28, 0x0e, 0x32, 0x34, 0x2e, 0x74, 0x61, 0x6c, 0x6f, 0x73, 0x2e, 0x72, 0x65, 0x73, 0x6f, 0x75,
- 0x72, 0x63, 0x65, 0x2e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e,
- 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x43, 0x6f, 0x6e,
- 0x66, 0x69, 0x67, 0x4c, 0x61, 0x79, 0x65, 0x72, 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67,
- 0x4c, 0x61, 0x79, 0x65, 0x72, 0x22, 0x44, 0x0a, 0x12, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65,
- 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, 0x70, 0x65, 0x63, 0x12, 0x2e, 0x0a, 0x0b, 0x64,
- 0x6e, 0x73, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b,
- 0x32, 0x0d, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4e, 0x65, 0x74, 0x49, 0x50, 0x52,
- 0x0a, 0x64, 0x6e, 0x73, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x73, 0x22, 0xde, 0x05, 0x0a, 0x0d,
- 0x52, 0x6f, 0x75, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x53, 0x70, 0x65, 0x63, 0x12, 0x4a, 0x0a,
- 0x06, 0x66, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x32, 0x2e,
- 0x74, 0x61, 0x6c, 0x6f, 0x73, 0x2e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x64,
- 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73,
- 0x2e, 0x4e, 0x65, 0x74, 0x68, 0x65, 0x6c, 0x70, 0x65, 0x72, 0x73, 0x46, 0x61, 0x6d, 0x69, 0x6c,
- 0x79, 0x52, 0x06, 0x66, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x12, 0x35, 0x0a, 0x0b, 0x64, 0x65, 0x73,
- 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13,
- 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4e, 0x65, 0x74, 0x49, 0x50, 0x50, 0x72, 0x65,
- 0x66, 0x69, 0x78, 0x52, 0x0b, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e,
- 0x12, 0x25, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b,
- 0x32, 0x0d, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4e, 0x65, 0x74, 0x49, 0x50, 0x52,
- 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x27, 0x0a, 0x07, 0x67, 0x61, 0x74, 0x65, 0x77,
- 0x61, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f,
- 0x6e, 0x2e, 0x4e, 0x65, 0x74, 0x49, 0x50, 0x52, 0x07, 0x67, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79,
- 0x12, 0x22, 0x0a, 0x0d, 0x6f, 0x75, 0x74, 0x5f, 0x6c, 0x69, 0x6e, 0x6b, 0x5f, 0x6e, 0x61, 0x6d,
- 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6f, 0x75, 0x74, 0x4c, 0x69, 0x6e, 0x6b,
- 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x4e, 0x0a, 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x06, 0x20,
- 0x01, 0x28, 0x0e, 0x32, 0x38, 0x2e, 0x74, 0x61, 0x6c, 0x6f, 0x73, 0x2e, 0x72, 0x65, 0x73, 0x6f,
- 0x75, 0x72, 0x63, 0x65, 0x2e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73,
- 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x4e, 0x65, 0x74, 0x68, 0x65, 0x6c, 0x70, 0x65, 0x72,
- 0x73, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x05, 0x74,
- 0x61, 0x62, 0x6c, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79,
- 0x18, 0x07, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79,
- 0x12, 0x47, 0x0a, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32,
- 0x31, 0x2e, 0x74, 0x61, 0x6c, 0x6f, 0x73, 0x2e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,
- 0x2e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x65, 0x6e, 0x75,
- 0x6d, 0x73, 0x2e, 0x4e, 0x65, 0x74, 0x68, 0x65, 0x6c, 0x70, 0x65, 0x72, 0x73, 0x53, 0x63, 0x6f,
- 0x70, 0x65, 0x52, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x49, 0x0a, 0x04, 0x74, 0x79, 0x70,
- 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x35, 0x2e, 0x74, 0x61, 0x6c, 0x6f, 0x73, 0x2e,
- 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74,
- 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x4e, 0x65, 0x74, 0x68, 0x65,
- 0x6c, 0x70, 0x65, 0x72, 0x73, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04,
- 0x74, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6c, 0x61, 0x67, 0x73, 0x18, 0x0a, 0x20,
- 0x01, 0x28, 0x0d, 0x52, 0x05, 0x66, 0x6c, 0x61, 0x67, 0x73, 0x12, 0x55, 0x0a, 0x08, 0x70, 0x72,
- 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x39, 0x2e, 0x74,
- 0x61, 0x6c, 0x6f, 0x73, 0x2e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x64, 0x65,
- 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e,
- 0x4e, 0x65, 0x74, 0x68, 0x65, 0x6c, 0x70, 0x65, 0x72, 0x73, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x50,
- 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f,
- 0x6c, 0x12, 0x57, 0x0a, 0x0c, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x5f, 0x6c, 0x61, 0x79, 0x65,
- 0x72, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x34, 0x2e, 0x74, 0x61, 0x6c, 0x6f, 0x73, 0x2e,
+ 0x69, 0x67, 0x4c, 0x61, 0x79, 0x65, 0x72, 0x22, 0x90, 0x02, 0x0a, 0x0d, 0x50, 0x72, 0x6f, 0x62,
+ 0x65, 0x53, 0x70, 0x65, 0x63, 0x53, 0x70, 0x65, 0x63, 0x12, 0x35, 0x0a, 0x08, 0x69, 0x6e, 0x74,
+ 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f,
+ 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75,
+ 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c,
+ 0x12, 0x2b, 0x0a, 0x11, 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x5f, 0x74, 0x68, 0x72, 0x65,
+ 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x66, 0x61, 0x69,
+ 0x6c, 0x75, 0x72, 0x65, 0x54, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x12, 0x42, 0x0a,
+ 0x03, 0x74, 0x63, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x74, 0x61, 0x6c,
+ 0x6f, 0x73, 0x2e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x64, 0x65, 0x66, 0x69,
+ 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x2e,
+ 0x54, 0x43, 0x50, 0x50, 0x72, 0x6f, 0x62, 0x65, 0x53, 0x70, 0x65, 0x63, 0x52, 0x03, 0x74, 0x63,
+ 0x70, 0x12, 0x57, 0x0a, 0x0c, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x5f, 0x6c, 0x61, 0x79, 0x65,
+ 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x34, 0x2e, 0x74, 0x61, 0x6c, 0x6f, 0x73, 0x2e,
0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74,
0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x4e, 0x65, 0x74, 0x77, 0x6f,
0x72, 0x6b, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x4c, 0x61, 0x79, 0x65, 0x72, 0x52, 0x0b, 0x63,
- 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x4c, 0x61, 0x79, 0x65, 0x72, 0x12, 0x10, 0x0a, 0x03, 0x6d, 0x74,
- 0x75, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, 0x6d, 0x74, 0x75, 0x22, 0xad, 0x05, 0x0a,
- 0x0f, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, 0x70, 0x65, 0x63,
- 0x12, 0x4a, 0x0a, 0x06, 0x66, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e,
- 0x32, 0x32, 0x2e, 0x74, 0x61, 0x6c, 0x6f, 0x73, 0x2e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63,
- 0x65, 0x2e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x65, 0x6e,
- 0x75, 0x6d, 0x73, 0x2e, 0x4e, 0x65, 0x74, 0x68, 0x65, 0x6c, 0x70, 0x65, 0x72, 0x73, 0x46, 0x61,
- 0x6d, 0x69, 0x6c, 0x79, 0x52, 0x06, 0x66, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x12, 0x35, 0x0a, 0x0b,
- 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28,
- 0x0b, 0x32, 0x13, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4e, 0x65, 0x74, 0x49, 0x50,
- 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, 0x52, 0x0b, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74,
- 0x69, 0x6f, 0x6e, 0x12, 0x25, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x03, 0x20,
- 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4e, 0x65, 0x74,
- 0x49, 0x50, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x27, 0x0a, 0x07, 0x67, 0x61,
- 0x74, 0x65, 0x77, 0x61, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x63, 0x6f,
- 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4e, 0x65, 0x74, 0x49, 0x50, 0x52, 0x07, 0x67, 0x61, 0x74, 0x65,
- 0x77, 0x61, 0x79, 0x12, 0x24, 0x0a, 0x0e, 0x6f, 0x75, 0x74, 0x5f, 0x6c, 0x69, 0x6e, 0x6b, 0x5f,
- 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x6f, 0x75, 0x74,
- 0x4c, 0x69, 0x6e, 0x6b, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x22, 0x0a, 0x0d, 0x6f, 0x75, 0x74,
- 0x5f, 0x6c, 0x69, 0x6e, 0x6b, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09,
- 0x52, 0x0b, 0x6f, 0x75, 0x74, 0x4c, 0x69, 0x6e, 0x6b, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x4e, 0x0a,
- 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x38, 0x2e, 0x74,
+ 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x4c, 0x61, 0x79, 0x65, 0x72, 0x22, 0x4a, 0x0a, 0x0f, 0x50, 0x72,
+ 0x6f, 0x62, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, 0x70, 0x65, 0x63, 0x12, 0x18, 0x0a,
+ 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07,
+ 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x6c, 0x61, 0x73, 0x74, 0x5f,
+ 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6c, 0x61, 0x73,
+ 0x74, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x9b, 0x01, 0x0a, 0x10, 0x52, 0x65, 0x73, 0x6f, 0x6c,
+ 0x76, 0x65, 0x72, 0x53, 0x70, 0x65, 0x63, 0x53, 0x70, 0x65, 0x63, 0x12, 0x2e, 0x0a, 0x0b, 0x64,
+ 0x6e, 0x73, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b,
+ 0x32, 0x0d, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4e, 0x65, 0x74, 0x49, 0x50, 0x52,
+ 0x0a, 0x64, 0x6e, 0x73, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x73, 0x12, 0x57, 0x0a, 0x0c, 0x63,
+ 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x5f, 0x6c, 0x61, 0x79, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28,
+ 0x0e, 0x32, 0x34, 0x2e, 0x74, 0x61, 0x6c, 0x6f, 0x73, 0x2e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72,
+ 0x63, 0x65, 0x2e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x65,
+ 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x43, 0x6f, 0x6e, 0x66,
+ 0x69, 0x67, 0x4c, 0x61, 0x79, 0x65, 0x72, 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x4c,
+ 0x61, 0x79, 0x65, 0x72, 0x22, 0x44, 0x0a, 0x12, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x72,
+ 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, 0x70, 0x65, 0x63, 0x12, 0x2e, 0x0a, 0x0b, 0x64, 0x6e,
+ 0x73, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32,
+ 0x0d, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4e, 0x65, 0x74, 0x49, 0x50, 0x52, 0x0a,
+ 0x64, 0x6e, 0x73, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x73, 0x22, 0xde, 0x05, 0x0a, 0x0d, 0x52,
+ 0x6f, 0x75, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x53, 0x70, 0x65, 0x63, 0x12, 0x4a, 0x0a, 0x06,
+ 0x66, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x32, 0x2e, 0x74,
0x61, 0x6c, 0x6f, 0x73, 0x2e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x64, 0x65,
0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e,
- 0x4e, 0x65, 0x74, 0x68, 0x65, 0x6c, 0x70, 0x65, 0x72, 0x73, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e,
- 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x1a, 0x0a,
- 0x08, 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0d, 0x52,
- 0x08, 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x12, 0x47, 0x0a, 0x05, 0x73, 0x63, 0x6f,
- 0x70, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x31, 0x2e, 0x74, 0x61, 0x6c, 0x6f, 0x73,
- 0x2e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69,
- 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x4e, 0x65, 0x74, 0x68,
- 0x65, 0x6c, 0x70, 0x65, 0x72, 0x73, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x52, 0x05, 0x73, 0x63, 0x6f,
- 0x70, 0x65, 0x12, 0x49, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0e,
- 0x32, 0x35, 0x2e, 0x74, 0x61, 0x6c, 0x6f, 0x73, 0x2e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63,
- 0x65, 0x2e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x65, 0x6e,
- 0x75, 0x6d, 0x73, 0x2e, 0x4e, 0x65, 0x74, 0x68, 0x65, 0x6c, 0x70, 0x65, 0x72, 0x73, 0x52, 0x6f,
- 0x75, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a,
- 0x05, 0x66, 0x6c, 0x61, 0x67, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x66, 0x6c,
- 0x61, 0x67, 0x73, 0x12, 0x55, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x18,
- 0x0c, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x39, 0x2e, 0x74, 0x61, 0x6c, 0x6f, 0x73, 0x2e, 0x72, 0x65,
- 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f,
- 0x6e, 0x73, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x4e, 0x65, 0x74, 0x68, 0x65, 0x6c, 0x70,
- 0x65, 0x72, 0x73, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c,
- 0x52, 0x08, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x12, 0x10, 0x0a, 0x03, 0x6d, 0x74,
- 0x75, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, 0x6d, 0x74, 0x75, 0x22, 0x23, 0x0a, 0x07,
- 0x53, 0x54, 0x50, 0x53, 0x70, 0x65, 0x63, 0x12, 0x18, 0x0a, 0x07, 0x65, 0x6e, 0x61, 0x62, 0x6c,
- 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65,
- 0x64, 0x22, 0xaf, 0x01, 0x0a, 0x0a, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, 0x70, 0x65, 0x63,
- 0x12, 0x23, 0x0a, 0x0d, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x5f, 0x72, 0x65, 0x61, 0x64,
- 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73,
- 0x52, 0x65, 0x61, 0x64, 0x79, 0x12, 0x2d, 0x0a, 0x12, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,
- 0x69, 0x76, 0x69, 0x74, 0x79, 0x5f, 0x72, 0x65, 0x61, 0x64, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28,
- 0x08, 0x52, 0x11, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x52,
- 0x65, 0x61, 0x64, 0x79, 0x12, 0x25, 0x0a, 0x0e, 0x68, 0x6f, 0x73, 0x74, 0x6e, 0x61, 0x6d, 0x65,
- 0x5f, 0x72, 0x65, 0x61, 0x64, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x68, 0x6f,
- 0x73, 0x74, 0x6e, 0x61, 0x6d, 0x65, 0x52, 0x65, 0x61, 0x64, 0x79, 0x12, 0x26, 0x0a, 0x0f, 0x65,
- 0x74, 0x63, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x5f, 0x72, 0x65, 0x61, 0x64, 0x79, 0x18, 0x04,
- 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x65, 0x74, 0x63, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x52, 0x65,
- 0x61, 0x64, 0x79, 0x22, 0x8e, 0x01, 0x0a, 0x12, 0x54, 0x69, 0x6d, 0x65, 0x53, 0x65, 0x72, 0x76,
+ 0x4e, 0x65, 0x74, 0x68, 0x65, 0x6c, 0x70, 0x65, 0x72, 0x73, 0x46, 0x61, 0x6d, 0x69, 0x6c, 0x79,
+ 0x52, 0x06, 0x66, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x12, 0x35, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x74,
+ 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e,
+ 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4e, 0x65, 0x74, 0x49, 0x50, 0x50, 0x72, 0x65, 0x66,
+ 0x69, 0x78, 0x52, 0x0b, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12,
+ 0x25, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32,
+ 0x0d, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4e, 0x65, 0x74, 0x49, 0x50, 0x52, 0x06,
+ 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x27, 0x0a, 0x07, 0x67, 0x61, 0x74, 0x65, 0x77, 0x61,
+ 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e,
+ 0x2e, 0x4e, 0x65, 0x74, 0x49, 0x50, 0x52, 0x07, 0x67, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x12,
+ 0x22, 0x0a, 0x0d, 0x6f, 0x75, 0x74, 0x5f, 0x6c, 0x69, 0x6e, 0x6b, 0x5f, 0x6e, 0x61, 0x6d, 0x65,
+ 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6f, 0x75, 0x74, 0x4c, 0x69, 0x6e, 0x6b, 0x4e,
+ 0x61, 0x6d, 0x65, 0x12, 0x4e, 0x0a, 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x06, 0x20, 0x01,
+ 0x28, 0x0e, 0x32, 0x38, 0x2e, 0x74, 0x61, 0x6c, 0x6f, 0x73, 0x2e, 0x72, 0x65, 0x73, 0x6f, 0x75,
+ 0x72, 0x63, 0x65, 0x2e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e,
+ 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x4e, 0x65, 0x74, 0x68, 0x65, 0x6c, 0x70, 0x65, 0x72, 0x73,
+ 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x05, 0x74, 0x61,
+ 0x62, 0x6c, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x18,
+ 0x07, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x12,
+ 0x47, 0x0a, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x31,
+ 0x2e, 0x74, 0x61, 0x6c, 0x6f, 0x73, 0x2e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e,
+ 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x65, 0x6e, 0x75, 0x6d,
+ 0x73, 0x2e, 0x4e, 0x65, 0x74, 0x68, 0x65, 0x6c, 0x70, 0x65, 0x72, 0x73, 0x53, 0x63, 0x6f, 0x70,
+ 0x65, 0x52, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x49, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65,
+ 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x35, 0x2e, 0x74, 0x61, 0x6c, 0x6f, 0x73, 0x2e, 0x72,
+ 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69,
+ 0x6f, 0x6e, 0x73, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x4e, 0x65, 0x74, 0x68, 0x65, 0x6c,
+ 0x70, 0x65, 0x72, 0x73, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74,
+ 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6c, 0x61, 0x67, 0x73, 0x18, 0x0a, 0x20, 0x01,
+ 0x28, 0x0d, 0x52, 0x05, 0x66, 0x6c, 0x61, 0x67, 0x73, 0x12, 0x55, 0x0a, 0x08, 0x70, 0x72, 0x6f,
+ 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x39, 0x2e, 0x74, 0x61,
+ 0x6c, 0x6f, 0x73, 0x2e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x64, 0x65, 0x66,
+ 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x4e,
+ 0x65, 0x74, 0x68, 0x65, 0x6c, 0x70, 0x65, 0x72, 0x73, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x50, 0x72,
+ 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c,
+ 0x12, 0x57, 0x0a, 0x0c, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x5f, 0x6c, 0x61, 0x79, 0x65, 0x72,
+ 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x34, 0x2e, 0x74, 0x61, 0x6c, 0x6f, 0x73, 0x2e, 0x72,
+ 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69,
+ 0x6f, 0x6e, 0x73, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72,
+ 0x6b, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x4c, 0x61, 0x79, 0x65, 0x72, 0x52, 0x0b, 0x63, 0x6f,
+ 0x6e, 0x66, 0x69, 0x67, 0x4c, 0x61, 0x79, 0x65, 0x72, 0x12, 0x10, 0x0a, 0x03, 0x6d, 0x74, 0x75,
+ 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, 0x6d, 0x74, 0x75, 0x22, 0xad, 0x05, 0x0a, 0x0f,
+ 0x52, 0x6f, 0x75, 0x74, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, 0x70, 0x65, 0x63, 0x12,
+ 0x4a, 0x0a, 0x06, 0x66, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32,
+ 0x32, 0x2e, 0x74, 0x61, 0x6c, 0x6f, 0x73, 0x2e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,
+ 0x2e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x65, 0x6e, 0x75,
+ 0x6d, 0x73, 0x2e, 0x4e, 0x65, 0x74, 0x68, 0x65, 0x6c, 0x70, 0x65, 0x72, 0x73, 0x46, 0x61, 0x6d,
+ 0x69, 0x6c, 0x79, 0x52, 0x06, 0x66, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x12, 0x35, 0x0a, 0x0b, 0x64,
+ 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b,
+ 0x32, 0x13, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4e, 0x65, 0x74, 0x49, 0x50, 0x50,
+ 0x72, 0x65, 0x66, 0x69, 0x78, 0x52, 0x0b, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69,
+ 0x6f, 0x6e, 0x12, 0x25, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01,
+ 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4e, 0x65, 0x74, 0x49,
+ 0x50, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x27, 0x0a, 0x07, 0x67, 0x61, 0x74,
+ 0x65, 0x77, 0x61, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x63, 0x6f, 0x6d,
+ 0x6d, 0x6f, 0x6e, 0x2e, 0x4e, 0x65, 0x74, 0x49, 0x50, 0x52, 0x07, 0x67, 0x61, 0x74, 0x65, 0x77,
+ 0x61, 0x79, 0x12, 0x24, 0x0a, 0x0e, 0x6f, 0x75, 0x74, 0x5f, 0x6c, 0x69, 0x6e, 0x6b, 0x5f, 0x69,
+ 0x6e, 0x64, 0x65, 0x78, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x6f, 0x75, 0x74, 0x4c,
+ 0x69, 0x6e, 0x6b, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x22, 0x0a, 0x0d, 0x6f, 0x75, 0x74, 0x5f,
+ 0x6c, 0x69, 0x6e, 0x6b, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52,
+ 0x0b, 0x6f, 0x75, 0x74, 0x4c, 0x69, 0x6e, 0x6b, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x4e, 0x0a, 0x05,
+ 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x38, 0x2e, 0x74, 0x61,
+ 0x6c, 0x6f, 0x73, 0x2e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x64, 0x65, 0x66,
+ 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x4e,
+ 0x65, 0x74, 0x68, 0x65, 0x6c, 0x70, 0x65, 0x72, 0x73, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67,
+ 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x1a, 0x0a, 0x08,
+ 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08,
+ 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x12, 0x47, 0x0a, 0x05, 0x73, 0x63, 0x6f, 0x70,
+ 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x31, 0x2e, 0x74, 0x61, 0x6c, 0x6f, 0x73, 0x2e,
+ 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74,
+ 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x4e, 0x65, 0x74, 0x68, 0x65,
+ 0x6c, 0x70, 0x65, 0x72, 0x73, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x52, 0x05, 0x73, 0x63, 0x6f, 0x70,
+ 0x65, 0x12, 0x49, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0e, 0x32,
+ 0x35, 0x2e, 0x74, 0x61, 0x6c, 0x6f, 0x73, 0x2e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,
+ 0x2e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x65, 0x6e, 0x75,
+ 0x6d, 0x73, 0x2e, 0x4e, 0x65, 0x74, 0x68, 0x65, 0x6c, 0x70, 0x65, 0x72, 0x73, 0x52, 0x6f, 0x75,
+ 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05,
+ 0x66, 0x6c, 0x61, 0x67, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x66, 0x6c, 0x61,
+ 0x67, 0x73, 0x12, 0x55, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x18, 0x0c,
+ 0x20, 0x01, 0x28, 0x0e, 0x32, 0x39, 0x2e, 0x74, 0x61, 0x6c, 0x6f, 0x73, 0x2e, 0x72, 0x65, 0x73,
+ 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e,
+ 0x73, 0x2e, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x4e, 0x65, 0x74, 0x68, 0x65, 0x6c, 0x70, 0x65,
+ 0x72, 0x73, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x52,
+ 0x08, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x12, 0x10, 0x0a, 0x03, 0x6d, 0x74, 0x75,
+ 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, 0x6d, 0x74, 0x75, 0x22, 0x23, 0x0a, 0x07, 0x53,
+ 0x54, 0x50, 0x53, 0x70, 0x65, 0x63, 0x12, 0x18, 0x0a, 0x07, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65,
+ 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x64,
+ 0x22, 0xaf, 0x01, 0x0a, 0x0a, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, 0x70, 0x65, 0x63, 0x12,
+ 0x23, 0x0a, 0x0d, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x5f, 0x72, 0x65, 0x61, 0x64, 0x79,
+ 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x52,
+ 0x65, 0x61, 0x64, 0x79, 0x12, 0x2d, 0x0a, 0x12, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69,
+ 0x76, 0x69, 0x74, 0x79, 0x5f, 0x72, 0x65, 0x61, 0x64, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08,
+ 0x52, 0x11, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x52, 0x65,
+ 0x61, 0x64, 0x79, 0x12, 0x25, 0x0a, 0x0e, 0x68, 0x6f, 0x73, 0x74, 0x6e, 0x61, 0x6d, 0x65, 0x5f,
+ 0x72, 0x65, 0x61, 0x64, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x68, 0x6f, 0x73,
+ 0x74, 0x6e, 0x61, 0x6d, 0x65, 0x52, 0x65, 0x61, 0x64, 0x79, 0x12, 0x26, 0x0a, 0x0f, 0x65, 0x74,
+ 0x63, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x5f, 0x72, 0x65, 0x61, 0x64, 0x79, 0x18, 0x04, 0x20,
+ 0x01, 0x28, 0x08, 0x52, 0x0d, 0x65, 0x74, 0x63, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x61,
+ 0x64, 0x79, 0x22, 0x5f, 0x0a, 0x0c, 0x54, 0x43, 0x50, 0x50, 0x72, 0x6f, 0x62, 0x65, 0x53, 0x70,
+ 0x65, 0x63, 0x12, 0x1a, 0x0a, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x01,
+ 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x33,
+ 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32,
+ 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75,
+ 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x74, 0x69, 0x6d, 0x65,
+ 0x6f, 0x75, 0x74, 0x22, 0x8e, 0x01, 0x0a, 0x12, 0x54, 0x69, 0x6d, 0x65, 0x53, 0x65, 0x72, 0x76,
0x65, 0x72, 0x53, 0x70, 0x65, 0x63, 0x53, 0x70, 0x65, 0x63, 0x12, 0x1f, 0x0a, 0x0b, 0x6e, 0x74,
0x70, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52,
0x0a, 0x6e, 0x74, 0x70, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x73, 0x12, 0x57, 0x0a, 0x0c, 0x63,
@@ -3228,7 +3440,7 @@ func file_resource_definitions_network_network_proto_rawDescGZIP() []byte {
return file_resource_definitions_network_network_proto_rawDescData
}
-var file_resource_definitions_network_network_proto_msgTypes = make([]protoimpl.MessageInfo, 31)
+var file_resource_definitions_network_network_proto_msgTypes = make([]protoimpl.MessageInfo, 34)
var file_resource_definitions_network_network_proto_goTypes = []interface{}{
(*AddressSpecSpec)(nil), // 0: talos.resource.definitions.network.AddressSpecSpec
(*AddressStatusSpec)(nil), // 1: talos.resource.definitions.network.AddressStatusSpec
@@ -3247,123 +3459,130 @@ var file_resource_definitions_network_network_proto_goTypes = []interface{}{
(*NodeAddressFilterSpec)(nil), // 14: talos.resource.definitions.network.NodeAddressFilterSpec
(*NodeAddressSpec)(nil), // 15: talos.resource.definitions.network.NodeAddressSpec
(*OperatorSpecSpec)(nil), // 16: talos.resource.definitions.network.OperatorSpecSpec
- (*ResolverSpecSpec)(nil), // 17: talos.resource.definitions.network.ResolverSpecSpec
- (*ResolverStatusSpec)(nil), // 18: talos.resource.definitions.network.ResolverStatusSpec
- (*RouteSpecSpec)(nil), // 19: talos.resource.definitions.network.RouteSpecSpec
- (*RouteStatusSpec)(nil), // 20: talos.resource.definitions.network.RouteStatusSpec
- (*STPSpec)(nil), // 21: talos.resource.definitions.network.STPSpec
- (*StatusSpec)(nil), // 22: talos.resource.definitions.network.StatusSpec
- (*TimeServerSpecSpec)(nil), // 23: talos.resource.definitions.network.TimeServerSpecSpec
- (*TimeServerStatusSpec)(nil), // 24: talos.resource.definitions.network.TimeServerStatusSpec
- (*VIPEquinixMetalSpec)(nil), // 25: talos.resource.definitions.network.VIPEquinixMetalSpec
- (*VIPHCloudSpec)(nil), // 26: talos.resource.definitions.network.VIPHCloudSpec
- (*VIPOperatorSpec)(nil), // 27: talos.resource.definitions.network.VIPOperatorSpec
- (*VLANSpec)(nil), // 28: talos.resource.definitions.network.VLANSpec
- (*WireguardPeer)(nil), // 29: talos.resource.definitions.network.WireguardPeer
- (*WireguardSpec)(nil), // 30: talos.resource.definitions.network.WireguardSpec
- (*common.NetIPPrefix)(nil), // 31: common.NetIPPrefix
- (enums.NethelpersFamily)(0), // 32: talos.resource.definitions.enums.NethelpersFamily
- (enums.NethelpersScope)(0), // 33: talos.resource.definitions.enums.NethelpersScope
- (enums.NetworkConfigLayer)(0), // 34: talos.resource.definitions.enums.NetworkConfigLayer
- (*common.NetIP)(nil), // 35: common.NetIP
- (enums.NethelpersBondMode)(0), // 36: talos.resource.definitions.enums.NethelpersBondMode
- (enums.NethelpersBondXmitHashPolicy)(0), // 37: talos.resource.definitions.enums.NethelpersBondXmitHashPolicy
- (enums.NethelpersLACPRate)(0), // 38: talos.resource.definitions.enums.NethelpersLACPRate
- (enums.NethelpersARPValidate)(0), // 39: talos.resource.definitions.enums.NethelpersARPValidate
- (enums.NethelpersARPAllTargets)(0), // 40: talos.resource.definitions.enums.NethelpersARPAllTargets
- (enums.NethelpersPrimaryReselect)(0), // 41: talos.resource.definitions.enums.NethelpersPrimaryReselect
- (enums.NethelpersFailOverMAC)(0), // 42: talos.resource.definitions.enums.NethelpersFailOverMAC
- (enums.NethelpersADSelect)(0), // 43: talos.resource.definitions.enums.NethelpersADSelect
- (enums.NethelpersLinkType)(0), // 44: talos.resource.definitions.enums.NethelpersLinkType
- (enums.NethelpersOperationalState)(0), // 45: talos.resource.definitions.enums.NethelpersOperationalState
- (enums.NethelpersPort)(0), // 46: talos.resource.definitions.enums.NethelpersPort
- (enums.NethelpersDuplex)(0), // 47: talos.resource.definitions.enums.NethelpersDuplex
- (enums.NetworkOperator)(0), // 48: talos.resource.definitions.enums.NetworkOperator
- (enums.NethelpersRoutingTable)(0), // 49: talos.resource.definitions.enums.NethelpersRoutingTable
- (enums.NethelpersRouteType)(0), // 50: talos.resource.definitions.enums.NethelpersRouteType
- (enums.NethelpersRouteProtocol)(0), // 51: talos.resource.definitions.enums.NethelpersRouteProtocol
- (enums.NethelpersVLANProtocol)(0), // 52: talos.resource.definitions.enums.NethelpersVLANProtocol
- (*durationpb.Duration)(nil), // 53: google.protobuf.Duration
+ (*ProbeSpecSpec)(nil), // 17: talos.resource.definitions.network.ProbeSpecSpec
+ (*ProbeStatusSpec)(nil), // 18: talos.resource.definitions.network.ProbeStatusSpec
+ (*ResolverSpecSpec)(nil), // 19: talos.resource.definitions.network.ResolverSpecSpec
+ (*ResolverStatusSpec)(nil), // 20: talos.resource.definitions.network.ResolverStatusSpec
+ (*RouteSpecSpec)(nil), // 21: talos.resource.definitions.network.RouteSpecSpec
+ (*RouteStatusSpec)(nil), // 22: talos.resource.definitions.network.RouteStatusSpec
+ (*STPSpec)(nil), // 23: talos.resource.definitions.network.STPSpec
+ (*StatusSpec)(nil), // 24: talos.resource.definitions.network.StatusSpec
+ (*TCPProbeSpec)(nil), // 25: talos.resource.definitions.network.TCPProbeSpec
+ (*TimeServerSpecSpec)(nil), // 26: talos.resource.definitions.network.TimeServerSpecSpec
+ (*TimeServerStatusSpec)(nil), // 27: talos.resource.definitions.network.TimeServerStatusSpec
+ (*VIPEquinixMetalSpec)(nil), // 28: talos.resource.definitions.network.VIPEquinixMetalSpec
+ (*VIPHCloudSpec)(nil), // 29: talos.resource.definitions.network.VIPHCloudSpec
+ (*VIPOperatorSpec)(nil), // 30: talos.resource.definitions.network.VIPOperatorSpec
+ (*VLANSpec)(nil), // 31: talos.resource.definitions.network.VLANSpec
+ (*WireguardPeer)(nil), // 32: talos.resource.definitions.network.WireguardPeer
+ (*WireguardSpec)(nil), // 33: talos.resource.definitions.network.WireguardSpec
+ (*common.NetIPPrefix)(nil), // 34: common.NetIPPrefix
+ (enums.NethelpersFamily)(0), // 35: talos.resource.definitions.enums.NethelpersFamily
+ (enums.NethelpersScope)(0), // 36: talos.resource.definitions.enums.NethelpersScope
+ (enums.NetworkConfigLayer)(0), // 37: talos.resource.definitions.enums.NetworkConfigLayer
+ (*common.NetIP)(nil), // 38: common.NetIP
+ (enums.NethelpersBondMode)(0), // 39: talos.resource.definitions.enums.NethelpersBondMode
+ (enums.NethelpersBondXmitHashPolicy)(0), // 40: talos.resource.definitions.enums.NethelpersBondXmitHashPolicy
+ (enums.NethelpersLACPRate)(0), // 41: talos.resource.definitions.enums.NethelpersLACPRate
+ (enums.NethelpersARPValidate)(0), // 42: talos.resource.definitions.enums.NethelpersARPValidate
+ (enums.NethelpersARPAllTargets)(0), // 43: talos.resource.definitions.enums.NethelpersARPAllTargets
+ (enums.NethelpersPrimaryReselect)(0), // 44: talos.resource.definitions.enums.NethelpersPrimaryReselect
+ (enums.NethelpersFailOverMAC)(0), // 45: talos.resource.definitions.enums.NethelpersFailOverMAC
+ (enums.NethelpersADSelect)(0), // 46: talos.resource.definitions.enums.NethelpersADSelect
+ (enums.NethelpersLinkType)(0), // 47: talos.resource.definitions.enums.NethelpersLinkType
+ (enums.NethelpersOperationalState)(0), // 48: talos.resource.definitions.enums.NethelpersOperationalState
+ (enums.NethelpersPort)(0), // 49: talos.resource.definitions.enums.NethelpersPort
+ (enums.NethelpersDuplex)(0), // 50: talos.resource.definitions.enums.NethelpersDuplex
+ (enums.NetworkOperator)(0), // 51: talos.resource.definitions.enums.NetworkOperator
+ (*durationpb.Duration)(nil), // 52: google.protobuf.Duration
+ (enums.NethelpersRoutingTable)(0), // 53: talos.resource.definitions.enums.NethelpersRoutingTable
+ (enums.NethelpersRouteType)(0), // 54: talos.resource.definitions.enums.NethelpersRouteType
+ (enums.NethelpersRouteProtocol)(0), // 55: talos.resource.definitions.enums.NethelpersRouteProtocol
+ (enums.NethelpersVLANProtocol)(0), // 56: talos.resource.definitions.enums.NethelpersVLANProtocol
}
var file_resource_definitions_network_network_proto_depIdxs = []int32{
- 31, // 0: talos.resource.definitions.network.AddressSpecSpec.address:type_name -> common.NetIPPrefix
- 32, // 1: talos.resource.definitions.network.AddressSpecSpec.family:type_name -> talos.resource.definitions.enums.NethelpersFamily
- 33, // 2: talos.resource.definitions.network.AddressSpecSpec.scope:type_name -> talos.resource.definitions.enums.NethelpersScope
- 34, // 3: talos.resource.definitions.network.AddressSpecSpec.config_layer:type_name -> talos.resource.definitions.enums.NetworkConfigLayer
- 31, // 4: talos.resource.definitions.network.AddressStatusSpec.address:type_name -> common.NetIPPrefix
- 35, // 5: talos.resource.definitions.network.AddressStatusSpec.local:type_name -> common.NetIP
- 35, // 6: talos.resource.definitions.network.AddressStatusSpec.broadcast:type_name -> common.NetIP
- 35, // 7: talos.resource.definitions.network.AddressStatusSpec.anycast:type_name -> common.NetIP
- 35, // 8: talos.resource.definitions.network.AddressStatusSpec.multicast:type_name -> common.NetIP
- 32, // 9: talos.resource.definitions.network.AddressStatusSpec.family:type_name -> talos.resource.definitions.enums.NethelpersFamily
- 33, // 10: talos.resource.definitions.network.AddressStatusSpec.scope:type_name -> talos.resource.definitions.enums.NethelpersScope
- 36, // 11: talos.resource.definitions.network.BondMasterSpec.mode:type_name -> talos.resource.definitions.enums.NethelpersBondMode
- 37, // 12: talos.resource.definitions.network.BondMasterSpec.hash_policy:type_name -> talos.resource.definitions.enums.NethelpersBondXmitHashPolicy
- 38, // 13: talos.resource.definitions.network.BondMasterSpec.lacp_rate:type_name -> talos.resource.definitions.enums.NethelpersLACPRate
- 39, // 14: talos.resource.definitions.network.BondMasterSpec.arp_validate:type_name -> talos.resource.definitions.enums.NethelpersARPValidate
- 40, // 15: talos.resource.definitions.network.BondMasterSpec.arp_all_targets:type_name -> talos.resource.definitions.enums.NethelpersARPAllTargets
- 41, // 16: talos.resource.definitions.network.BondMasterSpec.primary_reselect:type_name -> talos.resource.definitions.enums.NethelpersPrimaryReselect
- 42, // 17: talos.resource.definitions.network.BondMasterSpec.fail_over_mac:type_name -> talos.resource.definitions.enums.NethelpersFailOverMAC
- 43, // 18: talos.resource.definitions.network.BondMasterSpec.ad_select:type_name -> talos.resource.definitions.enums.NethelpersADSelect
- 21, // 19: talos.resource.definitions.network.BridgeMasterSpec.stp:type_name -> talos.resource.definitions.network.STPSpec
- 34, // 20: talos.resource.definitions.network.HostnameSpecSpec.config_layer:type_name -> talos.resource.definitions.enums.NetworkConfigLayer
- 44, // 21: talos.resource.definitions.network.LinkSpecSpec.type:type_name -> talos.resource.definitions.enums.NethelpersLinkType
+ 34, // 0: talos.resource.definitions.network.AddressSpecSpec.address:type_name -> common.NetIPPrefix
+ 35, // 1: talos.resource.definitions.network.AddressSpecSpec.family:type_name -> talos.resource.definitions.enums.NethelpersFamily
+ 36, // 2: talos.resource.definitions.network.AddressSpecSpec.scope:type_name -> talos.resource.definitions.enums.NethelpersScope
+ 37, // 3: talos.resource.definitions.network.AddressSpecSpec.config_layer:type_name -> talos.resource.definitions.enums.NetworkConfigLayer
+ 34, // 4: talos.resource.definitions.network.AddressStatusSpec.address:type_name -> common.NetIPPrefix
+ 38, // 5: talos.resource.definitions.network.AddressStatusSpec.local:type_name -> common.NetIP
+ 38, // 6: talos.resource.definitions.network.AddressStatusSpec.broadcast:type_name -> common.NetIP
+ 38, // 7: talos.resource.definitions.network.AddressStatusSpec.anycast:type_name -> common.NetIP
+ 38, // 8: talos.resource.definitions.network.AddressStatusSpec.multicast:type_name -> common.NetIP
+ 35, // 9: talos.resource.definitions.network.AddressStatusSpec.family:type_name -> talos.resource.definitions.enums.NethelpersFamily
+ 36, // 10: talos.resource.definitions.network.AddressStatusSpec.scope:type_name -> talos.resource.definitions.enums.NethelpersScope
+ 39, // 11: talos.resource.definitions.network.BondMasterSpec.mode:type_name -> talos.resource.definitions.enums.NethelpersBondMode
+ 40, // 12: talos.resource.definitions.network.BondMasterSpec.hash_policy:type_name -> talos.resource.definitions.enums.NethelpersBondXmitHashPolicy
+ 41, // 13: talos.resource.definitions.network.BondMasterSpec.lacp_rate:type_name -> talos.resource.definitions.enums.NethelpersLACPRate
+ 42, // 14: talos.resource.definitions.network.BondMasterSpec.arp_validate:type_name -> talos.resource.definitions.enums.NethelpersARPValidate
+ 43, // 15: talos.resource.definitions.network.BondMasterSpec.arp_all_targets:type_name -> talos.resource.definitions.enums.NethelpersARPAllTargets
+ 44, // 16: talos.resource.definitions.network.BondMasterSpec.primary_reselect:type_name -> talos.resource.definitions.enums.NethelpersPrimaryReselect
+ 45, // 17: talos.resource.definitions.network.BondMasterSpec.fail_over_mac:type_name -> talos.resource.definitions.enums.NethelpersFailOverMAC
+ 46, // 18: talos.resource.definitions.network.BondMasterSpec.ad_select:type_name -> talos.resource.definitions.enums.NethelpersADSelect
+ 23, // 19: talos.resource.definitions.network.BridgeMasterSpec.stp:type_name -> talos.resource.definitions.network.STPSpec
+ 37, // 20: talos.resource.definitions.network.HostnameSpecSpec.config_layer:type_name -> talos.resource.definitions.enums.NetworkConfigLayer
+ 47, // 21: talos.resource.definitions.network.LinkSpecSpec.type:type_name -> talos.resource.definitions.enums.NethelpersLinkType
3, // 22: talos.resource.definitions.network.LinkSpecSpec.bond_slave:type_name -> talos.resource.definitions.network.BondSlave
5, // 23: talos.resource.definitions.network.LinkSpecSpec.bridge_slave:type_name -> talos.resource.definitions.network.BridgeSlave
- 28, // 24: talos.resource.definitions.network.LinkSpecSpec.vlan:type_name -> talos.resource.definitions.network.VLANSpec
+ 31, // 24: talos.resource.definitions.network.LinkSpecSpec.vlan:type_name -> talos.resource.definitions.network.VLANSpec
2, // 25: talos.resource.definitions.network.LinkSpecSpec.bond_master:type_name -> talos.resource.definitions.network.BondMasterSpec
4, // 26: talos.resource.definitions.network.LinkSpecSpec.bridge_master:type_name -> talos.resource.definitions.network.BridgeMasterSpec
- 30, // 27: talos.resource.definitions.network.LinkSpecSpec.wireguard:type_name -> talos.resource.definitions.network.WireguardSpec
- 34, // 28: talos.resource.definitions.network.LinkSpecSpec.config_layer:type_name -> talos.resource.definitions.enums.NetworkConfigLayer
- 44, // 29: talos.resource.definitions.network.LinkStatusSpec.type:type_name -> talos.resource.definitions.enums.NethelpersLinkType
- 45, // 30: talos.resource.definitions.network.LinkStatusSpec.operational_state:type_name -> talos.resource.definitions.enums.NethelpersOperationalState
- 46, // 31: talos.resource.definitions.network.LinkStatusSpec.port:type_name -> talos.resource.definitions.enums.NethelpersPort
- 47, // 32: talos.resource.definitions.network.LinkStatusSpec.duplex:type_name -> talos.resource.definitions.enums.NethelpersDuplex
- 28, // 33: talos.resource.definitions.network.LinkStatusSpec.vlan:type_name -> talos.resource.definitions.network.VLANSpec
+ 33, // 27: talos.resource.definitions.network.LinkSpecSpec.wireguard:type_name -> talos.resource.definitions.network.WireguardSpec
+ 37, // 28: talos.resource.definitions.network.LinkSpecSpec.config_layer:type_name -> talos.resource.definitions.enums.NetworkConfigLayer
+ 47, // 29: talos.resource.definitions.network.LinkStatusSpec.type:type_name -> talos.resource.definitions.enums.NethelpersLinkType
+ 48, // 30: talos.resource.definitions.network.LinkStatusSpec.operational_state:type_name -> talos.resource.definitions.enums.NethelpersOperationalState
+ 49, // 31: talos.resource.definitions.network.LinkStatusSpec.port:type_name -> talos.resource.definitions.enums.NethelpersPort
+ 50, // 32: talos.resource.definitions.network.LinkStatusSpec.duplex:type_name -> talos.resource.definitions.enums.NethelpersDuplex
+ 31, // 33: talos.resource.definitions.network.LinkStatusSpec.vlan:type_name -> talos.resource.definitions.network.VLANSpec
4, // 34: talos.resource.definitions.network.LinkStatusSpec.bridge_master:type_name -> talos.resource.definitions.network.BridgeMasterSpec
2, // 35: talos.resource.definitions.network.LinkStatusSpec.bond_master:type_name -> talos.resource.definitions.network.BondMasterSpec
- 30, // 36: talos.resource.definitions.network.LinkStatusSpec.wireguard:type_name -> talos.resource.definitions.network.WireguardSpec
- 31, // 37: talos.resource.definitions.network.NodeAddressFilterSpec.include_subnets:type_name -> common.NetIPPrefix
- 31, // 38: talos.resource.definitions.network.NodeAddressFilterSpec.exclude_subnets:type_name -> common.NetIPPrefix
- 31, // 39: talos.resource.definitions.network.NodeAddressSpec.addresses:type_name -> common.NetIPPrefix
- 48, // 40: talos.resource.definitions.network.OperatorSpecSpec.operator:type_name -> talos.resource.definitions.enums.NetworkOperator
+ 33, // 36: talos.resource.definitions.network.LinkStatusSpec.wireguard:type_name -> talos.resource.definitions.network.WireguardSpec
+ 34, // 37: talos.resource.definitions.network.NodeAddressFilterSpec.include_subnets:type_name -> common.NetIPPrefix
+ 34, // 38: talos.resource.definitions.network.NodeAddressFilterSpec.exclude_subnets:type_name -> common.NetIPPrefix
+ 34, // 39: talos.resource.definitions.network.NodeAddressSpec.addresses:type_name -> common.NetIPPrefix
+ 51, // 40: talos.resource.definitions.network.OperatorSpecSpec.operator:type_name -> talos.resource.definitions.enums.NetworkOperator
6, // 41: talos.resource.definitions.network.OperatorSpecSpec.dhcp4:type_name -> talos.resource.definitions.network.DHCP4OperatorSpec
7, // 42: talos.resource.definitions.network.OperatorSpecSpec.dhcp6:type_name -> talos.resource.definitions.network.DHCP6OperatorSpec
- 27, // 43: talos.resource.definitions.network.OperatorSpecSpec.vip:type_name -> talos.resource.definitions.network.VIPOperatorSpec
- 34, // 44: talos.resource.definitions.network.OperatorSpecSpec.config_layer:type_name -> talos.resource.definitions.enums.NetworkConfigLayer
- 35, // 45: talos.resource.definitions.network.ResolverSpecSpec.dns_servers:type_name -> common.NetIP
- 34, // 46: talos.resource.definitions.network.ResolverSpecSpec.config_layer:type_name -> talos.resource.definitions.enums.NetworkConfigLayer
- 35, // 47: talos.resource.definitions.network.ResolverStatusSpec.dns_servers:type_name -> common.NetIP
- 32, // 48: talos.resource.definitions.network.RouteSpecSpec.family:type_name -> talos.resource.definitions.enums.NethelpersFamily
- 31, // 49: talos.resource.definitions.network.RouteSpecSpec.destination:type_name -> common.NetIPPrefix
- 35, // 50: talos.resource.definitions.network.RouteSpecSpec.source:type_name -> common.NetIP
- 35, // 51: talos.resource.definitions.network.RouteSpecSpec.gateway:type_name -> common.NetIP
- 49, // 52: talos.resource.definitions.network.RouteSpecSpec.table:type_name -> talos.resource.definitions.enums.NethelpersRoutingTable
- 33, // 53: talos.resource.definitions.network.RouteSpecSpec.scope:type_name -> talos.resource.definitions.enums.NethelpersScope
- 50, // 54: talos.resource.definitions.network.RouteSpecSpec.type:type_name -> talos.resource.definitions.enums.NethelpersRouteType
- 51, // 55: talos.resource.definitions.network.RouteSpecSpec.protocol:type_name -> talos.resource.definitions.enums.NethelpersRouteProtocol
- 34, // 56: talos.resource.definitions.network.RouteSpecSpec.config_layer:type_name -> talos.resource.definitions.enums.NetworkConfigLayer
- 32, // 57: talos.resource.definitions.network.RouteStatusSpec.family:type_name -> talos.resource.definitions.enums.NethelpersFamily
- 31, // 58: talos.resource.definitions.network.RouteStatusSpec.destination:type_name -> common.NetIPPrefix
- 35, // 59: talos.resource.definitions.network.RouteStatusSpec.source:type_name -> common.NetIP
- 35, // 60: talos.resource.definitions.network.RouteStatusSpec.gateway:type_name -> common.NetIP
- 49, // 61: talos.resource.definitions.network.RouteStatusSpec.table:type_name -> talos.resource.definitions.enums.NethelpersRoutingTable
- 33, // 62: talos.resource.definitions.network.RouteStatusSpec.scope:type_name -> talos.resource.definitions.enums.NethelpersScope
- 50, // 63: talos.resource.definitions.network.RouteStatusSpec.type:type_name -> talos.resource.definitions.enums.NethelpersRouteType
- 51, // 64: talos.resource.definitions.network.RouteStatusSpec.protocol:type_name -> talos.resource.definitions.enums.NethelpersRouteProtocol
- 34, // 65: talos.resource.definitions.network.TimeServerSpecSpec.config_layer:type_name -> talos.resource.definitions.enums.NetworkConfigLayer
- 35, // 66: talos.resource.definitions.network.VIPOperatorSpec.ip:type_name -> common.NetIP
- 25, // 67: talos.resource.definitions.network.VIPOperatorSpec.equinix_metal:type_name -> talos.resource.definitions.network.VIPEquinixMetalSpec
- 26, // 68: talos.resource.definitions.network.VIPOperatorSpec.h_cloud:type_name -> talos.resource.definitions.network.VIPHCloudSpec
- 52, // 69: talos.resource.definitions.network.VLANSpec.protocol:type_name -> talos.resource.definitions.enums.NethelpersVLANProtocol
- 53, // 70: talos.resource.definitions.network.WireguardPeer.persistent_keepalive_interval:type_name -> google.protobuf.Duration
- 31, // 71: talos.resource.definitions.network.WireguardPeer.allowed_ips:type_name -> common.NetIPPrefix
- 29, // 72: talos.resource.definitions.network.WireguardSpec.peers:type_name -> talos.resource.definitions.network.WireguardPeer
- 73, // [73:73] is the sub-list for method output_type
- 73, // [73:73] is the sub-list for method input_type
- 73, // [73:73] is the sub-list for extension type_name
- 73, // [73:73] is the sub-list for extension extendee
- 0, // [0:73] is the sub-list for field type_name
+ 30, // 43: talos.resource.definitions.network.OperatorSpecSpec.vip:type_name -> talos.resource.definitions.network.VIPOperatorSpec
+ 37, // 44: talos.resource.definitions.network.OperatorSpecSpec.config_layer:type_name -> talos.resource.definitions.enums.NetworkConfigLayer
+ 52, // 45: talos.resource.definitions.network.ProbeSpecSpec.interval:type_name -> google.protobuf.Duration
+ 25, // 46: talos.resource.definitions.network.ProbeSpecSpec.tcp:type_name -> talos.resource.definitions.network.TCPProbeSpec
+ 37, // 47: talos.resource.definitions.network.ProbeSpecSpec.config_layer:type_name -> talos.resource.definitions.enums.NetworkConfigLayer
+ 38, // 48: talos.resource.definitions.network.ResolverSpecSpec.dns_servers:type_name -> common.NetIP
+ 37, // 49: talos.resource.definitions.network.ResolverSpecSpec.config_layer:type_name -> talos.resource.definitions.enums.NetworkConfigLayer
+ 38, // 50: talos.resource.definitions.network.ResolverStatusSpec.dns_servers:type_name -> common.NetIP
+ 35, // 51: talos.resource.definitions.network.RouteSpecSpec.family:type_name -> talos.resource.definitions.enums.NethelpersFamily
+ 34, // 52: talos.resource.definitions.network.RouteSpecSpec.destination:type_name -> common.NetIPPrefix
+ 38, // 53: talos.resource.definitions.network.RouteSpecSpec.source:type_name -> common.NetIP
+ 38, // 54: talos.resource.definitions.network.RouteSpecSpec.gateway:type_name -> common.NetIP
+ 53, // 55: talos.resource.definitions.network.RouteSpecSpec.table:type_name -> talos.resource.definitions.enums.NethelpersRoutingTable
+ 36, // 56: talos.resource.definitions.network.RouteSpecSpec.scope:type_name -> talos.resource.definitions.enums.NethelpersScope
+ 54, // 57: talos.resource.definitions.network.RouteSpecSpec.type:type_name -> talos.resource.definitions.enums.NethelpersRouteType
+ 55, // 58: talos.resource.definitions.network.RouteSpecSpec.protocol:type_name -> talos.resource.definitions.enums.NethelpersRouteProtocol
+ 37, // 59: talos.resource.definitions.network.RouteSpecSpec.config_layer:type_name -> talos.resource.definitions.enums.NetworkConfigLayer
+ 35, // 60: talos.resource.definitions.network.RouteStatusSpec.family:type_name -> talos.resource.definitions.enums.NethelpersFamily
+ 34, // 61: talos.resource.definitions.network.RouteStatusSpec.destination:type_name -> common.NetIPPrefix
+ 38, // 62: talos.resource.definitions.network.RouteStatusSpec.source:type_name -> common.NetIP
+ 38, // 63: talos.resource.definitions.network.RouteStatusSpec.gateway:type_name -> common.NetIP
+ 53, // 64: talos.resource.definitions.network.RouteStatusSpec.table:type_name -> talos.resource.definitions.enums.NethelpersRoutingTable
+ 36, // 65: talos.resource.definitions.network.RouteStatusSpec.scope:type_name -> talos.resource.definitions.enums.NethelpersScope
+ 54, // 66: talos.resource.definitions.network.RouteStatusSpec.type:type_name -> talos.resource.definitions.enums.NethelpersRouteType
+ 55, // 67: talos.resource.definitions.network.RouteStatusSpec.protocol:type_name -> talos.resource.definitions.enums.NethelpersRouteProtocol
+ 52, // 68: talos.resource.definitions.network.TCPProbeSpec.timeout:type_name -> google.protobuf.Duration
+ 37, // 69: talos.resource.definitions.network.TimeServerSpecSpec.config_layer:type_name -> talos.resource.definitions.enums.NetworkConfigLayer
+ 38, // 70: talos.resource.definitions.network.VIPOperatorSpec.ip:type_name -> common.NetIP
+ 28, // 71: talos.resource.definitions.network.VIPOperatorSpec.equinix_metal:type_name -> talos.resource.definitions.network.VIPEquinixMetalSpec
+ 29, // 72: talos.resource.definitions.network.VIPOperatorSpec.h_cloud:type_name -> talos.resource.definitions.network.VIPHCloudSpec
+ 56, // 73: talos.resource.definitions.network.VLANSpec.protocol:type_name -> talos.resource.definitions.enums.NethelpersVLANProtocol
+ 52, // 74: talos.resource.definitions.network.WireguardPeer.persistent_keepalive_interval:type_name -> google.protobuf.Duration
+ 34, // 75: talos.resource.definitions.network.WireguardPeer.allowed_ips:type_name -> common.NetIPPrefix
+ 32, // 76: talos.resource.definitions.network.WireguardSpec.peers:type_name -> talos.resource.definitions.network.WireguardPeer
+ 77, // [77:77] is the sub-list for method output_type
+ 77, // [77:77] is the sub-list for method input_type
+ 77, // [77:77] is the sub-list for extension type_name
+ 77, // [77:77] is the sub-list for extension extendee
+ 0, // [0:77] is the sub-list for field type_name
}
func init() { file_resource_definitions_network_network_proto_init() }
@@ -3577,7 +3796,7 @@ func file_resource_definitions_network_network_proto_init() {
}
}
file_resource_definitions_network_network_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*ResolverSpecSpec); i {
+ switch v := v.(*ProbeSpecSpec); i {
case 0:
return &v.state
case 1:
@@ -3589,7 +3808,7 @@ func file_resource_definitions_network_network_proto_init() {
}
}
file_resource_definitions_network_network_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*ResolverStatusSpec); i {
+ switch v := v.(*ProbeStatusSpec); i {
case 0:
return &v.state
case 1:
@@ -3601,7 +3820,7 @@ func file_resource_definitions_network_network_proto_init() {
}
}
file_resource_definitions_network_network_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*RouteSpecSpec); i {
+ switch v := v.(*ResolverSpecSpec); i {
case 0:
return &v.state
case 1:
@@ -3613,7 +3832,7 @@ func file_resource_definitions_network_network_proto_init() {
}
}
file_resource_definitions_network_network_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*RouteStatusSpec); i {
+ switch v := v.(*ResolverStatusSpec); i {
case 0:
return &v.state
case 1:
@@ -3625,7 +3844,7 @@ func file_resource_definitions_network_network_proto_init() {
}
}
file_resource_definitions_network_network_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*STPSpec); i {
+ switch v := v.(*RouteSpecSpec); i {
case 0:
return &v.state
case 1:
@@ -3637,7 +3856,7 @@ func file_resource_definitions_network_network_proto_init() {
}
}
file_resource_definitions_network_network_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*StatusSpec); i {
+ switch v := v.(*RouteStatusSpec); i {
case 0:
return &v.state
case 1:
@@ -3649,7 +3868,7 @@ func file_resource_definitions_network_network_proto_init() {
}
}
file_resource_definitions_network_network_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*TimeServerSpecSpec); i {
+ switch v := v.(*STPSpec); i {
case 0:
return &v.state
case 1:
@@ -3661,7 +3880,7 @@ func file_resource_definitions_network_network_proto_init() {
}
}
file_resource_definitions_network_network_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*TimeServerStatusSpec); i {
+ switch v := v.(*StatusSpec); i {
case 0:
return &v.state
case 1:
@@ -3673,7 +3892,7 @@ func file_resource_definitions_network_network_proto_init() {
}
}
file_resource_definitions_network_network_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VIPEquinixMetalSpec); i {
+ switch v := v.(*TCPProbeSpec); i {
case 0:
return &v.state
case 1:
@@ -3685,7 +3904,7 @@ func file_resource_definitions_network_network_proto_init() {
}
}
file_resource_definitions_network_network_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VIPHCloudSpec); i {
+ switch v := v.(*TimeServerSpecSpec); i {
case 0:
return &v.state
case 1:
@@ -3697,7 +3916,7 @@ func file_resource_definitions_network_network_proto_init() {
}
}
file_resource_definitions_network_network_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VIPOperatorSpec); i {
+ switch v := v.(*TimeServerStatusSpec); i {
case 0:
return &v.state
case 1:
@@ -3709,7 +3928,7 @@ func file_resource_definitions_network_network_proto_init() {
}
}
file_resource_definitions_network_network_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*VLANSpec); i {
+ switch v := v.(*VIPEquinixMetalSpec); i {
case 0:
return &v.state
case 1:
@@ -3721,7 +3940,7 @@ func file_resource_definitions_network_network_proto_init() {
}
}
file_resource_definitions_network_network_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*WireguardPeer); i {
+ switch v := v.(*VIPHCloudSpec); i {
case 0:
return &v.state
case 1:
@@ -3733,6 +3952,42 @@ func file_resource_definitions_network_network_proto_init() {
}
}
file_resource_definitions_network_network_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} {
+ switch v := v.(*VIPOperatorSpec); i {
+ case 0:
+ return &v.state
+ case 1:
+ return &v.sizeCache
+ case 2:
+ return &v.unknownFields
+ default:
+ return nil
+ }
+ }
+ file_resource_definitions_network_network_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} {
+ switch v := v.(*VLANSpec); i {
+ case 0:
+ return &v.state
+ case 1:
+ return &v.sizeCache
+ case 2:
+ return &v.unknownFields
+ default:
+ return nil
+ }
+ }
+ file_resource_definitions_network_network_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} {
+ switch v := v.(*WireguardPeer); i {
+ case 0:
+ return &v.state
+ case 1:
+ return &v.sizeCache
+ case 2:
+ return &v.unknownFields
+ default:
+ return nil
+ }
+ }
+ file_resource_definitions_network_network_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*WireguardSpec); i {
case 0:
return &v.state
@@ -3751,7 +4006,7 @@ func file_resource_definitions_network_network_proto_init() {
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_resource_definitions_network_network_proto_rawDesc,
NumEnums: 0,
- NumMessages: 31,
+ NumMessages: 34,
NumExtensions: 0,
NumServices: 0,
},
diff --git a/pkg/machinery/api/resource/definitions/network/network_vtproto.pb.go b/pkg/machinery/api/resource/definitions/network/network_vtproto.pb.go
index 7df5a29da..f2129d078 100644
--- a/pkg/machinery/api/resource/definitions/network/network_vtproto.pb.go
+++ b/pkg/machinery/api/resource/definitions/network/network_vtproto.pb.go
@@ -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
diff --git a/pkg/machinery/resources/network/address_spec.go b/pkg/machinery/resources/network/address_spec.go
index 469a0bb9b..a79997c8c 100644
--- a/pkg/machinery/resources/network/address_spec.go
+++ b/pkg/machinery/resources/network/address_spec.go
@@ -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")
diff --git a/pkg/machinery/resources/network/deep_copy.generated.go b/pkg/machinery/resources/network/deep_copy.generated.go
index 81a1c9c26..89b13fb51 100644
--- a/pkg/machinery/resources/network/deep_copy.generated.go
+++ b/pkg/machinery/resources/network/deep_copy.generated.go
@@ -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
diff --git a/pkg/machinery/resources/network/network_test.go b/pkg/machinery/resources/network/network_test.go
index 34c386a33..70c1ade48 100644
--- a/pkg/machinery/resources/network/network_test.go
+++ b/pkg/machinery/resources/network/network_test.go
@@ -36,6 +36,7 @@ func TestRegisterResource(t *testing.T) {
&network.NodeAddress{},
&network.NodeAddressFilter{},
&network.OperatorSpec{},
+ &network.ProbeSpec{},
&network.ResolverStatus{},
&network.ResolverSpec{},
&network.RouteStatus{},
diff --git a/pkg/machinery/resources/network/probe_spec.go b/pkg/machinery/resources/network/probe_spec.go
new file mode 100644
index 000000000..3d4d61dfe
--- /dev/null
+++ b/pkg/machinery/resources/network/probe_spec.go
@@ -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)
+ }
+}
diff --git a/pkg/machinery/resources/network/probe_status.go b/pkg/machinery/resources/network/probe_status.go
new file mode 100644
index 000000000..09322b945
--- /dev/null
+++ b/pkg/machinery/resources/network/probe_status.go
@@ -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)
+ }
+}
diff --git a/pkg/machinery/resources/network/status.go b/pkg/machinery/resources/network/status.go
index e7f7acdab..9876e3706 100644
--- a/pkg/machinery/resources/network/status.go
+++ b/pkg/machinery/resources/network/status.go
@@ -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}",
+ },
+ },
}
}
diff --git a/website/content/v1.4/reference/api.md b/website/content/v1.4/reference/api.md
index 6aab7b06b..b42398064 100644
--- a/website/content/v1.4/reference/api.md
+++ b/website/content/v1.4/reference/api.md
@@ -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.
+
+
+### 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) | | |
+
+
+
+
+
+
+
+
+### ProbeStatusSpec
+ProbeStatusSpec describes the Probe.
+
+
+| Field | Type | Label | Description |
+| ----- | ---- | ----- | ----------- |
+| success | [bool](#bool) | | |
+| last_error | [string](#string) | | |
+
+
+
+
+
+
### ResolverSpecSpec
@@ -2855,6 +2892,22 @@ StatusSpec describes network state.
+
+
+### TCPProbeSpec
+TCPProbeSpec describes the TCP Probe.
+
+
+| Field | Type | Label | Description |
+| ----- | ---- | ----- | ----------- |
+| endpoint | [string](#string) | | |
+| timeout | [google.protobuf.Duration](#google.protobuf.Duration) | | |
+
+
+
+
+
+
### TimeServerSpecSpec