refactor: allow kmsg log streaming to be reconfigured on the fly
Fixes #7226 This follows same flow as other similar changes - split out logging configuration as a separate resource, source it for now in the cmdline. Rewrite the controller to allow multiple log outputs, add send retries. Signed-off-by: Andrey Smirnov <andrey.smirnov@talos-systems.com>
This commit is contained in:
parent
8a02ecd4cb
commit
5dab45e869
@ -4,6 +4,7 @@ package talos.resource.definitions.runtime;
|
||||
|
||||
option go_package = "github.com/siderolabs/talos/pkg/machinery/api/resource/definitions/runtime";
|
||||
|
||||
import "common/common.proto";
|
||||
import "resource/definitions/enums/enums.proto";
|
||||
|
||||
// DevicesStatusSpec is the spec for devices status.
|
||||
@ -30,6 +31,11 @@ message KernelParamStatusSpec {
|
||||
bool unsupported = 3;
|
||||
}
|
||||
|
||||
// KmsgLogConfigSpec describes status of the defined sysctls.
|
||||
message KmsgLogConfigSpec {
|
||||
repeated common.URL destinations = 1;
|
||||
}
|
||||
|
||||
// MachineStatusSpec describes status of the defined sysctls.
|
||||
message MachineStatusSpec {
|
||||
talos.resource.definitions.enums.RuntimeMachineStage stage = 1;
|
||||
|
66
internal/app/machined/pkg/controllers/network/utils/utils.go
Normal file
66
internal/app/machined/pkg/controllers/network/utils/utils.go
Normal file
@ -0,0 +1,66 @@
|
||||
// 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 networkutils provides utilities for controllers to interact with network resources.
|
||||
package networkutils
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/cosi-project/runtime/pkg/controller"
|
||||
"github.com/cosi-project/runtime/pkg/safe"
|
||||
"github.com/cosi-project/runtime/pkg/state"
|
||||
"github.com/siderolabs/go-pointer"
|
||||
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/network"
|
||||
)
|
||||
|
||||
// WaitForNetworkReady waits for devices to be ready.
|
||||
//
|
||||
// It is a helper function for controllers.
|
||||
func WaitForNetworkReady(ctx context.Context, r controller.Runtime, condition func(*network.StatusSpec) bool, nextInputs []controller.Input) error {
|
||||
// set inputs temporarily to a service only
|
||||
if err := r.UpdateInputs([]controller.Input{
|
||||
{
|
||||
Namespace: network.NamespaceName,
|
||||
Type: network.StatusType,
|
||||
ID: pointer.To(network.StatusID),
|
||||
Kind: controller.InputWeak,
|
||||
},
|
||||
}); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
case <-r.EventCh():
|
||||
}
|
||||
|
||||
status, err := safe.ReaderGetByID[*network.Status](ctx, r, network.StatusID)
|
||||
if err != nil {
|
||||
if state.IsNotFoundError(err) {
|
||||
continue
|
||||
}
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
if condition(status.TypedSpec()) {
|
||||
// condition met
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
// restore inputs
|
||||
if err := r.UpdateInputs(nextInputs); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// queue an update to reprocess with new inputs
|
||||
r.QueueReconcile()
|
||||
|
||||
return nil
|
||||
}
|
@ -6,34 +6,41 @@ package runtime
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"net/url"
|
||||
"time"
|
||||
|
||||
"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/channel"
|
||||
"github.com/siderolabs/gen/slices"
|
||||
"github.com/siderolabs/go-kmsg"
|
||||
"github.com/siderolabs/go-pointer"
|
||||
"github.com/siderolabs/go-procfs/procfs"
|
||||
"go.uber.org/zap"
|
||||
"go.uber.org/zap/zapcore"
|
||||
|
||||
"github.com/siderolabs/talos/internal/app/machined/pkg/runtime"
|
||||
networkutils "github.com/siderolabs/talos/internal/app/machined/pkg/controllers/network/utils"
|
||||
machinedruntime "github.com/siderolabs/talos/internal/app/machined/pkg/runtime"
|
||||
"github.com/siderolabs/talos/internal/app/machined/pkg/runtime/logging"
|
||||
"github.com/siderolabs/talos/pkg/machinery/constants"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/network"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/runtime"
|
||||
)
|
||||
|
||||
const drainTimeout = 100 * time.Millisecond
|
||||
const (
|
||||
drainTimeout = 100 * time.Millisecond
|
||||
logSendTimeout = 5 * time.Second
|
||||
logRetryTimeout = 1 * time.Second
|
||||
logCloseTimeout = 5 * time.Second
|
||||
)
|
||||
|
||||
// KmsgLogDeliveryController watches events and forwards them to the events sink server
|
||||
// if it's configured.
|
||||
type KmsgLogDeliveryController struct {
|
||||
Cmdline *procfs.Cmdline
|
||||
Drainer *runtime.Drainer
|
||||
Drainer *machinedruntime.Drainer
|
||||
|
||||
drainSub *runtime.DrainSubscription
|
||||
drainSub *machinedruntime.DrainSubscription
|
||||
}
|
||||
|
||||
// Name implements controller.Controller interface.
|
||||
@ -43,14 +50,7 @@ func (ctrl *KmsgLogDeliveryController) Name() string {
|
||||
|
||||
// Inputs implements controller.Controller interface.
|
||||
func (ctrl *KmsgLogDeliveryController) Inputs() []controller.Input {
|
||||
return []controller.Input{
|
||||
{
|
||||
Namespace: network.NamespaceName,
|
||||
Type: network.StatusType,
|
||||
ID: pointer.To(network.StatusID),
|
||||
Kind: controller.InputWeak,
|
||||
},
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Outputs implements controller.Controller interface.
|
||||
@ -59,52 +59,24 @@ func (ctrl *KmsgLogDeliveryController) Outputs() []controller.Output {
|
||||
}
|
||||
|
||||
// Run implements controller.Controller interface.
|
||||
//
|
||||
//nolint:gocyclo,cyclop
|
||||
func (ctrl *KmsgLogDeliveryController) Run(ctx context.Context, r controller.Runtime, logger *zap.Logger) (err error) {
|
||||
if ctrl.Cmdline == nil || ctrl.Cmdline.Get(constants.KernelParamLoggingKernel).First() == nil {
|
||||
return nil
|
||||
func (ctrl *KmsgLogDeliveryController) Run(ctx context.Context, r controller.Runtime, logger *zap.Logger) error {
|
||||
if err := networkutils.WaitForNetworkReady(ctx, r,
|
||||
func(status *network.StatusSpec) bool {
|
||||
return status.AddressReady
|
||||
},
|
||||
[]controller.Input{
|
||||
{
|
||||
Namespace: runtime.NamespaceName,
|
||||
Type: runtime.KmsgLogConfigType,
|
||||
ID: pointer.To(runtime.KmsgLogConfigID),
|
||||
Kind: controller.InputWeak,
|
||||
},
|
||||
},
|
||||
); err != nil {
|
||||
return fmt.Errorf("error waiting for network: %w", err)
|
||||
}
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return nil
|
||||
case <-r.EventCh():
|
||||
}
|
||||
|
||||
var netStatus resource.Resource
|
||||
|
||||
netStatus, err = r.Get(ctx, resource.NewMetadata(network.NamespaceName, network.StatusType, network.StatusID, resource.VersionUndefined))
|
||||
if err != nil {
|
||||
if state.IsNotFoundError(err) {
|
||||
// no network state yet
|
||||
continue
|
||||
}
|
||||
|
||||
return fmt.Errorf("error reading network status: %w", err)
|
||||
}
|
||||
|
||||
if !netStatus.(*network.Status).TypedSpec().AddressReady {
|
||||
// wait for address
|
||||
continue
|
||||
}
|
||||
|
||||
break
|
||||
}
|
||||
|
||||
if ctrl.drainSub == nil {
|
||||
ctrl.drainSub = ctrl.Drainer.Subscribe()
|
||||
}
|
||||
|
||||
destURL, err := url.Parse(*ctrl.Cmdline.Get(constants.KernelParamLoggingKernel).First())
|
||||
if err != nil {
|
||||
return fmt.Errorf("error parsing %q: %w", constants.KernelParamLoggingKernel, err)
|
||||
}
|
||||
|
||||
sender := logging.NewJSONLines(destURL)
|
||||
defer sender.Close(ctx) //nolint:errcheck
|
||||
|
||||
// initilalize kmsg reader early, so that we don't lose position on config changes
|
||||
reader, err := kmsg.NewReader(kmsg.Follow())
|
||||
if err != nil {
|
||||
return fmt.Errorf("error reading kernel messages: %w", err)
|
||||
@ -114,6 +86,49 @@ func (ctrl *KmsgLogDeliveryController) Run(ctx context.Context, r controller.Run
|
||||
|
||||
kmsgCh := reader.Scan(ctx)
|
||||
|
||||
for {
|
||||
if _, ok := channel.RecvWithContext(ctx, r.EventCh()); !ok {
|
||||
return nil
|
||||
}
|
||||
|
||||
cfg, err := safe.ReaderGetByID[*runtime.KmsgLogConfig](ctx, r, runtime.KmsgLogConfigID)
|
||||
if err != nil && !state.IsNotFoundError(err) {
|
||||
return fmt.Errorf("error getting configuration: %w", err)
|
||||
}
|
||||
|
||||
if cfg == nil {
|
||||
// no config, wait for the next event
|
||||
continue
|
||||
}
|
||||
|
||||
if err = ctrl.deliverLogs(ctx, r, logger, kmsgCh, cfg.TypedSpec().Destinations); err != nil {
|
||||
return fmt.Errorf("error delivering logs: %w", err)
|
||||
}
|
||||
|
||||
r.ResetRestartBackoff()
|
||||
}
|
||||
}
|
||||
|
||||
//nolint:gocyclo
|
||||
func (ctrl *KmsgLogDeliveryController) deliverLogs(ctx context.Context, r controller.Runtime, logger *zap.Logger, kmsgCh <-chan kmsg.Packet, destURLs []*url.URL) error {
|
||||
if ctrl.drainSub == nil {
|
||||
ctrl.drainSub = ctrl.Drainer.Subscribe()
|
||||
}
|
||||
|
||||
// initialize all log senders
|
||||
senders := slices.Map(destURLs, logging.NewJSONLines)
|
||||
|
||||
defer func() {
|
||||
closeCtx, closeCtxCancel := context.WithTimeout(context.Background(), logCloseTimeout)
|
||||
defer closeCtxCancel()
|
||||
|
||||
for _, sender := range senders {
|
||||
if err := sender.Close(closeCtx); err != nil {
|
||||
logger.Error("error closing log sender", zap.Error(err))
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
var (
|
||||
drainTimer *time.Timer
|
||||
drainTimerCh <-chan time.Time
|
||||
@ -126,6 +141,19 @@ func (ctrl *KmsgLogDeliveryController) Run(ctx context.Context, r controller.Run
|
||||
case <-ctx.Done():
|
||||
ctrl.drainSub.Cancel()
|
||||
|
||||
return nil
|
||||
case <-r.EventCh():
|
||||
// config changed, restart the loop
|
||||
return nil
|
||||
case <-ctrl.drainSub.EventCh():
|
||||
// drain started, assume that ksmg is drained if there're no new messages in drainTimeout
|
||||
drainTimer = time.NewTimer(drainTimeout)
|
||||
drainTimerCh = drainTimer.C
|
||||
|
||||
continue
|
||||
case <-drainTimerCh:
|
||||
ctrl.drainSub.Cancel()
|
||||
|
||||
return nil
|
||||
case msg = <-kmsgCh:
|
||||
if drainTimer != nil {
|
||||
@ -136,21 +164,13 @@ func (ctrl *KmsgLogDeliveryController) Run(ctx context.Context, r controller.Run
|
||||
|
||||
drainTimer.Reset(drainTimeout)
|
||||
}
|
||||
case <-ctrl.drainSub.EventCh():
|
||||
// drain started, assume that ksmg is drained if there're no new messages in drainTimeout
|
||||
drainTimer = time.NewTimer(drainTimeout)
|
||||
drainTimerCh = drainTimer.C
|
||||
case <-drainTimerCh:
|
||||
ctrl.drainSub.Cancel()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
if msg.Err != nil {
|
||||
return fmt.Errorf("error receiving kernel logs: %w", msg.Err)
|
||||
}
|
||||
|
||||
if err = sender.Send(ctx, &runtime.LogEvent{
|
||||
event := machinedruntime.LogEvent{
|
||||
Msg: msg.Message.Message,
|
||||
Time: msg.Message.Timestamp,
|
||||
Level: kmsgPriorityToLevel(msg.Message.Priority),
|
||||
@ -160,11 +180,61 @@ func (ctrl *KmsgLogDeliveryController) Run(ctx context.Context, r controller.Run
|
||||
"clock": msg.Message.Clock,
|
||||
"priority": msg.Message.Priority.String(),
|
||||
},
|
||||
}); err != nil {
|
||||
return fmt.Errorf("error sending logs: %w", err)
|
||||
}
|
||||
|
||||
r.ResetRestartBackoff()
|
||||
if err := ctrl.resend(ctx, r, logger, senders, &event); err != nil {
|
||||
return fmt.Errorf("error sending log event: %w", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//nolint:gocyclo
|
||||
func (ctrl *KmsgLogDeliveryController) resend(ctx context.Context, r controller.Runtime, logger *zap.Logger, senders []machinedruntime.LogSender, e *machinedruntime.LogEvent) error {
|
||||
for {
|
||||
sendCtx, sendCancel := context.WithTimeout(ctx, logSendTimeout)
|
||||
sendErrors := make(chan error, len(senders))
|
||||
|
||||
for _, sender := range senders {
|
||||
sender := sender
|
||||
|
||||
go func() {
|
||||
sendErrors <- sender.Send(sendCtx, e)
|
||||
}()
|
||||
}
|
||||
|
||||
var dontRetry bool
|
||||
|
||||
for range senders {
|
||||
err := <-sendErrors
|
||||
|
||||
// don't retry if at least one sender succeed to avoid implementing per-sender queue, etc
|
||||
if err == nil {
|
||||
dontRetry = true
|
||||
|
||||
continue
|
||||
}
|
||||
|
||||
logger.Debug("error sending log event", zap.Error(err))
|
||||
|
||||
if errors.Is(err, machinedruntime.ErrDontRetry) || errors.Is(err, context.Canceled) {
|
||||
dontRetry = true
|
||||
}
|
||||
}
|
||||
|
||||
sendCancel()
|
||||
|
||||
if dontRetry {
|
||||
return nil
|
||||
}
|
||||
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return nil
|
||||
case <-r.EventCh():
|
||||
// config changed, restart the loop
|
||||
return fmt.Errorf("config changed")
|
||||
case <-time.After(logRetryTimeout):
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -0,0 +1,87 @@
|
||||
// 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 runtime
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"net/url"
|
||||
|
||||
"github.com/cosi-project/runtime/pkg/controller"
|
||||
"github.com/cosi-project/runtime/pkg/safe"
|
||||
"github.com/cosi-project/runtime/pkg/state"
|
||||
"github.com/siderolabs/go-procfs/procfs"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/siderolabs/talos/pkg/machinery/constants"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/runtime"
|
||||
)
|
||||
|
||||
// KmsgLogConfigController generates configuration for kmsg log delivery.
|
||||
type KmsgLogConfigController struct {
|
||||
Cmdline *procfs.Cmdline
|
||||
}
|
||||
|
||||
// Name implements controller.Controller interface.
|
||||
func (ctrl *KmsgLogConfigController) Name() string {
|
||||
return "runtime.KmsgLogConfigController"
|
||||
}
|
||||
|
||||
// Inputs implements controller.Controller interface.
|
||||
func (ctrl *KmsgLogConfigController) Inputs() []controller.Input {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Outputs implements controller.Controller interface.
|
||||
func (ctrl *KmsgLogConfigController) Outputs() []controller.Output {
|
||||
return []controller.Output{
|
||||
{
|
||||
Type: runtime.KmsgLogConfigType,
|
||||
Kind: controller.OutputExclusive,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// Run implements controller.Controller interface.
|
||||
//
|
||||
//nolint:gocyclo,cyclop
|
||||
func (ctrl *KmsgLogConfigController) Run(ctx context.Context, r controller.Runtime, logger *zap.Logger) (err error) {
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return nil
|
||||
case <-r.EventCh():
|
||||
}
|
||||
|
||||
destinations := []*url.URL{}
|
||||
|
||||
if ctrl.Cmdline != nil {
|
||||
if val := ctrl.Cmdline.Get(constants.KernelParamLoggingKernel).First(); val != nil {
|
||||
destURL, err := url.Parse(*val)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error parsing %q: %w", constants.KernelParamLoggingKernel, err)
|
||||
}
|
||||
|
||||
destinations = append(destinations, destURL)
|
||||
}
|
||||
}
|
||||
|
||||
if len(destinations) == 0 {
|
||||
if err := r.Destroy(ctx, runtime.NewKmsgLogConfig().Metadata()); err != nil && !state.IsNotFoundError(err) {
|
||||
return fmt.Errorf("error destroying kmsg log config: %w", err)
|
||||
}
|
||||
} else {
|
||||
if err := safe.WriterModify(ctx, r, runtime.NewKmsgLogConfig(), func(cfg *runtime.KmsgLogConfig) error {
|
||||
cfg.TypedSpec().Destinations = destinations
|
||||
|
||||
return nil
|
||||
}); err != nil {
|
||||
return fmt.Errorf("error updating kmsg log config: %w", err)
|
||||
}
|
||||
}
|
||||
|
||||
r.ResetRestartBackoff()
|
||||
}
|
||||
}
|
@ -0,0 +1,53 @@
|
||||
// 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 runtime_test
|
||||
|
||||
import (
|
||||
"net/url"
|
||||
"testing"
|
||||
|
||||
"github.com/cosi-project/runtime/pkg/resource"
|
||||
"github.com/cosi-project/runtime/pkg/resource/rtestutils"
|
||||
"github.com/siderolabs/gen/slices"
|
||||
"github.com/siderolabs/go-procfs/procfs"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/suite"
|
||||
|
||||
"github.com/siderolabs/talos/internal/app/machined/pkg/controllers/ctest"
|
||||
runtimectrls "github.com/siderolabs/talos/internal/app/machined/pkg/controllers/runtime"
|
||||
"github.com/siderolabs/talos/pkg/machinery/constants"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/runtime"
|
||||
)
|
||||
|
||||
type KmsgLogConfigSuite struct {
|
||||
ctest.DefaultSuite
|
||||
}
|
||||
|
||||
func TestKmsgLogConfigSuite(t *testing.T) {
|
||||
suite.Run(t, new(KmsgLogConfigSuite))
|
||||
}
|
||||
|
||||
func (suite *KmsgLogConfigSuite) TestKmsgLogConfigNone() {
|
||||
suite.Require().NoError(suite.Runtime().RegisterController(&runtimectrls.KmsgLogConfigController{}))
|
||||
|
||||
rtestutils.AssertNoResource[*runtime.KmsgLogConfig](suite.Ctx(), suite.T(), suite.State(), runtime.KmsgLogConfigID)
|
||||
}
|
||||
|
||||
func (suite *KmsgLogConfigSuite) TestKmsgLogConfigCmdline() {
|
||||
cmdline := procfs.NewCmdline("")
|
||||
cmdline.Append(constants.KernelParamLoggingKernel, "https://10.0.0.1:3333/logs")
|
||||
|
||||
suite.Require().NoError(suite.Runtime().RegisterController(&runtimectrls.KmsgLogConfigController{
|
||||
Cmdline: cmdline,
|
||||
}))
|
||||
|
||||
rtestutils.AssertResources[*runtime.KmsgLogConfig](suite.Ctx(), suite.T(), suite.State(), []resource.ID{runtime.KmsgLogConfigID},
|
||||
func(cfg *runtime.KmsgLogConfig, asrt *assert.Assertions) {
|
||||
asrt.Equal(
|
||||
[]string{"https://10.0.0.1:3333/logs"},
|
||||
slices.Map(cfg.TypedSpec().Destinations, func(u *url.URL) string { return u.String() }),
|
||||
)
|
||||
})
|
||||
}
|
@ -6,10 +6,9 @@ package runtime_test
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"log"
|
||||
"net"
|
||||
"net/netip"
|
||||
"net/url"
|
||||
"os"
|
||||
"sync"
|
||||
"testing"
|
||||
@ -19,16 +18,15 @@ import (
|
||||
"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-procfs/procfs"
|
||||
"github.com/siderolabs/go-retry/retry"
|
||||
"github.com/siderolabs/siderolink/pkg/logreceiver"
|
||||
"github.com/stretchr/testify/suite"
|
||||
"go.uber.org/zap/zaptest"
|
||||
|
||||
controllerruntime "github.com/siderolabs/talos/internal/app/machined/pkg/controllers/runtime"
|
||||
runtimectrl "github.com/siderolabs/talos/internal/app/machined/pkg/controllers/runtime"
|
||||
talosruntime "github.com/siderolabs/talos/internal/app/machined/pkg/runtime"
|
||||
"github.com/siderolabs/talos/pkg/logging"
|
||||
"github.com/siderolabs/talos/pkg/machinery/constants"
|
||||
"github.com/siderolabs/talos/pkg/machinery/resources/network"
|
||||
runtimeres "github.com/siderolabs/talos/pkg/machinery/resources/runtime"
|
||||
)
|
||||
|
||||
type logHandler struct {
|
||||
@ -54,8 +52,7 @@ func (s *logHandler) getCount() int {
|
||||
type KmsgLogDeliverySuite struct {
|
||||
suite.Suite
|
||||
|
||||
state state.State
|
||||
cmdline *procfs.Cmdline
|
||||
state state.State
|
||||
|
||||
runtime *runtime.Runtime
|
||||
drainer *talosruntime.Drainer
|
||||
@ -64,48 +61,52 @@ type KmsgLogDeliverySuite struct {
|
||||
ctx context.Context //nolint:containedctx
|
||||
ctxCancel context.CancelFunc
|
||||
|
||||
handler *logHandler
|
||||
handler1, handler2 *logHandler
|
||||
|
||||
srv *logreceiver.Server
|
||||
listener1, listener2 net.Listener
|
||||
srv1, srv2 *logreceiver.Server
|
||||
}
|
||||
|
||||
func (suite *KmsgLogDeliverySuite) SetupTest() {
|
||||
suite.ctx, suite.ctxCancel = context.WithTimeout(context.Background(), 3*time.Minute)
|
||||
suite.ctx, suite.ctxCancel = context.WithTimeout(context.Background(), 10*time.Second)
|
||||
|
||||
suite.state = state.WrapCore(namespaced.NewState(inmem.Build))
|
||||
|
||||
logger := logging.Wrap(log.Writer())
|
||||
logger := zaptest.NewLogger(suite.T())
|
||||
|
||||
var err error
|
||||
|
||||
suite.runtime, err = runtime.NewRuntime(suite.state, logger)
|
||||
suite.Require().NoError(err)
|
||||
|
||||
suite.handler = &logHandler{}
|
||||
suite.handler1 = &logHandler{}
|
||||
suite.handler2 = &logHandler{}
|
||||
|
||||
listener, err := net.Listen("tcp", "localhost:0")
|
||||
suite.listener1, err = net.Listen("tcp", "localhost:0")
|
||||
suite.Require().NoError(err)
|
||||
|
||||
suite.srv, err = logreceiver.NewServer(logger, listener, suite.handler.HandleLog)
|
||||
suite.listener2, err = net.Listen("tcp", "localhost:0")
|
||||
suite.Require().NoError(err)
|
||||
|
||||
suite.srv1, err = logreceiver.NewServer(logger, suite.listener1, suite.handler1.HandleLog)
|
||||
suite.Require().NoError(err)
|
||||
|
||||
suite.srv2, err = logreceiver.NewServer(logger, suite.listener2, suite.handler2.HandleLog)
|
||||
suite.Require().NoError(err)
|
||||
|
||||
go func() {
|
||||
suite.srv.Serve() //nolint:errcheck
|
||||
suite.srv1.Serve() //nolint:errcheck
|
||||
}()
|
||||
|
||||
go func() {
|
||||
suite.srv2.Serve() //nolint:errcheck
|
||||
}()
|
||||
|
||||
suite.cmdline = procfs.NewCmdline(
|
||||
fmt.Sprintf(
|
||||
"%s=%s",
|
||||
constants.KernelParamLoggingKernel,
|
||||
fmt.Sprintf("tcp://%s", listener.Addr()),
|
||||
),
|
||||
)
|
||||
suite.drainer = talosruntime.NewDrainer()
|
||||
|
||||
suite.Require().NoError(
|
||||
suite.runtime.RegisterController(
|
||||
&controllerruntime.KmsgLogDeliveryController{
|
||||
Cmdline: suite.cmdline,
|
||||
&runtimectrl.KmsgLogDeliveryController{
|
||||
Drainer: suite.drainer,
|
||||
},
|
||||
),
|
||||
@ -127,29 +128,115 @@ func (suite *KmsgLogDeliverySuite) startRuntime() {
|
||||
}()
|
||||
}
|
||||
|
||||
func (suite *KmsgLogDeliverySuite) TestDelivery() {
|
||||
func (suite *KmsgLogDeliverySuite) TestDeliverySingleDestination() {
|
||||
suite.startRuntime()
|
||||
|
||||
// controller should deliver some kernel logs from host's kmsg buffer
|
||||
err := retry.Constant(time.Second*5, retry.WithUnits(time.Millisecond*100)).Retry(
|
||||
func() error {
|
||||
if suite.handler.getCount() == 0 {
|
||||
return retry.ExpectedErrorf("no logs received")
|
||||
}
|
||||
|
||||
return nil
|
||||
kmsgLogConfig := runtimeres.NewKmsgLogConfig()
|
||||
kmsgLogConfig.TypedSpec().Destinations = []*url.URL{
|
||||
{
|
||||
Scheme: "tcp",
|
||||
Host: suite.listener1.Addr().String(),
|
||||
},
|
||||
)
|
||||
suite.Require().NoError(err)
|
||||
}
|
||||
|
||||
suite.Require().NoError(suite.state.Create(suite.ctx, kmsgLogConfig))
|
||||
|
||||
// controller should deliver some kernel logs from host's kmsg buffer
|
||||
suite.assertLogsSeen(suite.handler1)
|
||||
}
|
||||
|
||||
func (suite *KmsgLogDeliverySuite) TestDeliveryMultipleDestinations() {
|
||||
suite.startRuntime()
|
||||
|
||||
kmsgLogConfig := runtimeres.NewKmsgLogConfig()
|
||||
kmsgLogConfig.TypedSpec().Destinations = []*url.URL{
|
||||
{
|
||||
Scheme: "tcp",
|
||||
Host: suite.listener1.Addr().String(),
|
||||
},
|
||||
{
|
||||
Scheme: "tcp",
|
||||
Host: suite.listener2.Addr().String(),
|
||||
},
|
||||
}
|
||||
|
||||
suite.Require().NoError(suite.state.Create(suite.ctx, kmsgLogConfig))
|
||||
|
||||
// controller should deliver logs to both destinations
|
||||
suite.assertLogsSeen(suite.handler1)
|
||||
suite.assertLogsSeen(suite.handler2)
|
||||
}
|
||||
|
||||
func (suite *KmsgLogDeliverySuite) TestDeliveryOneDeadDestination() {
|
||||
suite.startRuntime()
|
||||
|
||||
// stop one listener
|
||||
suite.Require().NoError(suite.listener1.Close())
|
||||
|
||||
kmsgLogConfig := runtimeres.NewKmsgLogConfig()
|
||||
kmsgLogConfig.TypedSpec().Destinations = []*url.URL{
|
||||
{
|
||||
Scheme: "tcp",
|
||||
Host: suite.listener1.Addr().String(),
|
||||
},
|
||||
{
|
||||
Scheme: "tcp",
|
||||
Host: suite.listener2.Addr().String(),
|
||||
},
|
||||
}
|
||||
|
||||
suite.Require().NoError(suite.state.Create(suite.ctx, kmsgLogConfig))
|
||||
|
||||
// controller should deliver logs to live destination
|
||||
suite.assertLogsSeen(suite.handler2)
|
||||
}
|
||||
|
||||
func (suite *KmsgLogDeliverySuite) TestDeliveryAllDeadDestinations() {
|
||||
suite.startRuntime()
|
||||
|
||||
// stop all listeners
|
||||
suite.Require().NoError(suite.listener1.Close())
|
||||
suite.Require().NoError(suite.listener2.Close())
|
||||
|
||||
kmsgLogConfig := runtimeres.NewKmsgLogConfig()
|
||||
kmsgLogConfig.TypedSpec().Destinations = []*url.URL{
|
||||
{
|
||||
Scheme: "tcp",
|
||||
Host: suite.listener1.Addr().String(),
|
||||
},
|
||||
{
|
||||
Scheme: "tcp",
|
||||
Host: suite.listener2.Addr().String(),
|
||||
},
|
||||
}
|
||||
|
||||
suite.Require().NoError(suite.state.Create(suite.ctx, kmsgLogConfig))
|
||||
}
|
||||
|
||||
func (suite *KmsgLogDeliverySuite) TestDrain() {
|
||||
suite.startRuntime()
|
||||
|
||||
kmsgLogConfig := runtimeres.NewKmsgLogConfig()
|
||||
kmsgLogConfig.TypedSpec().Destinations = []*url.URL{
|
||||
{
|
||||
Scheme: "tcp",
|
||||
Host: suite.listener1.Addr().String(),
|
||||
},
|
||||
}
|
||||
|
||||
suite.Require().NoError(suite.state.Create(suite.ctx, kmsgLogConfig))
|
||||
|
||||
// wait for controller to start delivering some logs
|
||||
suite.assertLogsSeen(suite.handler1)
|
||||
|
||||
// drain should be successful, i.e. controller should stop on its own before context is canceled
|
||||
suite.Assert().NoError(suite.drainer.Drain(suite.ctx))
|
||||
}
|
||||
|
||||
func (suite *KmsgLogDeliverySuite) assertLogsSeen(handler *logHandler) {
|
||||
err := retry.Constant(time.Second*5, retry.WithUnits(time.Millisecond*100)).Retry(
|
||||
func() error {
|
||||
if suite.handler.getCount() == 0 {
|
||||
if handler.getCount() == 0 {
|
||||
return retry.ExpectedErrorf("no logs received")
|
||||
}
|
||||
|
||||
@ -157,15 +244,11 @@ func (suite *KmsgLogDeliverySuite) TestDrain() {
|
||||
},
|
||||
)
|
||||
suite.Require().NoError(err)
|
||||
|
||||
// drain should be successful, i.e. controller should stop on its own before context is canceled
|
||||
suite.Assert().NoError(suite.drainer.Drain(suite.ctx))
|
||||
}
|
||||
|
||||
func (suite *KmsgLogDeliverySuite) TearDownTest() {
|
||||
suite.T().Log("tear down")
|
||||
|
||||
suite.srv.Stop()
|
||||
suite.srv1.Stop()
|
||||
suite.srv2.Stop()
|
||||
|
||||
suite.ctxCancel()
|
||||
|
||||
|
@ -236,8 +236,10 @@ func (ctrl *Controller) Run(ctx context.Context, drainer *runtime.Drainer) error
|
||||
V1Alpha1Mode: ctrl.v1alpha1Runtime.State().Platform().Mode(),
|
||||
},
|
||||
&runtimecontrollers.KernelParamSpecController{},
|
||||
&runtimecontrollers.KmsgLogDeliveryController{
|
||||
&runtimecontrollers.KmsgLogConfigController{
|
||||
Cmdline: procfs.ProcCmdline(),
|
||||
},
|
||||
&runtimecontrollers.KmsgLogDeliveryController{
|
||||
Drainer: drainer,
|
||||
},
|
||||
&runtimecontrollers.MachineStatusController{
|
||||
|
@ -170,6 +170,7 @@ func NewState() (*State, error) {
|
||||
&runtime.KernelParamSpec{},
|
||||
&runtime.KernelParamDefaultSpec{},
|
||||
&runtime.KernelParamStatus{},
|
||||
&runtime.KmsgLogConfig{},
|
||||
&runtime.MachineStatus{},
|
||||
&runtime.MetaKey{},
|
||||
&runtime.MountStatus{},
|
||||
|
@ -13,6 +13,7 @@ import (
|
||||
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
||||
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
||||
|
||||
common "github.com/siderolabs/talos/pkg/machinery/api/common"
|
||||
enums "github.com/siderolabs/talos/pkg/machinery/api/resource/definitions/enums"
|
||||
)
|
||||
|
||||
@ -247,6 +248,54 @@ func (x *KernelParamStatusSpec) GetUnsupported() bool {
|
||||
return false
|
||||
}
|
||||
|
||||
// KmsgLogConfigSpec describes status of the defined sysctls.
|
||||
type KmsgLogConfigSpec struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Destinations []*common.URL `protobuf:"bytes,1,rep,name=destinations,proto3" json:"destinations,omitempty"`
|
||||
}
|
||||
|
||||
func (x *KmsgLogConfigSpec) Reset() {
|
||||
*x = KmsgLogConfigSpec{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[4]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *KmsgLogConfigSpec) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*KmsgLogConfigSpec) ProtoMessage() {}
|
||||
|
||||
func (x *KmsgLogConfigSpec) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[4]
|
||||
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 KmsgLogConfigSpec.ProtoReflect.Descriptor instead.
|
||||
func (*KmsgLogConfigSpec) Descriptor() ([]byte, []int) {
|
||||
return file_resource_definitions_runtime_runtime_proto_rawDescGZIP(), []int{4}
|
||||
}
|
||||
|
||||
func (x *KmsgLogConfigSpec) GetDestinations() []*common.URL {
|
||||
if x != nil {
|
||||
return x.Destinations
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// MachineStatusSpec describes status of the defined sysctls.
|
||||
type MachineStatusSpec struct {
|
||||
state protoimpl.MessageState
|
||||
@ -260,7 +309,7 @@ type MachineStatusSpec struct {
|
||||
func (x *MachineStatusSpec) Reset() {
|
||||
*x = MachineStatusSpec{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[4]
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[5]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@ -273,7 +322,7 @@ func (x *MachineStatusSpec) String() string {
|
||||
func (*MachineStatusSpec) ProtoMessage() {}
|
||||
|
||||
func (x *MachineStatusSpec) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[4]
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[5]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@ -286,7 +335,7 @@ func (x *MachineStatusSpec) ProtoReflect() protoreflect.Message {
|
||||
|
||||
// Deprecated: Use MachineStatusSpec.ProtoReflect.Descriptor instead.
|
||||
func (*MachineStatusSpec) Descriptor() ([]byte, []int) {
|
||||
return file_resource_definitions_runtime_runtime_proto_rawDescGZIP(), []int{4}
|
||||
return file_resource_definitions_runtime_runtime_proto_rawDescGZIP(), []int{5}
|
||||
}
|
||||
|
||||
func (x *MachineStatusSpec) GetStage() enums.RuntimeMachineStage {
|
||||
@ -316,7 +365,7 @@ type MachineStatusStatus struct {
|
||||
func (x *MachineStatusStatus) Reset() {
|
||||
*x = MachineStatusStatus{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[5]
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[6]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@ -329,7 +378,7 @@ func (x *MachineStatusStatus) String() string {
|
||||
func (*MachineStatusStatus) ProtoMessage() {}
|
||||
|
||||
func (x *MachineStatusStatus) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[5]
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[6]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@ -342,7 +391,7 @@ func (x *MachineStatusStatus) ProtoReflect() protoreflect.Message {
|
||||
|
||||
// Deprecated: Use MachineStatusStatus.ProtoReflect.Descriptor instead.
|
||||
func (*MachineStatusStatus) Descriptor() ([]byte, []int) {
|
||||
return file_resource_definitions_runtime_runtime_proto_rawDescGZIP(), []int{5}
|
||||
return file_resource_definitions_runtime_runtime_proto_rawDescGZIP(), []int{6}
|
||||
}
|
||||
|
||||
func (x *MachineStatusStatus) GetReady() bool {
|
||||
@ -371,7 +420,7 @@ type MetaKeySpec struct {
|
||||
func (x *MetaKeySpec) Reset() {
|
||||
*x = MetaKeySpec{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[6]
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[7]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@ -384,7 +433,7 @@ func (x *MetaKeySpec) String() string {
|
||||
func (*MetaKeySpec) ProtoMessage() {}
|
||||
|
||||
func (x *MetaKeySpec) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[6]
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[7]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@ -397,7 +446,7 @@ func (x *MetaKeySpec) ProtoReflect() protoreflect.Message {
|
||||
|
||||
// Deprecated: Use MetaKeySpec.ProtoReflect.Descriptor instead.
|
||||
func (*MetaKeySpec) Descriptor() ([]byte, []int) {
|
||||
return file_resource_definitions_runtime_runtime_proto_rawDescGZIP(), []int{6}
|
||||
return file_resource_definitions_runtime_runtime_proto_rawDescGZIP(), []int{7}
|
||||
}
|
||||
|
||||
func (x *MetaKeySpec) GetValue() string {
|
||||
@ -422,7 +471,7 @@ type MountStatusSpec struct {
|
||||
func (x *MountStatusSpec) Reset() {
|
||||
*x = MountStatusSpec{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[7]
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[8]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@ -435,7 +484,7 @@ func (x *MountStatusSpec) String() string {
|
||||
func (*MountStatusSpec) ProtoMessage() {}
|
||||
|
||||
func (x *MountStatusSpec) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[7]
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[8]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@ -448,7 +497,7 @@ func (x *MountStatusSpec) ProtoReflect() protoreflect.Message {
|
||||
|
||||
// Deprecated: Use MountStatusSpec.ProtoReflect.Descriptor instead.
|
||||
func (*MountStatusSpec) Descriptor() ([]byte, []int) {
|
||||
return file_resource_definitions_runtime_runtime_proto_rawDescGZIP(), []int{7}
|
||||
return file_resource_definitions_runtime_runtime_proto_rawDescGZIP(), []int{8}
|
||||
}
|
||||
|
||||
func (x *MountStatusSpec) GetSource() string {
|
||||
@ -498,7 +547,7 @@ type PlatformMetadataSpec struct {
|
||||
func (x *PlatformMetadataSpec) Reset() {
|
||||
*x = PlatformMetadataSpec{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[8]
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[9]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@ -511,7 +560,7 @@ func (x *PlatformMetadataSpec) String() string {
|
||||
func (*PlatformMetadataSpec) ProtoMessage() {}
|
||||
|
||||
func (x *PlatformMetadataSpec) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[8]
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[9]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@ -524,7 +573,7 @@ func (x *PlatformMetadataSpec) ProtoReflect() protoreflect.Message {
|
||||
|
||||
// Deprecated: Use PlatformMetadataSpec.ProtoReflect.Descriptor instead.
|
||||
func (*PlatformMetadataSpec) Descriptor() ([]byte, []int) {
|
||||
return file_resource_definitions_runtime_runtime_proto_rawDescGZIP(), []int{8}
|
||||
return file_resource_definitions_runtime_runtime_proto_rawDescGZIP(), []int{9}
|
||||
}
|
||||
|
||||
func (x *PlatformMetadataSpec) GetPlatform() string {
|
||||
@ -596,7 +645,7 @@ type UnmetCondition struct {
|
||||
func (x *UnmetCondition) Reset() {
|
||||
*x = UnmetCondition{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[9]
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[10]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@ -609,7 +658,7 @@ func (x *UnmetCondition) String() string {
|
||||
func (*UnmetCondition) ProtoMessage() {}
|
||||
|
||||
func (x *UnmetCondition) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[9]
|
||||
mi := &file_resource_definitions_runtime_runtime_proto_msgTypes[10]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@ -622,7 +671,7 @@ func (x *UnmetCondition) ProtoReflect() protoreflect.Message {
|
||||
|
||||
// Deprecated: Use UnmetCondition.ProtoReflect.Descriptor instead.
|
||||
func (*UnmetCondition) Descriptor() ([]byte, []int) {
|
||||
return file_resource_definitions_runtime_runtime_proto_rawDescGZIP(), []int{9}
|
||||
return file_resource_definitions_runtime_runtime_proto_rawDescGZIP(), []int{10}
|
||||
}
|
||||
|
||||
func (x *UnmetCondition) GetName() string {
|
||||
@ -647,84 +696,90 @@ var file_resource_definitions_runtime_runtime_proto_rawDesc = []byte{
|
||||
0x75, 0x6e, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x22, 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, 0x72, 0x75, 0x6e, 0x74, 0x69, 0x6d, 0x65,
|
||||
0x1a, 0x26, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2f, 0x64, 0x65, 0x66, 0x69, 0x6e,
|
||||
0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2f, 0x65, 0x6e, 0x75,
|
||||
0x6d, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x29, 0x0a, 0x11, 0x44, 0x65, 0x76, 0x69,
|
||||
0x63, 0x65, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, 0x70, 0x65, 0x63, 0x12, 0x14, 0x0a,
|
||||
0x05, 0x72, 0x65, 0x61, 0x64, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x72, 0x65,
|
||||
0x61, 0x64, 0x79, 0x22, 0x4a, 0x0a, 0x14, 0x4b, 0x65, 0x72, 0x6e, 0x65, 0x6c, 0x4d, 0x6f, 0x64,
|
||||
0x75, 0x6c, 0x65, 0x53, 0x70, 0x65, 0x63, 0x53, 0x70, 0x65, 0x63, 0x12, 0x12, 0x0a, 0x04, 0x6e,
|
||||
0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12,
|
||||
0x1e, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, 0x18, 0x02, 0x20,
|
||||
0x03, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, 0x22,
|
||||
0x50, 0x0a, 0x13, 0x4b, 0x65, 0x72, 0x6e, 0x65, 0x6c, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x53, 0x70,
|
||||
0x65, 0x63, 0x53, 0x70, 0x65, 0x63, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18,
|
||||
0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x23, 0x0a, 0x0d,
|
||||
0x69, 0x67, 0x6e, 0x6f, 0x72, 0x65, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x73, 0x18, 0x02, 0x20,
|
||||
0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x67, 0x6e, 0x6f, 0x72, 0x65, 0x45, 0x72, 0x72, 0x6f, 0x72,
|
||||
0x73, 0x22, 0x6d, 0x0a, 0x15, 0x4b, 0x65, 0x72, 0x6e, 0x65, 0x6c, 0x50, 0x61, 0x72, 0x61, 0x6d,
|
||||
0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, 0x70, 0x65, 0x63, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x75,
|
||||
0x72, 0x72, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x75, 0x72,
|
||||
0x72, 0x65, 0x6e, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x18,
|
||||
0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x12, 0x20,
|
||||
0x0a, 0x0b, 0x75, 0x6e, 0x73, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x18, 0x03, 0x20,
|
||||
0x01, 0x28, 0x08, 0x52, 0x0b, 0x75, 0x6e, 0x73, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64,
|
||||
0x22, 0xb1, 0x01, 0x0a, 0x11, 0x4d, 0x61, 0x63, 0x68, 0x69, 0x6e, 0x65, 0x53, 0x74, 0x61, 0x74,
|
||||
0x75, 0x73, 0x53, 0x70, 0x65, 0x63, 0x12, 0x4b, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x67, 0x65, 0x18,
|
||||
0x01, 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, 0x52, 0x75, 0x6e, 0x74, 0x69, 0x6d, 0x65,
|
||||
0x4d, 0x61, 0x63, 0x68, 0x69, 0x6e, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x52, 0x05, 0x73, 0x74,
|
||||
0x61, 0x67, 0x65, 0x12, 0x4f, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20,
|
||||
0x01, 0x28, 0x0b, 0x32, 0x37, 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, 0x72, 0x75, 0x6e, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x4d, 0x61, 0x63, 0x68, 0x69, 0x6e, 0x65,
|
||||
0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74,
|
||||
0x61, 0x74, 0x75, 0x73, 0x22, 0x8a, 0x01, 0x0a, 0x13, 0x4d, 0x61, 0x63, 0x68, 0x69, 0x6e, 0x65,
|
||||
0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x14, 0x0a, 0x05,
|
||||
0x72, 0x65, 0x61, 0x64, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x72, 0x65, 0x61,
|
||||
0x64, 0x79, 0x12, 0x5d, 0x0a, 0x10, 0x75, 0x6e, 0x6d, 0x65, 0x74, 0x5f, 0x63, 0x6f, 0x6e, 0x64,
|
||||
0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 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, 0x72, 0x75, 0x6e, 0x74, 0x69, 0x6d,
|
||||
0x65, 0x2e, 0x55, 0x6e, 0x6d, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e,
|
||||
0x52, 0x0f, 0x75, 0x6e, 0x6d, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e,
|
||||
0x73, 0x22, 0x23, 0x0a, 0x0b, 0x4d, 0x65, 0x74, 0x61, 0x4b, 0x65, 0x79, 0x53, 0x70, 0x65, 0x63,
|
||||
0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0x84, 0x01, 0x0a, 0x0f, 0x4d, 0x6f, 0x75, 0x6e, 0x74,
|
||||
0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, 0x70, 0x65, 0x63, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x6f,
|
||||
0x75, 0x72, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72,
|
||||
0x63, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01,
|
||||
0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x27, 0x0a, 0x0f, 0x66, 0x69,
|
||||
0x6c, 0x65, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20,
|
||||
0x01, 0x28, 0x09, 0x52, 0x0e, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x54,
|
||||
0x79, 0x70, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04,
|
||||
0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xf5, 0x01,
|
||||
0x0a, 0x14, 0x50, 0x6c, 0x61, 0x74, 0x66, 0x6f, 0x72, 0x6d, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61,
|
||||
0x74, 0x61, 0x53, 0x70, 0x65, 0x63, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f,
|
||||
0x72, 0x6d, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f,
|
||||
0x72, 0x6d, 0x12, 0x1a, 0x0a, 0x08, 0x68, 0x6f, 0x73, 0x74, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02,
|
||||
0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x68, 0x6f, 0x73, 0x74, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x16,
|
||||
0x0a, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06,
|
||||
0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x7a, 0x6f, 0x6e, 0x65, 0x18, 0x04,
|
||||
0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x7a, 0x6f, 0x6e, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e,
|
||||
0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28,
|
||||
0x09, 0x52, 0x0c, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12,
|
||||
0x1f, 0x0a, 0x0b, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x06,
|
||||
0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x49, 0x64,
|
||||
0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18,
|
||||
0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x49,
|
||||
0x64, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x70, 0x6f, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52,
|
||||
0x04, 0x73, 0x70, 0x6f, 0x74, 0x22, 0x3c, 0x0a, 0x0e, 0x55, 0x6e, 0x6d, 0x65, 0x74, 0x43, 0x6f,
|
||||
0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18,
|
||||
0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x72,
|
||||
0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61,
|
||||
0x73, 0x6f, 0x6e, 0x42, 0x4c, 0x5a, 0x4a, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f,
|
||||
0x6d, 0x2f, 0x73, 0x69, 0x64, 0x65, 0x72, 0x6f, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x74, 0x61, 0x6c,
|
||||
0x6f, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x6d, 0x61, 0x63, 0x68, 0x69, 0x6e, 0x65, 0x72, 0x79,
|
||||
0x2f, 0x61, 0x70, 0x69, 0x2f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2f, 0x64, 0x65,
|
||||
0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2f, 0x72, 0x75, 0x6e, 0x74, 0x69, 0x6d,
|
||||
0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
0x1a, 0x13, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x26, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2f,
|
||||
0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2f, 0x65, 0x6e, 0x75, 0x6d,
|
||||
0x73, 0x2f, 0x65, 0x6e, 0x75, 0x6d, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x29, 0x0a,
|
||||
0x11, 0x44, 0x65, 0x76, 0x69, 0x63, 0x65, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, 0x70,
|
||||
0x65, 0x63, 0x12, 0x14, 0x0a, 0x05, 0x72, 0x65, 0x61, 0x64, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28,
|
||||
0x08, 0x52, 0x05, 0x72, 0x65, 0x61, 0x64, 0x79, 0x22, 0x4a, 0x0a, 0x14, 0x4b, 0x65, 0x72, 0x6e,
|
||||
0x65, 0x6c, 0x4d, 0x6f, 0x64, 0x75, 0x6c, 0x65, 0x53, 0x70, 0x65, 0x63, 0x53, 0x70, 0x65, 0x63,
|
||||
0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04,
|
||||
0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65,
|
||||
0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x65,
|
||||
0x74, 0x65, 0x72, 0x73, 0x22, 0x50, 0x0a, 0x13, 0x4b, 0x65, 0x72, 0x6e, 0x65, 0x6c, 0x50, 0x61,
|
||||
0x72, 0x61, 0x6d, 0x53, 0x70, 0x65, 0x63, 0x53, 0x70, 0x65, 0x63, 0x12, 0x14, 0x0a, 0x05, 0x76,
|
||||
0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75,
|
||||
0x65, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x67, 0x6e, 0x6f, 0x72, 0x65, 0x5f, 0x65, 0x72, 0x72, 0x6f,
|
||||
0x72, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x67, 0x6e, 0x6f, 0x72, 0x65,
|
||||
0x45, 0x72, 0x72, 0x6f, 0x72, 0x73, 0x22, 0x6d, 0x0a, 0x15, 0x4b, 0x65, 0x72, 0x6e, 0x65, 0x6c,
|
||||
0x50, 0x61, 0x72, 0x61, 0x6d, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, 0x70, 0x65, 0x63, 0x12,
|
||||
0x18, 0x0a, 0x07, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
|
||||
0x52, 0x07, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x64, 0x65, 0x66,
|
||||
0x61, 0x75, 0x6c, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x64, 0x65, 0x66, 0x61,
|
||||
0x75, 0x6c, 0x74, 0x12, 0x20, 0x0a, 0x0b, 0x75, 0x6e, 0x73, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74,
|
||||
0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x75, 0x6e, 0x73, 0x75, 0x70, 0x70,
|
||||
0x6f, 0x72, 0x74, 0x65, 0x64, 0x22, 0x44, 0x0a, 0x11, 0x4b, 0x6d, 0x73, 0x67, 0x4c, 0x6f, 0x67,
|
||||
0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x53, 0x70, 0x65, 0x63, 0x12, 0x2f, 0x0a, 0x0c, 0x64, 0x65,
|
||||
0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b,
|
||||
0x32, 0x0b, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x55, 0x52, 0x4c, 0x52, 0x0c, 0x64,
|
||||
0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xb1, 0x01, 0x0a, 0x11,
|
||||
0x4d, 0x61, 0x63, 0x68, 0x69, 0x6e, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x53, 0x70, 0x65,
|
||||
0x63, 0x12, 0x4b, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x67, 0x65, 0x18, 0x01, 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, 0x52, 0x75, 0x6e, 0x74, 0x69, 0x6d, 0x65, 0x4d, 0x61, 0x63, 0x68, 0x69,
|
||||
0x6e, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x67, 0x65, 0x12, 0x4f,
|
||||
0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37,
|
||||
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, 0x72, 0x75, 0x6e, 0x74,
|
||||
0x69, 0x6d, 0x65, 0x2e, 0x4d, 0x61, 0x63, 0x68, 0x69, 0x6e, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75,
|
||||
0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22,
|
||||
0x8a, 0x01, 0x0a, 0x13, 0x4d, 0x61, 0x63, 0x68, 0x69, 0x6e, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75,
|
||||
0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x72, 0x65, 0x61, 0x64, 0x79,
|
||||
0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x72, 0x65, 0x61, 0x64, 0x79, 0x12, 0x5d, 0x0a,
|
||||
0x10, 0x75, 0x6e, 0x6d, 0x65, 0x74, 0x5f, 0x63, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e,
|
||||
0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 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, 0x72, 0x75, 0x6e, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x55, 0x6e, 0x6d,
|
||||
0x65, 0x74, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0f, 0x75, 0x6e, 0x6d,
|
||||
0x65, 0x74, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x23, 0x0a, 0x0b,
|
||||
0x4d, 0x65, 0x74, 0x61, 0x4b, 0x65, 0x79, 0x53, 0x70, 0x65, 0x63, 0x12, 0x14, 0x0a, 0x05, 0x76,
|
||||
0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75,
|
||||
0x65, 0x22, 0x84, 0x01, 0x0a, 0x0f, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75,
|
||||
0x73, 0x53, 0x70, 0x65, 0x63, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18,
|
||||
0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x16, 0x0a,
|
||||
0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74,
|
||||
0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x27, 0x0a, 0x0f, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x79, 0x73,
|
||||
0x74, 0x65, 0x6d, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e,
|
||||
0x66, 0x69, 0x6c, 0x65, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x54, 0x79, 0x70, 0x65, 0x12, 0x18,
|
||||
0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52,
|
||||
0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xf5, 0x01, 0x0a, 0x14, 0x50, 0x6c, 0x61,
|
||||
0x74, 0x66, 0x6f, 0x72, 0x6d, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x53, 0x70, 0x65,
|
||||
0x63, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f, 0x72, 0x6d, 0x18, 0x01, 0x20,
|
||||
0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f, 0x72, 0x6d, 0x12, 0x1a, 0x0a,
|
||||
0x08, 0x68, 0x6f, 0x73, 0x74, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x08, 0x68, 0x6f, 0x73, 0x74, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x67,
|
||||
0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f,
|
||||
0x6e, 0x12, 0x12, 0x0a, 0x04, 0x7a, 0x6f, 0x6e, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x04, 0x7a, 0x6f, 0x6e, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63,
|
||||
0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x69, 0x6e,
|
||||
0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x6e,
|
||||
0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x0a, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x49, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x70,
|
||||
0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09,
|
||||
0x52, 0x0a, 0x70, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04,
|
||||
0x73, 0x70, 0x6f, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x04, 0x73, 0x70, 0x6f, 0x74,
|
||||
0x22, 0x3c, 0x0a, 0x0e, 0x55, 0x6e, 0x6d, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69,
|
||||
0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
|
||||
0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e,
|
||||
0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x42, 0x4c,
|
||||
0x5a, 0x4a, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x73, 0x69, 0x64,
|
||||
0x65, 0x72, 0x6f, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x74, 0x61, 0x6c, 0x6f, 0x73, 0x2f, 0x70, 0x6b,
|
||||
0x67, 0x2f, 0x6d, 0x61, 0x63, 0x68, 0x69, 0x6e, 0x65, 0x72, 0x79, 0x2f, 0x61, 0x70, 0x69, 0x2f,
|
||||
0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2f, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74,
|
||||
0x69, 0x6f, 0x6e, 0x73, 0x2f, 0x72, 0x75, 0x6e, 0x74, 0x69, 0x6d, 0x65, 0x62, 0x06, 0x70, 0x72,
|
||||
0x6f, 0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
@ -739,29 +794,32 @@ func file_resource_definitions_runtime_runtime_proto_rawDescGZIP() []byte {
|
||||
return file_resource_definitions_runtime_runtime_proto_rawDescData
|
||||
}
|
||||
|
||||
var file_resource_definitions_runtime_runtime_proto_msgTypes = make([]protoimpl.MessageInfo, 10)
|
||||
var file_resource_definitions_runtime_runtime_proto_msgTypes = make([]protoimpl.MessageInfo, 11)
|
||||
var file_resource_definitions_runtime_runtime_proto_goTypes = []interface{}{
|
||||
(*DevicesStatusSpec)(nil), // 0: talos.resource.definitions.runtime.DevicesStatusSpec
|
||||
(*KernelModuleSpecSpec)(nil), // 1: talos.resource.definitions.runtime.KernelModuleSpecSpec
|
||||
(*KernelParamSpecSpec)(nil), // 2: talos.resource.definitions.runtime.KernelParamSpecSpec
|
||||
(*KernelParamStatusSpec)(nil), // 3: talos.resource.definitions.runtime.KernelParamStatusSpec
|
||||
(*MachineStatusSpec)(nil), // 4: talos.resource.definitions.runtime.MachineStatusSpec
|
||||
(*MachineStatusStatus)(nil), // 5: talos.resource.definitions.runtime.MachineStatusStatus
|
||||
(*MetaKeySpec)(nil), // 6: talos.resource.definitions.runtime.MetaKeySpec
|
||||
(*MountStatusSpec)(nil), // 7: talos.resource.definitions.runtime.MountStatusSpec
|
||||
(*PlatformMetadataSpec)(nil), // 8: talos.resource.definitions.runtime.PlatformMetadataSpec
|
||||
(*UnmetCondition)(nil), // 9: talos.resource.definitions.runtime.UnmetCondition
|
||||
(enums.RuntimeMachineStage)(0), // 10: talos.resource.definitions.enums.RuntimeMachineStage
|
||||
(*KmsgLogConfigSpec)(nil), // 4: talos.resource.definitions.runtime.KmsgLogConfigSpec
|
||||
(*MachineStatusSpec)(nil), // 5: talos.resource.definitions.runtime.MachineStatusSpec
|
||||
(*MachineStatusStatus)(nil), // 6: talos.resource.definitions.runtime.MachineStatusStatus
|
||||
(*MetaKeySpec)(nil), // 7: talos.resource.definitions.runtime.MetaKeySpec
|
||||
(*MountStatusSpec)(nil), // 8: talos.resource.definitions.runtime.MountStatusSpec
|
||||
(*PlatformMetadataSpec)(nil), // 9: talos.resource.definitions.runtime.PlatformMetadataSpec
|
||||
(*UnmetCondition)(nil), // 10: talos.resource.definitions.runtime.UnmetCondition
|
||||
(*common.URL)(nil), // 11: common.URL
|
||||
(enums.RuntimeMachineStage)(0), // 12: talos.resource.definitions.enums.RuntimeMachineStage
|
||||
}
|
||||
var file_resource_definitions_runtime_runtime_proto_depIdxs = []int32{
|
||||
10, // 0: talos.resource.definitions.runtime.MachineStatusSpec.stage:type_name -> talos.resource.definitions.enums.RuntimeMachineStage
|
||||
5, // 1: talos.resource.definitions.runtime.MachineStatusSpec.status:type_name -> talos.resource.definitions.runtime.MachineStatusStatus
|
||||
9, // 2: talos.resource.definitions.runtime.MachineStatusStatus.unmet_conditions:type_name -> talos.resource.definitions.runtime.UnmetCondition
|
||||
3, // [3:3] is the sub-list for method output_type
|
||||
3, // [3:3] is the sub-list for method input_type
|
||||
3, // [3:3] is the sub-list for extension type_name
|
||||
3, // [3:3] is the sub-list for extension extendee
|
||||
0, // [0:3] is the sub-list for field type_name
|
||||
11, // 0: talos.resource.definitions.runtime.KmsgLogConfigSpec.destinations:type_name -> common.URL
|
||||
12, // 1: talos.resource.definitions.runtime.MachineStatusSpec.stage:type_name -> talos.resource.definitions.enums.RuntimeMachineStage
|
||||
6, // 2: talos.resource.definitions.runtime.MachineStatusSpec.status:type_name -> talos.resource.definitions.runtime.MachineStatusStatus
|
||||
10, // 3: talos.resource.definitions.runtime.MachineStatusStatus.unmet_conditions:type_name -> talos.resource.definitions.runtime.UnmetCondition
|
||||
4, // [4:4] is the sub-list for method output_type
|
||||
4, // [4:4] is the sub-list for method input_type
|
||||
4, // [4:4] is the sub-list for extension type_name
|
||||
4, // [4:4] is the sub-list for extension extendee
|
||||
0, // [0:4] is the sub-list for field type_name
|
||||
}
|
||||
|
||||
func init() { file_resource_definitions_runtime_runtime_proto_init() }
|
||||
@ -819,7 +877,7 @@ func file_resource_definitions_runtime_runtime_proto_init() {
|
||||
}
|
||||
}
|
||||
file_resource_definitions_runtime_runtime_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*MachineStatusSpec); i {
|
||||
switch v := v.(*KmsgLogConfigSpec); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
@ -831,7 +889,7 @@ func file_resource_definitions_runtime_runtime_proto_init() {
|
||||
}
|
||||
}
|
||||
file_resource_definitions_runtime_runtime_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*MachineStatusStatus); i {
|
||||
switch v := v.(*MachineStatusSpec); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
@ -843,7 +901,7 @@ func file_resource_definitions_runtime_runtime_proto_init() {
|
||||
}
|
||||
}
|
||||
file_resource_definitions_runtime_runtime_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*MetaKeySpec); i {
|
||||
switch v := v.(*MachineStatusStatus); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
@ -855,7 +913,7 @@ func file_resource_definitions_runtime_runtime_proto_init() {
|
||||
}
|
||||
}
|
||||
file_resource_definitions_runtime_runtime_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*MountStatusSpec); i {
|
||||
switch v := v.(*MetaKeySpec); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
@ -867,7 +925,7 @@ func file_resource_definitions_runtime_runtime_proto_init() {
|
||||
}
|
||||
}
|
||||
file_resource_definitions_runtime_runtime_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*PlatformMetadataSpec); i {
|
||||
switch v := v.(*MountStatusSpec); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
@ -879,6 +937,18 @@ func file_resource_definitions_runtime_runtime_proto_init() {
|
||||
}
|
||||
}
|
||||
file_resource_definitions_runtime_runtime_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*PlatformMetadataSpec); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_resource_definitions_runtime_runtime_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*UnmetCondition); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
@ -897,7 +967,7 @@ func file_resource_definitions_runtime_runtime_proto_init() {
|
||||
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
||||
RawDescriptor: file_resource_definitions_runtime_runtime_proto_rawDesc,
|
||||
NumEnums: 0,
|
||||
NumMessages: 10,
|
||||
NumMessages: 11,
|
||||
NumExtensions: 0,
|
||||
NumServices: 0,
|
||||
},
|
||||
|
@ -9,8 +9,10 @@ import (
|
||||
io "io"
|
||||
bits "math/bits"
|
||||
|
||||
proto "google.golang.org/protobuf/proto"
|
||||
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
||||
|
||||
common "github.com/siderolabs/talos/pkg/machinery/api/common"
|
||||
enums "github.com/siderolabs/talos/pkg/machinery/api/resource/definitions/enums"
|
||||
)
|
||||
|
||||
@ -220,6 +222,63 @@ func (m *KernelParamStatusSpec) MarshalToSizedBufferVT(dAtA []byte) (int, error)
|
||||
return len(dAtA) - i, nil
|
||||
}
|
||||
|
||||
func (m *KmsgLogConfigSpec) 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 *KmsgLogConfigSpec) MarshalToVT(dAtA []byte) (int, error) {
|
||||
size := m.SizeVT()
|
||||
return m.MarshalToSizedBufferVT(dAtA[:size])
|
||||
}
|
||||
|
||||
func (m *KmsgLogConfigSpec) 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.Destinations) > 0 {
|
||||
for iNdEx := len(m.Destinations) - 1; iNdEx >= 0; iNdEx-- {
|
||||
if vtmsg, ok := interface{}(m.Destinations[iNdEx]).(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.Destinations[iNdEx])
|
||||
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 *MachineStatusSpec) MarshalVT() (dAtA []byte, err error) {
|
||||
if m == nil {
|
||||
return nil, nil
|
||||
@ -647,6 +706,28 @@ func (m *KernelParamStatusSpec) SizeVT() (n int) {
|
||||
return n
|
||||
}
|
||||
|
||||
func (m *KmsgLogConfigSpec) SizeVT() (n int) {
|
||||
if m == nil {
|
||||
return 0
|
||||
}
|
||||
var l int
|
||||
_ = l
|
||||
if len(m.Destinations) > 0 {
|
||||
for _, e := range m.Destinations {
|
||||
if size, ok := interface{}(e).(interface {
|
||||
SizeVT() int
|
||||
}); ok {
|
||||
l = size.SizeVT()
|
||||
} else {
|
||||
l = proto.Size(e)
|
||||
}
|
||||
n += 1 + l + sov(uint64(l))
|
||||
}
|
||||
}
|
||||
n += len(m.unknownFields)
|
||||
return n
|
||||
}
|
||||
|
||||
func (m *MachineStatusSpec) SizeVT() (n int) {
|
||||
if m == nil {
|
||||
return 0
|
||||
@ -1214,6 +1295,99 @@ func (m *KernelParamStatusSpec) UnmarshalVT(dAtA []byte) error {
|
||||
}
|
||||
return nil
|
||||
}
|
||||
func (m *KmsgLogConfigSpec) 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: KmsgLogConfigSpec: wiretype end group for non-group")
|
||||
}
|
||||
if fieldNum <= 0 {
|
||||
return fmt.Errorf("proto: KmsgLogConfigSpec: illegal tag %d (wire type %d)", fieldNum, wire)
|
||||
}
|
||||
switch fieldNum {
|
||||
case 1:
|
||||
if wireType != 2 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field Destinations", 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
|
||||
}
|
||||
m.Destinations = append(m.Destinations, &common.URL{})
|
||||
if unmarshal, ok := interface{}(m.Destinations[len(m.Destinations)-1]).(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.Destinations[len(m.Destinations)-1]); 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 *MachineStatusSpec) UnmarshalVT(dAtA []byte) error {
|
||||
l := len(dAtA)
|
||||
iNdEx := 0
|
||||
|
@ -2,10 +2,14 @@
|
||||
// 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 DevicesStatusSpec -type KernelModuleSpecSpec -type KernelParamSpecSpec -type KernelParamStatusSpec -type MachineStatusSpec -type MetaKeySpec -type MountStatusSpec -type PlatformMetadataSpec -header-file ../../../../hack/boilerplate.txt -o deep_copy.generated.go ."; DO NOT EDIT.
|
||||
// Code generated by "deep-copy -type DevicesStatusSpec -type KernelModuleSpecSpec -type KernelParamSpecSpec -type KernelParamStatusSpec -type KmsgLogConfigSpec -type MachineStatusSpec -type MetaKeySpec -type MountStatusSpec -type PlatformMetadataSpec -header-file ../../../../hack/boilerplate.txt -o deep_copy.generated.go ."; DO NOT EDIT.
|
||||
|
||||
package runtime
|
||||
|
||||
import (
|
||||
"net/url"
|
||||
)
|
||||
|
||||
// DeepCopy generates a deep copy of DevicesStatusSpec.
|
||||
func (o DevicesStatusSpec) DeepCopy() DevicesStatusSpec {
|
||||
var cp DevicesStatusSpec = o
|
||||
@ -34,6 +38,26 @@ func (o KernelParamStatusSpec) DeepCopy() KernelParamStatusSpec {
|
||||
return cp
|
||||
}
|
||||
|
||||
// DeepCopy generates a deep copy of KmsgLogConfigSpec.
|
||||
func (o KmsgLogConfigSpec) DeepCopy() KmsgLogConfigSpec {
|
||||
var cp KmsgLogConfigSpec = o
|
||||
if o.Destinations != nil {
|
||||
cp.Destinations = make([]*url.URL, len(o.Destinations))
|
||||
copy(cp.Destinations, o.Destinations)
|
||||
for i2 := range o.Destinations {
|
||||
if o.Destinations[i2] != nil {
|
||||
cp.Destinations[i2] = new(url.URL)
|
||||
*cp.Destinations[i2] = *o.Destinations[i2]
|
||||
if o.Destinations[i2].User != nil {
|
||||
cp.Destinations[i2].User = new(url.Userinfo)
|
||||
*cp.Destinations[i2].User = *o.Destinations[i2].User
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return cp
|
||||
}
|
||||
|
||||
// DeepCopy generates a deep copy of MachineStatusSpec.
|
||||
func (o MachineStatusSpec) DeepCopy() MachineStatusSpec {
|
||||
var cp MachineStatusSpec = o
|
||||
|
61
pkg/machinery/resources/runtime/kmsg_log_config.go
Normal file
61
pkg/machinery/resources/runtime/kmsg_log_config.go
Normal file
@ -0,0 +1,61 @@
|
||||
// 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 runtime
|
||||
|
||||
import (
|
||||
"net/url"
|
||||
|
||||
"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"
|
||||
)
|
||||
|
||||
// KmsgLogConfigType is type of KernelParam resource.
|
||||
const KmsgLogConfigType = resource.Type("KmsgLogConfigs.runtime.talos.dev")
|
||||
|
||||
// KmsgLogConfig resource holds configuration for kernel message log streaming.
|
||||
type KmsgLogConfig = typed.Resource[KmsgLogConfigSpec, KmsgLogConfigExtension]
|
||||
|
||||
// KmsgLogConfigID is a resource ID for KmsgLogConfig.
|
||||
const KmsgLogConfigID resource.ID = "kmsg-log"
|
||||
|
||||
// KmsgLogConfigSpec describes status of the defined sysctls.
|
||||
//
|
||||
//gotagsrewrite:gen
|
||||
type KmsgLogConfigSpec struct {
|
||||
Destinations []*url.URL `yaml:"destinations" protobuf:"1"`
|
||||
}
|
||||
|
||||
// NewKmsgLogConfig initializes a KmsgLogConfig resource.
|
||||
func NewKmsgLogConfig() *KmsgLogConfig {
|
||||
return typed.NewResource[KmsgLogConfigSpec, KmsgLogConfigExtension](
|
||||
resource.NewMetadata(NamespaceName, KmsgLogConfigType, KmsgLogConfigID, resource.VersionUndefined),
|
||||
KmsgLogConfigSpec{},
|
||||
)
|
||||
}
|
||||
|
||||
// KmsgLogConfigExtension is auxiliary resource data for KmsgLogConfig.
|
||||
type KmsgLogConfigExtension struct{}
|
||||
|
||||
// ResourceDefinition implements meta.ResourceDefinitionProvider interface.
|
||||
func (KmsgLogConfigExtension) ResourceDefinition() meta.ResourceDefinitionSpec {
|
||||
return meta.ResourceDefinitionSpec{
|
||||
Type: KmsgLogConfigType,
|
||||
Aliases: []resource.Type{},
|
||||
DefaultNamespace: NamespaceName,
|
||||
}
|
||||
}
|
||||
|
||||
func init() {
|
||||
proto.RegisterDefaultTypes()
|
||||
|
||||
err := protobuf.RegisterDynamic[KmsgLogConfigSpec](KmsgLogConfigType, &KmsgLogConfig{})
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
@ -5,4 +5,4 @@
|
||||
package runtime
|
||||
|
||||
//nolint:lll
|
||||
//go:generate deep-copy -type DevicesStatusSpec -type KernelModuleSpecSpec -type KernelParamSpecSpec -type KernelParamStatusSpec -type MachineStatusSpec -type MetaKeySpec -type MountStatusSpec -type PlatformMetadataSpec -header-file ../../../../hack/boilerplate.txt -o deep_copy.generated.go .
|
||||
//go:generate deep-copy -type DevicesStatusSpec -type KernelModuleSpecSpec -type KernelParamSpecSpec -type KernelParamStatusSpec -type KmsgLogConfigSpec -type MachineStatusSpec -type MetaKeySpec -type MountStatusSpec -type PlatformMetadataSpec -header-file ../../../../hack/boilerplate.txt -o deep_copy.generated.go .
|
||||
|
@ -30,6 +30,7 @@ func TestRegisterResource(t *testing.T) {
|
||||
&runtime.KernelModuleSpec{},
|
||||
&runtime.KernelParamSpec{},
|
||||
&runtime.KernelParamStatus{},
|
||||
&runtime.KmsgLogConfig{},
|
||||
&runtime.MachineStatus{},
|
||||
&runtime.MetaKey{},
|
||||
&runtime.MountStatus{},
|
||||
|
@ -182,6 +182,7 @@ description: Talos gRPC API reference.
|
||||
- [KernelModuleSpecSpec](#talos.resource.definitions.runtime.KernelModuleSpecSpec)
|
||||
- [KernelParamSpecSpec](#talos.resource.definitions.runtime.KernelParamSpecSpec)
|
||||
- [KernelParamStatusSpec](#talos.resource.definitions.runtime.KernelParamStatusSpec)
|
||||
- [KmsgLogConfigSpec](#talos.resource.definitions.runtime.KmsgLogConfigSpec)
|
||||
- [MachineStatusSpec](#talos.resource.definitions.runtime.MachineStatusSpec)
|
||||
- [MachineStatusStatus](#talos.resource.definitions.runtime.MachineStatusStatus)
|
||||
- [MetaKeySpec](#talos.resource.definitions.runtime.MetaKeySpec)
|
||||
@ -3322,6 +3323,21 @@ KernelParamStatusSpec describes status of the defined sysctls.
|
||||
|
||||
|
||||
|
||||
<a name="talos.resource.definitions.runtime.KmsgLogConfigSpec"></a>
|
||||
|
||||
### KmsgLogConfigSpec
|
||||
KmsgLogConfigSpec describes status of the defined sysctls.
|
||||
|
||||
|
||||
| Field | Type | Label | Description |
|
||||
| ----- | ---- | ----- | ----------- |
|
||||
| destinations | [common.URL](#common.URL) | repeated | |
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
<a name="talos.resource.definitions.runtime.MachineStatusSpec"></a>
|
||||
|
||||
### MachineStatusSpec
|
||||
|
Loading…
x
Reference in New Issue
Block a user