cgroup2: implement containerd.events.TaskOOM event
				
					
				
			How to test (from https://github.com/opencontainers/runc/pull/2352#issuecomment-620834524): (host)$ sudo swapoff -a (host)$ sudo ctr run -t --rm --memory-limit $((1024*1024*32)) docker.io/library/alpine:latest foo (container)$ sh -c 'VAR=$(seq 1 100000000)' An event `/tasks/oom {"container_id":"foo"}` will be displayed in `ctr events`. Signed-off-by: Akihiro Suda <akihiro.suda.cz@hco.ntt.co.jp>
This commit is contained in:
		| @@ -21,10 +21,8 @@ package v2 | |||||||
| import ( | import ( | ||||||
| 	"context" | 	"context" | ||||||
|  |  | ||||||
| 	"github.com/containerd/containerd/errdefs" |  | ||||||
| 	"github.com/containerd/containerd/events" | 	"github.com/containerd/containerd/events" | ||||||
| 	"github.com/containerd/containerd/runtime" | 	"github.com/containerd/containerd/runtime" | ||||||
| 	"github.com/containerd/containerd/runtime/v1/linux" |  | ||||||
| 	metrics "github.com/docker/go-metrics" | 	metrics "github.com/docker/go-metrics" | ||||||
| ) | ) | ||||||
|  |  | ||||||
| @@ -48,19 +46,6 @@ func (m *cgroupsMonitor) Monitor(c runtime.Task) error { | |||||||
| 	if err := m.collector.Add(c); err != nil { | 	if err := m.collector.Add(c); err != nil { | ||||||
| 		return err | 		return err | ||||||
| 	} | 	} | ||||||
| 	t, ok := c.(*linux.Task) |  | ||||||
| 	if !ok { |  | ||||||
| 		return nil |  | ||||||
| 	} |  | ||||||
| 	cg, err := t.Cgroup() |  | ||||||
| 	if err != nil { |  | ||||||
| 		if errdefs.IsNotFound(err) { |  | ||||||
| 			return nil |  | ||||||
| 		} |  | ||||||
| 		return err |  | ||||||
| 	} |  | ||||||
| 	// OOM handler is not implemented yet |  | ||||||
| 	_ = cg |  | ||||||
| 	return nil | 	return nil | ||||||
| } | } | ||||||
|  |  | ||||||
|   | |||||||
| @@ -586,4 +586,20 @@ var memoryMetrics = []*metric{ | |||||||
| 			} | 			} | ||||||
| 		}, | 		}, | ||||||
| 	}, | 	}, | ||||||
|  | 	{ | ||||||
|  | 		name: "memory_oom", | ||||||
|  | 		help: "The number of times a container has received an oom event", | ||||||
|  | 		unit: metrics.Total, | ||||||
|  | 		vt:   prometheus.GaugeValue, | ||||||
|  | 		getValues: func(stats *v2.Metrics) []value { | ||||||
|  | 			if stats.MemoryEvents == nil { | ||||||
|  | 				return nil | ||||||
|  | 			} | ||||||
|  | 			return []value{ | ||||||
|  | 				{ | ||||||
|  | 					v: float64(stats.MemoryEvents.Oom), | ||||||
|  | 				}, | ||||||
|  | 			} | ||||||
|  | 		}, | ||||||
|  | 	}, | ||||||
| } | } | ||||||
|   | |||||||
							
								
								
									
										30
									
								
								pkg/oom/oom.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										30
									
								
								pkg/oom/oom.go
									
									
									
									
									
										Normal file
									
								
							| @@ -0,0 +1,30 @@ | |||||||
|  | // +build linux | ||||||
|  |  | ||||||
|  | /* | ||||||
|  |    Copyright The containerd Authors. | ||||||
|  |  | ||||||
|  |    Licensed under the Apache License, Version 2.0 (the "License"); | ||||||
|  |    you may not use this file except in compliance with the License. | ||||||
|  |    You may obtain a copy of the License at | ||||||
|  |  | ||||||
|  |        http://www.apache.org/licenses/LICENSE-2.0 | ||||||
|  |  | ||||||
|  |    Unless required by applicable law or agreed to in writing, software | ||||||
|  |    distributed under the License is distributed on an "AS IS" BASIS, | ||||||
|  |    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||||||
|  |    See the License for the specific language governing permissions and | ||||||
|  |    limitations under the License. | ||||||
|  | */ | ||||||
|  |  | ||||||
|  | package oom | ||||||
|  |  | ||||||
|  | import ( | ||||||
|  | 	"context" | ||||||
|  | ) | ||||||
|  |  | ||||||
|  | // Watcher watches OOM events | ||||||
|  | type Watcher interface { | ||||||
|  | 	Close() error | ||||||
|  | 	Run(ctx context.Context) | ||||||
|  | 	Add(id string, cg interface{}) error | ||||||
|  | } | ||||||
| @@ -16,7 +16,7 @@ | |||||||
|    limitations under the License. |    limitations under the License. | ||||||
| */ | */ | ||||||
| 
 | 
 | ||||||
| package oom | package v1 | ||||||
| 
 | 
 | ||||||
| import ( | import ( | ||||||
| 	"context" | 	"context" | ||||||
| @@ -24,28 +24,30 @@ import ( | |||||||
| 
 | 
 | ||||||
| 	"github.com/containerd/cgroups" | 	"github.com/containerd/cgroups" | ||||||
| 	eventstypes "github.com/containerd/containerd/api/events" | 	eventstypes "github.com/containerd/containerd/api/events" | ||||||
|  | 	"github.com/containerd/containerd/pkg/oom" | ||||||
| 	"github.com/containerd/containerd/runtime" | 	"github.com/containerd/containerd/runtime" | ||||||
| 	"github.com/containerd/containerd/runtime/v2/shim" | 	"github.com/containerd/containerd/runtime/v2/shim" | ||||||
|  | 	"github.com/pkg/errors" | ||||||
| 	"github.com/sirupsen/logrus" | 	"github.com/sirupsen/logrus" | ||||||
| 	"golang.org/x/sys/unix" | 	"golang.org/x/sys/unix" | ||||||
| ) | ) | ||||||
| 
 | 
 | ||||||
| // New returns an epoll implementation that listens to OOM events | // New returns an epoll implementation that listens to OOM events | ||||||
| // from a container's cgroups. | // from a container's cgroups. | ||||||
| func New(publisher shim.Publisher) (*Epoller, error) { | func New(publisher shim.Publisher) (oom.Watcher, error) { | ||||||
| 	fd, err := unix.EpollCreate1(unix.EPOLL_CLOEXEC) | 	fd, err := unix.EpollCreate1(unix.EPOLL_CLOEXEC) | ||||||
| 	if err != nil { | 	if err != nil { | ||||||
| 		return nil, err | 		return nil, err | ||||||
| 	} | 	} | ||||||
| 	return &Epoller{ | 	return &epoller{ | ||||||
| 		fd:        fd, | 		fd:        fd, | ||||||
| 		publisher: publisher, | 		publisher: publisher, | ||||||
| 		set:       make(map[uintptr]*item), | 		set:       make(map[uintptr]*item), | ||||||
| 	}, nil | 	}, nil | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // Epoller implementation for handling OOM events from a container's cgroup | // epoller implementation for handling OOM events from a container's cgroup | ||||||
| type Epoller struct { | type epoller struct { | ||||||
| 	mu sync.Mutex | 	mu sync.Mutex | ||||||
| 
 | 
 | ||||||
| 	fd        int | 	fd        int | ||||||
| @@ -59,12 +61,12 @@ type item struct { | |||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // Close the epoll fd | // Close the epoll fd | ||||||
| func (e *Epoller) Close() error { | func (e *epoller) Close() error { | ||||||
| 	return unix.Close(e.fd) | 	return unix.Close(e.fd) | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // Run the epoll loop | // Run the epoll loop | ||||||
| func (e *Epoller) Run(ctx context.Context) { | func (e *epoller) Run(ctx context.Context) { | ||||||
| 	var events [128]unix.EpollEvent | 	var events [128]unix.EpollEvent | ||||||
| 	for { | 	for { | ||||||
| 		select { | 		select { | ||||||
| @@ -86,8 +88,12 @@ func (e *Epoller) Run(ctx context.Context) { | |||||||
| 	} | 	} | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // Add the cgroup to the epoll monitor | // Add cgroups.Cgroup to the epoll monitor | ||||||
| func (e *Epoller) Add(id string, cg cgroups.Cgroup) error { | func (e *epoller) Add(id string, cgx interface{}) error { | ||||||
|  | 	cg, ok := cgx.(cgroups.Cgroup) | ||||||
|  | 	if !ok { | ||||||
|  | 		return errors.Errorf("expected cgroups.Cgroup, got: %T", cgx) | ||||||
|  | 	} | ||||||
| 	e.mu.Lock() | 	e.mu.Lock() | ||||||
| 	defer e.mu.Unlock() | 	defer e.mu.Unlock() | ||||||
| 	fd, err := cg.OOMEventFD() | 	fd, err := cg.OOMEventFD() | ||||||
| @@ -105,7 +111,7 @@ func (e *Epoller) Add(id string, cg cgroups.Cgroup) error { | |||||||
| 	return unix.EpollCtl(e.fd, unix.EPOLL_CTL_ADD, int(fd), &event) | 	return unix.EpollCtl(e.fd, unix.EPOLL_CTL_ADD, int(fd), &event) | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| func (e *Epoller) process(ctx context.Context, fd uintptr) { | func (e *epoller) process(ctx context.Context, fd uintptr) { | ||||||
| 	flush(fd) | 	flush(fd) | ||||||
| 	e.mu.Lock() | 	e.mu.Lock() | ||||||
| 	i, ok := e.set[fd] | 	i, ok := e.set[fd] | ||||||
							
								
								
									
										113
									
								
								pkg/oom/v2/v2.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										113
									
								
								pkg/oom/v2/v2.go
									
									
									
									
									
										Normal file
									
								
							| @@ -0,0 +1,113 @@ | |||||||
|  | // +build linux | ||||||
|  |  | ||||||
|  | /* | ||||||
|  |    Copyright The containerd Authors. | ||||||
|  |  | ||||||
|  |    Licensed under the Apache License, Version 2.0 (the "License"); | ||||||
|  |    you may not use this file except in compliance with the License. | ||||||
|  |    You may obtain a copy of the License at | ||||||
|  |  | ||||||
|  |        http://www.apache.org/licenses/LICENSE-2.0 | ||||||
|  |  | ||||||
|  |    Unless required by applicable law or agreed to in writing, software | ||||||
|  |    distributed under the License is distributed on an "AS IS" BASIS, | ||||||
|  |    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||||||
|  |    See the License for the specific language governing permissions and | ||||||
|  |    limitations under the License. | ||||||
|  | */ | ||||||
|  |  | ||||||
|  | package v2 | ||||||
|  |  | ||||||
|  | import ( | ||||||
|  | 	"context" | ||||||
|  |  | ||||||
|  | 	cgroupsv2 "github.com/containerd/cgroups/v2" | ||||||
|  | 	eventstypes "github.com/containerd/containerd/api/events" | ||||||
|  | 	"github.com/containerd/containerd/pkg/oom" | ||||||
|  | 	"github.com/containerd/containerd/runtime" | ||||||
|  | 	"github.com/containerd/containerd/runtime/v2/shim" | ||||||
|  | 	"github.com/pkg/errors" | ||||||
|  | 	"github.com/sirupsen/logrus" | ||||||
|  | ) | ||||||
|  |  | ||||||
|  | // New returns an implementation that listens to OOM events | ||||||
|  | // from a container's cgroups. | ||||||
|  | func New(publisher shim.Publisher) (oom.Watcher, error) { | ||||||
|  | 	return &watcher{ | ||||||
|  | 		itemCh:    make(chan item), | ||||||
|  | 		publisher: publisher, | ||||||
|  | 	}, nil | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // watcher implementation for handling OOM events from a container's cgroup | ||||||
|  | type watcher struct { | ||||||
|  | 	itemCh    chan item | ||||||
|  | 	publisher shim.Publisher | ||||||
|  | } | ||||||
|  |  | ||||||
|  | type item struct { | ||||||
|  | 	id  string | ||||||
|  | 	ev  cgroupsv2.Event | ||||||
|  | 	err error | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Close closes the watcher | ||||||
|  | func (w *watcher) Close() error { | ||||||
|  | 	return nil | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Run the loop | ||||||
|  | func (w *watcher) Run(ctx context.Context) { | ||||||
|  | 	lastOOMMap := make(map[string]uint64) // key: id, value: ev.OOM | ||||||
|  | 	for { | ||||||
|  | 		select { | ||||||
|  | 		case <-ctx.Done(): | ||||||
|  | 			w.Close() | ||||||
|  | 			return | ||||||
|  | 		case i := <-w.itemCh: | ||||||
|  | 			if i.err != nil { | ||||||
|  | 				delete(lastOOMMap, i.id) | ||||||
|  | 				continue | ||||||
|  | 			} | ||||||
|  | 			lastOOM := lastOOMMap[i.id] | ||||||
|  | 			if i.ev.OOM > lastOOM { | ||||||
|  | 				if err := w.publisher.Publish(ctx, runtime.TaskOOMEventTopic, &eventstypes.TaskOOM{ | ||||||
|  | 					ContainerID: i.id, | ||||||
|  | 				}); err != nil { | ||||||
|  | 					logrus.WithError(err).Error("publish OOM event") | ||||||
|  | 				} | ||||||
|  | 			} | ||||||
|  | 			if i.ev.OOM > 0 { | ||||||
|  | 				lastOOMMap[i.id] = i.ev.OOM | ||||||
|  | 			} | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Add cgroups.Cgroup to the epoll monitor | ||||||
|  | func (w *watcher) Add(id string, cgx interface{}) error { | ||||||
|  | 	cg, ok := cgx.(*cgroupsv2.Manager) | ||||||
|  | 	if !ok { | ||||||
|  | 		return errors.Errorf("expected *cgroupsv2.Manager, got: %T", cgx) | ||||||
|  | 	} | ||||||
|  | 	// FIXME: cgroupsv2.Manager does not support closing eventCh routine currently. | ||||||
|  | 	// The routine shuts down when an error happens, mostly when the cgroup is deleted. | ||||||
|  | 	eventCh, errCh := cg.EventChan() | ||||||
|  | 	go func() { | ||||||
|  | 		for { | ||||||
|  | 			i := item{id: id} | ||||||
|  | 			select { | ||||||
|  | 			case ev := <-eventCh: | ||||||
|  | 				i.ev = ev | ||||||
|  | 				w.itemCh <- i | ||||||
|  | 			case err := <-errCh: | ||||||
|  | 				i.err = err | ||||||
|  | 				w.itemCh <- i | ||||||
|  | 				// we no longer get any event/err when we got an err | ||||||
|  | 				logrus.WithError(err).Warn("error from *cgroupsv2.Manager.EventChan") | ||||||
|  | 				return | ||||||
|  | 			} | ||||||
|  | 		} | ||||||
|  | 	}() | ||||||
|  | 	return nil | ||||||
|  | } | ||||||
| @@ -35,6 +35,7 @@ import ( | |||||||
| 	"github.com/containerd/containerd/mount" | 	"github.com/containerd/containerd/mount" | ||||||
| 	"github.com/containerd/containerd/namespaces" | 	"github.com/containerd/containerd/namespaces" | ||||||
| 	"github.com/containerd/containerd/pkg/oom" | 	"github.com/containerd/containerd/pkg/oom" | ||||||
|  | 	oomv1 "github.com/containerd/containerd/pkg/oom/v1" | ||||||
| 	"github.com/containerd/containerd/pkg/process" | 	"github.com/containerd/containerd/pkg/process" | ||||||
| 	"github.com/containerd/containerd/pkg/stdio" | 	"github.com/containerd/containerd/pkg/stdio" | ||||||
| 	"github.com/containerd/containerd/runtime/v2/runc" | 	"github.com/containerd/containerd/runtime/v2/runc" | ||||||
| @@ -58,7 +59,7 @@ var ( | |||||||
|  |  | ||||||
| // New returns a new shim service that can be used via GRPC | // New returns a new shim service that can be used via GRPC | ||||||
| func New(ctx context.Context, id string, publisher shim.Publisher, shutdown func()) (shim.Shim, error) { | func New(ctx context.Context, id string, publisher shim.Publisher, shutdown func()) (shim.Shim, error) { | ||||||
| 	ep, err := oom.New(publisher) | 	ep, err := oomv1.New(publisher) | ||||||
| 	if err != nil { | 	if err != nil { | ||||||
| 		return nil, err | 		return nil, err | ||||||
| 	} | 	} | ||||||
| @@ -90,7 +91,7 @@ type service struct { | |||||||
| 	events   chan interface{} | 	events   chan interface{} | ||||||
| 	platform stdio.Platform | 	platform stdio.Platform | ||||||
| 	ec       chan runcC.Exit | 	ec       chan runcC.Exit | ||||||
| 	ep       *oom.Epoller | 	ep       oom.Watcher | ||||||
|  |  | ||||||
| 	id        string | 	id        string | ||||||
| 	container *runc.Container | 	container *runc.Container | ||||||
|   | |||||||
| @@ -38,6 +38,8 @@ import ( | |||||||
| 	"github.com/containerd/containerd/mount" | 	"github.com/containerd/containerd/mount" | ||||||
| 	"github.com/containerd/containerd/namespaces" | 	"github.com/containerd/containerd/namespaces" | ||||||
| 	"github.com/containerd/containerd/pkg/oom" | 	"github.com/containerd/containerd/pkg/oom" | ||||||
|  | 	oomv1 "github.com/containerd/containerd/pkg/oom/v1" | ||||||
|  | 	oomv2 "github.com/containerd/containerd/pkg/oom/v2" | ||||||
| 	"github.com/containerd/containerd/pkg/process" | 	"github.com/containerd/containerd/pkg/process" | ||||||
| 	"github.com/containerd/containerd/pkg/stdio" | 	"github.com/containerd/containerd/pkg/stdio" | ||||||
| 	"github.com/containerd/containerd/runtime/v2/runc" | 	"github.com/containerd/containerd/runtime/v2/runc" | ||||||
| @@ -73,7 +75,15 @@ type spec struct { | |||||||
|  |  | ||||||
| // New returns a new shim service that can be used via GRPC | // New returns a new shim service that can be used via GRPC | ||||||
| func New(ctx context.Context, id string, publisher shim.Publisher, shutdown func()) (shim.Shim, error) { | func New(ctx context.Context, id string, publisher shim.Publisher, shutdown func()) (shim.Shim, error) { | ||||||
| 	ep, err := oom.New(publisher) | 	var ( | ||||||
|  | 		ep  oom.Watcher | ||||||
|  | 		err error | ||||||
|  | 	) | ||||||
|  | 	if cgroups.Mode() == cgroups.Unified { | ||||||
|  | 		ep, err = oomv2.New(publisher) | ||||||
|  | 	} else { | ||||||
|  | 		ep, err = oomv1.New(publisher) | ||||||
|  | 	} | ||||||
| 	if err != nil { | 	if err != nil { | ||||||
| 		return nil, err | 		return nil, err | ||||||
| 	} | 	} | ||||||
| @@ -106,7 +116,7 @@ type service struct { | |||||||
| 	events   chan interface{} | 	events   chan interface{} | ||||||
| 	platform stdio.Platform | 	platform stdio.Platform | ||||||
| 	ec       chan runcC.Exit | 	ec       chan runcC.Exit | ||||||
| 	ep       *oom.Epoller | 	ep       oom.Watcher | ||||||
|  |  | ||||||
| 	// id only used in cleanup case | 	// id only used in cleanup case | ||||||
| 	id string | 	id string | ||||||
| @@ -344,9 +354,9 @@ func (s *service) Start(ctx context.Context, r *taskAPI.StartRequest) (*taskAPI. | |||||||
| 					logrus.WithError(err).Errorf("failed to enable controllers (%v)", allControllers) | 					logrus.WithError(err).Errorf("failed to enable controllers (%v)", allControllers) | ||||||
| 				} | 				} | ||||||
| 			} | 			} | ||||||
|  | 			if err := s.ep.Add(container.ID, cg); err != nil { | ||||||
| 			// OOM monitor is not implemented yet | 				logrus.WithError(err).Error("add cg to OOM monitor") | ||||||
| 			logrus.WithError(errdefs.ErrNotImplemented).Warn("add cg to OOM monitor") | 			} | ||||||
| 		} | 		} | ||||||
|  |  | ||||||
| 		s.send(&eventstypes.TaskStart{ | 		s.send(&eventstypes.TaskStart{ | ||||||
|   | |||||||
| @@ -2,7 +2,7 @@ github.com/beorn7/perks                             v1.0.1 | |||||||
| github.com/BurntSushi/toml                          v0.3.1 | github.com/BurntSushi/toml                          v0.3.1 | ||||||
| github.com/cespare/xxhash/v2                        v2.1.1 | github.com/cespare/xxhash/v2                        v2.1.1 | ||||||
| github.com/containerd/btrfs                         153935315f4ab9be5bf03650a1341454b05efa5d | github.com/containerd/btrfs                         153935315f4ab9be5bf03650a1341454b05efa5d | ||||||
| github.com/containerd/cgroups                       b4448137398923af7f4918b8b2ad8249172ca7a6 | github.com/containerd/cgroups                       0dbf7f05ba59274095946e2c0c89540726e8a8aa | ||||||
| github.com/containerd/console                       v1.0.0 | github.com/containerd/console                       v1.0.0 | ||||||
| github.com/containerd/continuity                    d3ef23f19fbb106bb73ffde425d07a9187e30745 | github.com/containerd/continuity                    d3ef23f19fbb106bb73ffde425d07a9187e30745 | ||||||
| github.com/containerd/fifo                          f15a3290365b9d2627d189e619ab4008e0069caf | github.com/containerd/fifo                          f15a3290365b9d2627d189e619ab4008e0069caf | ||||||
|   | |||||||
							
								
								
									
										21
									
								
								vendor/github.com/containerd/cgroups/README.md
									
									
									
										generated
									
									
										vendored
									
									
								
							
							
						
						
									
										21
									
								
								vendor/github.com/containerd/cgroups/README.md
									
									
									
										generated
									
									
										vendored
									
									
								
							| @@ -112,6 +112,27 @@ err := control.MoveTo(destination) | |||||||
| subCgroup, err := control.New("child", resources) | subCgroup, err := control.New("child", resources) | ||||||
| ``` | ``` | ||||||
|  |  | ||||||
|  | ### Registering for memory events | ||||||
|  |  | ||||||
|  | This allows you to get notified by an eventfd for v1 memory cgroups events. | ||||||
|  |  | ||||||
|  | ```go | ||||||
|  | event := cgroups.MemoryThresholdEvent(50 * 1024 * 1024, false) | ||||||
|  | efd, err := control.RegisterMemoryEvent(event) | ||||||
|  | ``` | ||||||
|  |  | ||||||
|  | ```go | ||||||
|  | event := cgroups.MemoryPressureEvent(cgroups.MediumPressure, cgroups.DefaultMode) | ||||||
|  | efd, err := control.RegisterMemoryEvent(event) | ||||||
|  | ``` | ||||||
|  |  | ||||||
|  | ```go | ||||||
|  | efd, err := control.OOMEventFD() | ||||||
|  | // or by using RegisterMemoryEvent | ||||||
|  | event := cgroups.OOMEvent() | ||||||
|  | efd, err := control.RegisterMemoryEvent(event) | ||||||
|  | ``` | ||||||
|  |  | ||||||
| ### Attention | ### Attention | ||||||
|  |  | ||||||
| All static path should not include `/sys/fs/cgroup/` prefix, it should start with your own cgroups name | All static path should not include `/sys/fs/cgroup/` prefix, it should start with your own cgroups name | ||||||
|   | |||||||
							
								
								
									
										21
									
								
								vendor/github.com/containerd/cgroups/cgroup.go
									
									
									
										generated
									
									
										vendored
									
									
								
							
							
						
						
									
										21
									
								
								vendor/github.com/containerd/cgroups/cgroup.go
									
									
									
										generated
									
									
										vendored
									
									
								
							| @@ -458,7 +458,26 @@ func (c *cgroup) OOMEventFD() (uintptr, error) { | |||||||
| 	if err != nil { | 	if err != nil { | ||||||
| 		return 0, err | 		return 0, err | ||||||
| 	} | 	} | ||||||
| 	return s.(*memoryController).OOMEventFD(sp) | 	return s.(*memoryController).memoryEvent(sp, OOMEvent()) | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // RegisterMemoryEvent allows the ability to register for all v1 memory cgroups | ||||||
|  | // notifications. | ||||||
|  | func (c *cgroup) RegisterMemoryEvent(event MemoryEvent) (uintptr, error) { | ||||||
|  | 	c.mu.Lock() | ||||||
|  | 	defer c.mu.Unlock() | ||||||
|  | 	if c.err != nil { | ||||||
|  | 		return 0, c.err | ||||||
|  | 	} | ||||||
|  | 	s := c.getSubsystem(Memory) | ||||||
|  | 	if s == nil { | ||||||
|  | 		return 0, ErrMemoryNotSupported | ||||||
|  | 	} | ||||||
|  | 	sp, err := c.path(Memory) | ||||||
|  | 	if err != nil { | ||||||
|  | 		return 0, err | ||||||
|  | 	} | ||||||
|  | 	return s.(*memoryController).memoryEvent(sp, event) | ||||||
| } | } | ||||||
|  |  | ||||||
| // State returns the state of the cgroup and its processes | // State returns the state of the cgroup and its processes | ||||||
|   | |||||||
							
								
								
									
										3
									
								
								vendor/github.com/containerd/cgroups/control.go
									
									
									
										generated
									
									
										vendored
									
									
								
							
							
						
						
									
										3
									
								
								vendor/github.com/containerd/cgroups/control.go
									
									
									
										generated
									
									
										vendored
									
									
								
							| @@ -82,6 +82,9 @@ type Cgroup interface { | |||||||
| 	Thaw() error | 	Thaw() error | ||||||
| 	// OOMEventFD returns the memory subsystem's event fd for OOM events | 	// OOMEventFD returns the memory subsystem's event fd for OOM events | ||||||
| 	OOMEventFD() (uintptr, error) | 	OOMEventFD() (uintptr, error) | ||||||
|  | 	// RegisterMemoryEvent returns the memory subsystems event fd for whatever memory event was | ||||||
|  | 	// registered for. Can alternatively register for the oom event with this method. | ||||||
|  | 	RegisterMemoryEvent(MemoryEvent) (uintptr, error) | ||||||
| 	// State returns the cgroups current state | 	// State returns the cgroups current state | ||||||
| 	State() State | 	State() State | ||||||
| 	// Subsystems returns all the subsystems in the cgroup | 	// Subsystems returns all the subsystems in the cgroup | ||||||
|   | |||||||
							
								
								
									
										2
									
								
								vendor/github.com/containerd/cgroups/go.mod
									
									
									
										generated
									
									
										vendored
									
									
								
							
							
						
						
									
										2
									
								
								vendor/github.com/containerd/cgroups/go.mod
									
									
									
										generated
									
									
										vendored
									
									
								
							| @@ -10,7 +10,7 @@ require ( | |||||||
| 	github.com/godbus/dbus/v5 v5.0.3 | 	github.com/godbus/dbus/v5 v5.0.3 | ||||||
| 	github.com/gogo/protobuf v1.3.1 | 	github.com/gogo/protobuf v1.3.1 | ||||||
| 	github.com/konsorten/go-windows-terminal-sequences v1.0.2 // indirect | 	github.com/konsorten/go-windows-terminal-sequences v1.0.2 // indirect | ||||||
| 	github.com/opencontainers/runtime-spec v0.1.2-0.20190507144316-5b71a03e2700 | 	github.com/opencontainers/runtime-spec v1.0.2 | ||||||
| 	github.com/pkg/errors v0.9.1 | 	github.com/pkg/errors v0.9.1 | ||||||
| 	github.com/sirupsen/logrus v1.4.2 | 	github.com/sirupsen/logrus v1.4.2 | ||||||
| 	github.com/stretchr/testify v1.2.2 | 	github.com/stretchr/testify v1.2.2 | ||||||
|   | |||||||
							
								
								
									
										2
									
								
								vendor/github.com/containerd/cgroups/hierarchy.go
									
									
									
										generated
									
									
										vendored
									
									
								
							
							
						
						
									
										2
									
								
								vendor/github.com/containerd/cgroups/hierarchy.go
									
									
									
										generated
									
									
										vendored
									
									
								
							| @@ -16,5 +16,5 @@ | |||||||
|  |  | ||||||
| package cgroups | package cgroups | ||||||
|  |  | ||||||
| // Hierarchy enableds both unified and split hierarchy for cgroups | // Hierarchy enables both unified and split hierarchy for cgroups | ||||||
| type Hierarchy func() ([]Subsystem, error) | type Hierarchy func() ([]Subsystem, error) | ||||||
|   | |||||||
							
								
								
									
										171
									
								
								vendor/github.com/containerd/cgroups/memory.go
									
									
									
										generated
									
									
										vendored
									
									
								
							
							
						
						
									
										171
									
								
								vendor/github.com/containerd/cgroups/memory.go
									
									
									
										generated
									
									
										vendored
									
									
								
							| @@ -32,6 +32,128 @@ import ( | |||||||
| 	"golang.org/x/sys/unix" | 	"golang.org/x/sys/unix" | ||||||
| ) | ) | ||||||
|  |  | ||||||
|  | // MemoryEvent is an interface that V1 memory Cgroup notifications implement. Arg returns the | ||||||
|  | // file name whose fd should be written to "cgroups.event_control". EventFile returns the name of | ||||||
|  | // the file that supports the notification api e.g. "memory.usage_in_bytes". | ||||||
|  | type MemoryEvent interface { | ||||||
|  | 	Arg() string | ||||||
|  | 	EventFile() string | ||||||
|  | } | ||||||
|  |  | ||||||
|  | type memoryThresholdEvent struct { | ||||||
|  | 	threshold uint64 | ||||||
|  | 	swap      bool | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // MemoryThresholdEvent returns a new memory threshold event to be used with RegisterMemoryEvent. | ||||||
|  | // If swap is true, the event will be registered using memory.memsw.usage_in_bytes | ||||||
|  | func MemoryThresholdEvent(threshold uint64, swap bool) MemoryEvent { | ||||||
|  | 	return &memoryThresholdEvent{ | ||||||
|  | 		threshold, | ||||||
|  | 		swap, | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (m *memoryThresholdEvent) Arg() string { | ||||||
|  | 	return strconv.FormatUint(m.threshold, 10) | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (m *memoryThresholdEvent) EventFile() string { | ||||||
|  | 	if m.swap { | ||||||
|  | 		return "memory.memsw.usage_in_bytes" | ||||||
|  | 	} | ||||||
|  | 	return "memory.usage_in_bytes" | ||||||
|  | } | ||||||
|  |  | ||||||
|  | type oomEvent struct{} | ||||||
|  |  | ||||||
|  | // OOMEvent returns a new oom event to be used with RegisterMemoryEvent. | ||||||
|  | func OOMEvent() MemoryEvent { | ||||||
|  | 	return &oomEvent{} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (oom *oomEvent) Arg() string { | ||||||
|  | 	return "" | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (oom *oomEvent) EventFile() string { | ||||||
|  | 	return "memory.oom_control" | ||||||
|  | } | ||||||
|  |  | ||||||
|  | type memoryPressureEvent struct { | ||||||
|  | 	pressureLevel MemoryPressureLevel | ||||||
|  | 	hierarchy     EventNotificationMode | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // MemoryPressureEvent returns a new memory pressure event to be used with RegisterMemoryEvent. | ||||||
|  | func MemoryPressureEvent(pressureLevel MemoryPressureLevel, hierarchy EventNotificationMode) MemoryEvent { | ||||||
|  | 	return &memoryPressureEvent{ | ||||||
|  | 		pressureLevel, | ||||||
|  | 		hierarchy, | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (m *memoryPressureEvent) Arg() string { | ||||||
|  | 	return string(m.pressureLevel) + "," + string(m.hierarchy) | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (m *memoryPressureEvent) EventFile() string { | ||||||
|  | 	return "memory.pressure_level" | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // MemoryPressureLevel corresponds to the memory pressure levels defined | ||||||
|  | // for memory cgroups. | ||||||
|  | type MemoryPressureLevel string | ||||||
|  |  | ||||||
|  | // The three memory pressure levels are as follows. | ||||||
|  | //  - The "low" level means that the system is reclaiming memory for new | ||||||
|  | //    allocations. Monitoring this reclaiming activity might be useful for | ||||||
|  | //    maintaining cache level. Upon notification, the program (typically | ||||||
|  | //    "Activity Manager") might analyze vmstat and act in advance (i.e. | ||||||
|  | //    prematurely shutdown unimportant services). | ||||||
|  | //  - The "medium" level means that the system is experiencing medium memory | ||||||
|  | //    pressure, the system might be making swap, paging out active file caches, | ||||||
|  | //    etc. Upon this event applications may decide to further analyze | ||||||
|  | //    vmstat/zoneinfo/memcg or internal memory usage statistics and free any | ||||||
|  | //    resources that can be easily reconstructed or re-read from a disk. | ||||||
|  | //  - The "critical" level means that the system is actively thrashing, it is | ||||||
|  | //    about to out of memory (OOM) or even the in-kernel OOM killer is on its | ||||||
|  | //    way to trigger. Applications should do whatever they can to help the | ||||||
|  | //    system. It might be too late to consult with vmstat or any other | ||||||
|  | //    statistics, so it is advisable to take an immediate action. | ||||||
|  | //    "https://www.kernel.org/doc/Documentation/cgroup-v1/memory.txt" Section 11 | ||||||
|  | const ( | ||||||
|  | 	LowPressure      MemoryPressureLevel = "low" | ||||||
|  | 	MediumPressure   MemoryPressureLevel = "medium" | ||||||
|  | 	CriticalPressure MemoryPressureLevel = "critical" | ||||||
|  | ) | ||||||
|  |  | ||||||
|  | // EventNotificationMode corresponds to the notification modes | ||||||
|  | // for the memory cgroups pressure level notifications. | ||||||
|  | type EventNotificationMode string | ||||||
|  |  | ||||||
|  | // There are three optional modes that specify different propagation behavior: | ||||||
|  | //  - "default": this is the default behavior specified above. This mode is the | ||||||
|  | //    same as omitting the optional mode parameter, preserved by backwards | ||||||
|  | //    compatibility. | ||||||
|  | //  - "hierarchy": events always propagate up to the root, similar to the default | ||||||
|  | //    behavior, except that propagation continues regardless of whether there are | ||||||
|  | //    event listeners at each level, with the "hierarchy" mode. In the above | ||||||
|  | //    example, groups A, B, and C will receive notification of memory pressure. | ||||||
|  | //  - "local": events are pass-through, i.e. they only receive notifications when | ||||||
|  | //    memory pressure is experienced in the memcg for which the notification is | ||||||
|  | //    registered. In the above example, group C will receive notification if | ||||||
|  | //    registered for "local" notification and the group experiences memory | ||||||
|  | //    pressure. However, group B will never receive notification, regardless if | ||||||
|  | //    there is an event listener for group C or not, if group B is registered for | ||||||
|  | //    local notification. | ||||||
|  | //    "https://www.kernel.org/doc/Documentation/cgroup-v1/memory.txt" Section 11 | ||||||
|  | const ( | ||||||
|  | 	DefaultMode   EventNotificationMode = "default" | ||||||
|  | 	LocalMode     EventNotificationMode = "local" | ||||||
|  | 	HierarchyMode EventNotificationMode = "hierarchy" | ||||||
|  | ) | ||||||
|  |  | ||||||
| // NewMemory returns a Memory controller given the root folder of cgroups. | // NewMemory returns a Memory controller given the root folder of cgroups. | ||||||
| // It may optionally accept other configuration options, such as IgnoreModules(...) | // It may optionally accept other configuration options, such as IgnoreModules(...) | ||||||
| func NewMemory(root string, options ...func(*memoryController)) *memoryController { | func NewMemory(root string, options ...func(*memoryController)) *memoryController { | ||||||
| @@ -201,34 +323,6 @@ func (m *memoryController) Stat(path string, stats *v1.Metrics) error { | |||||||
| 	return nil | 	return nil | ||||||
| } | } | ||||||
|  |  | ||||||
| func (m *memoryController) OOMEventFD(path string) (uintptr, error) { |  | ||||||
| 	root := m.Path(path) |  | ||||||
| 	f, err := os.Open(filepath.Join(root, "memory.oom_control")) |  | ||||||
| 	if err != nil { |  | ||||||
| 		return 0, err |  | ||||||
| 	} |  | ||||||
| 	defer f.Close() |  | ||||||
| 	fd, _, serr := unix.RawSyscall(unix.SYS_EVENTFD2, 0, unix.EFD_CLOEXEC, 0) |  | ||||||
| 	if serr != 0 { |  | ||||||
| 		return 0, serr |  | ||||||
| 	} |  | ||||||
| 	if err := writeEventFD(root, f.Fd(), fd); err != nil { |  | ||||||
| 		unix.Close(int(fd)) |  | ||||||
| 		return 0, err |  | ||||||
| 	} |  | ||||||
| 	return fd, nil |  | ||||||
| } |  | ||||||
|  |  | ||||||
| func writeEventFD(root string, cfd, efd uintptr) error { |  | ||||||
| 	f, err := os.OpenFile(filepath.Join(root, "cgroup.event_control"), os.O_WRONLY, 0) |  | ||||||
| 	if err != nil { |  | ||||||
| 		return err |  | ||||||
| 	} |  | ||||||
| 	_, err = f.WriteString(fmt.Sprintf("%d %d", efd, cfd)) |  | ||||||
| 	f.Close() |  | ||||||
| 	return err |  | ||||||
| } |  | ||||||
|  |  | ||||||
| func (m *memoryController) parseStats(r io.Reader, stat *v1.MemoryStat) error { | func (m *memoryController) parseStats(r io.Reader, stat *v1.MemoryStat) error { | ||||||
| 	var ( | 	var ( | ||||||
| 		raw  = make(map[string]uint64) | 		raw  = make(map[string]uint64) | ||||||
| @@ -359,3 +453,24 @@ func getOomControlValue(mem *specs.LinuxMemory) *int64 { | |||||||
| 	} | 	} | ||||||
| 	return nil | 	return nil | ||||||
| } | } | ||||||
|  |  | ||||||
|  | func (m *memoryController) memoryEvent(path string, event MemoryEvent) (uintptr, error) { | ||||||
|  | 	root := m.Path(path) | ||||||
|  | 	efd, err := unix.Eventfd(0, unix.EFD_CLOEXEC) | ||||||
|  | 	if err != nil { | ||||||
|  | 		return 0, err | ||||||
|  | 	} | ||||||
|  | 	evtFile, err := os.Open(filepath.Join(root, event.EventFile())) | ||||||
|  | 	if err != nil { | ||||||
|  | 		unix.Close(efd) | ||||||
|  | 		return 0, err | ||||||
|  | 	} | ||||||
|  | 	defer evtFile.Close() | ||||||
|  | 	data := fmt.Sprintf("%d %d %s", efd, evtFile.Fd(), event.Arg()) | ||||||
|  | 	evctlPath := filepath.Join(root, "cgroup.event_control") | ||||||
|  | 	if err := ioutil.WriteFile(evctlPath, []byte(data), 0700); err != nil { | ||||||
|  | 		unix.Close(efd) | ||||||
|  | 		return 0, err | ||||||
|  | 	} | ||||||
|  | 	return uintptr(efd), nil | ||||||
|  | } | ||||||
|   | |||||||
							
								
								
									
										64
									
								
								vendor/github.com/containerd/cgroups/v2/manager.go
									
									
									
										generated
									
									
										vendored
									
									
								
							
							
						
						
									
										64
									
								
								vendor/github.com/containerd/cgroups/v2/manager.go
									
									
									
										generated
									
									
										vendored
									
									
								
							| @@ -166,6 +166,9 @@ func writeValues(path string, values []Value) error { | |||||||
| } | } | ||||||
|  |  | ||||||
| func NewManager(mountpoint string, group string, resources *Resources) (*Manager, error) { | func NewManager(mountpoint string, group string, resources *Resources) (*Manager, error) { | ||||||
|  | 	if resources == nil { | ||||||
|  | 		return nil, errors.New("resources reference is nil") | ||||||
|  | 	} | ||||||
| 	if err := VerifyGroupPath(group); err != nil { | 	if err := VerifyGroupPath(group); err != nil { | ||||||
| 		return nil, err | 		return nil, err | ||||||
| 	} | 	} | ||||||
| @@ -376,6 +379,12 @@ func (c *Manager) Stat() (*stats.Metrics, error) { | |||||||
| 			return nil, err | 			return nil, err | ||||||
| 		} | 		} | ||||||
| 	} | 	} | ||||||
|  | 	memoryEvents := make(map[string]interface{}) | ||||||
|  | 	if err := readKVStatsFile(c.path, "memory.events", memoryEvents); err != nil { | ||||||
|  | 		if !os.IsNotExist(err) { | ||||||
|  | 			return nil, err | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
| 	var metrics stats.Metrics | 	var metrics stats.Metrics | ||||||
|  |  | ||||||
| 	metrics.Pids = &stats.PidsStat{ | 	metrics.Pids = &stats.PidsStat{ | ||||||
| @@ -427,7 +436,15 @@ func (c *Manager) Stat() (*stats.Metrics, error) { | |||||||
| 		SwapUsage:             getStatFileContentUint64(filepath.Join(c.path, "memory.swap.current")), | 		SwapUsage:             getStatFileContentUint64(filepath.Join(c.path, "memory.swap.current")), | ||||||
| 		SwapLimit:             getStatFileContentUint64(filepath.Join(c.path, "memory.swap.max")), | 		SwapLimit:             getStatFileContentUint64(filepath.Join(c.path, "memory.swap.max")), | ||||||
| 	} | 	} | ||||||
|  | 	if len(memoryEvents) > 0 { | ||||||
|  | 		metrics.MemoryEvents = &stats.MemoryEvents{ | ||||||
|  | 			Low:     getUint64Value("low", memoryEvents), | ||||||
|  | 			High:    getUint64Value("high", memoryEvents), | ||||||
|  | 			Max:     getUint64Value("max", memoryEvents), | ||||||
|  | 			Oom:     getUint64Value("oom", memoryEvents), | ||||||
|  | 			OomKill: getUint64Value("oom_kill", memoryEvents), | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
| 	metrics.Io = &stats.IOStat{Usage: readIoStats(c.path)} | 	metrics.Io = &stats.IOStat{Usage: readIoStats(c.path)} | ||||||
| 	metrics.Rdma = &stats.RdmaStat{ | 	metrics.Rdma = &stats.RdmaStat{ | ||||||
| 		Current: rdmaStats(filepath.Join(c.path, "rdma.current")), | 		Current: rdmaStats(filepath.Join(c.path, "rdma.current")), | ||||||
| @@ -572,15 +589,44 @@ func (c *Manager) waitForEvents(ec chan<- Event, errCh chan<- error) { | |||||||
| 			errCh <- err | 			errCh <- err | ||||||
| 			return | 			return | ||||||
| 		} | 		} | ||||||
| 		var out map[string]interface{} |  | ||||||
| 		if bytesRead >= syscall.SizeofInotifyEvent { | 		if bytesRead >= syscall.SizeofInotifyEvent { | ||||||
| 			if err := readKVStatsFile(c.path, "memory.events", out); err != nil { | 			out := make(map[string]interface{}) | ||||||
| 				e := Event{ | 			if err := readKVStatsFile(c.path, "memory.events", out); err == nil { | ||||||
| 					High:    out["high"].(uint64), | 				e := Event{} | ||||||
| 					Low:     out["low"].(uint64), | 				if v, ok := out["high"]; ok { | ||||||
| 					Max:     out["max"].(uint64), | 					e.High, ok = v.(uint64) | ||||||
| 					OOM:     out["oom"].(uint64), | 					if !ok { | ||||||
| 					OOMKill: out["oom_kill"].(uint64), | 						errCh <- errors.Errorf("cannot convert high to uint64: %+v", v) | ||||||
|  | 						return | ||||||
|  | 					} | ||||||
|  | 				} | ||||||
|  | 				if v, ok := out["low"]; ok { | ||||||
|  | 					e.Low, ok = v.(uint64) | ||||||
|  | 					if !ok { | ||||||
|  | 						errCh <- errors.Errorf("cannot convert low to uint64: %+v", v) | ||||||
|  | 						return | ||||||
|  | 					} | ||||||
|  | 				} | ||||||
|  | 				if v, ok := out["max"]; ok { | ||||||
|  | 					e.Max, ok = v.(uint64) | ||||||
|  | 					if !ok { | ||||||
|  | 						errCh <- errors.Errorf("cannot convert max to uint64: %+v", v) | ||||||
|  | 						return | ||||||
|  | 					} | ||||||
|  | 				} | ||||||
|  | 				if v, ok := out["oom"]; ok { | ||||||
|  | 					e.OOM, ok = v.(uint64) | ||||||
|  | 					if !ok { | ||||||
|  | 						errCh <- errors.Errorf("cannot convert oom to uint64: %+v", v) | ||||||
|  | 						return | ||||||
|  | 					} | ||||||
|  | 				} | ||||||
|  | 				if v, ok := out["oom_kill"]; ok { | ||||||
|  | 					e.OOMKill, ok = v.(uint64) | ||||||
|  | 					if !ok { | ||||||
|  | 						errCh <- errors.Errorf("cannot convert oom_kill to uint64: %+v", v) | ||||||
|  | 						return | ||||||
|  | 					} | ||||||
| 				} | 				} | ||||||
| 				ec <- e | 				ec <- e | ||||||
| 			} else { | 			} else { | ||||||
|   | |||||||
							
								
								
									
										490
									
								
								vendor/github.com/containerd/cgroups/v2/stats/metrics.pb.go
									
									
									
										generated
									
									
										vendored
									
									
								
							
							
						
						
									
										490
									
								
								vendor/github.com/containerd/cgroups/v2/stats/metrics.pb.go
									
									
									
										generated
									
									
										vendored
									
									
								
							| @@ -31,6 +31,7 @@ type Metrics struct { | |||||||
| 	Rdma                 *RdmaStat      `protobuf:"bytes,5,opt,name=rdma,proto3" json:"rdma,omitempty"` | 	Rdma                 *RdmaStat      `protobuf:"bytes,5,opt,name=rdma,proto3" json:"rdma,omitempty"` | ||||||
| 	Io                   *IOStat        `protobuf:"bytes,6,opt,name=io,proto3" json:"io,omitempty"` | 	Io                   *IOStat        `protobuf:"bytes,6,opt,name=io,proto3" json:"io,omitempty"` | ||||||
| 	Hugetlb              []*HugeTlbStat `protobuf:"bytes,7,rep,name=hugetlb,proto3" json:"hugetlb,omitempty"` | 	Hugetlb              []*HugeTlbStat `protobuf:"bytes,7,rep,name=hugetlb,proto3" json:"hugetlb,omitempty"` | ||||||
|  | 	MemoryEvents         *MemoryEvents  `protobuf:"bytes,8,opt,name=memory_events,json=memoryEvents,proto3" json:"memory_events,omitempty"` | ||||||
| 	XXX_NoUnkeyedLiteral struct{}       `json:"-"` | 	XXX_NoUnkeyedLiteral struct{}       `json:"-"` | ||||||
| 	XXX_unrecognized     []byte         `json:"-"` | 	XXX_unrecognized     []byte         `json:"-"` | ||||||
| 	XXX_sizecache        int32          `json:"-"` | 	XXX_sizecache        int32          `json:"-"` | ||||||
| @@ -225,6 +226,49 @@ func (m *MemoryStat) XXX_DiscardUnknown() { | |||||||
|  |  | ||||||
| var xxx_messageInfo_MemoryStat proto.InternalMessageInfo | var xxx_messageInfo_MemoryStat proto.InternalMessageInfo | ||||||
|  |  | ||||||
|  | type MemoryEvents struct { | ||||||
|  | 	Low                  uint64   `protobuf:"varint,1,opt,name=low,proto3" json:"low,omitempty"` | ||||||
|  | 	High                 uint64   `protobuf:"varint,2,opt,name=high,proto3" json:"high,omitempty"` | ||||||
|  | 	Max                  uint64   `protobuf:"varint,3,opt,name=max,proto3" json:"max,omitempty"` | ||||||
|  | 	Oom                  uint64   `protobuf:"varint,4,opt,name=oom,proto3" json:"oom,omitempty"` | ||||||
|  | 	OomKill              uint64   `protobuf:"varint,5,opt,name=oom_kill,json=oomKill,proto3" json:"oom_kill,omitempty"` | ||||||
|  | 	XXX_NoUnkeyedLiteral struct{} `json:"-"` | ||||||
|  | 	XXX_unrecognized     []byte   `json:"-"` | ||||||
|  | 	XXX_sizecache        int32    `json:"-"` | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (m *MemoryEvents) Reset()      { *m = MemoryEvents{} } | ||||||
|  | func (*MemoryEvents) ProtoMessage() {} | ||||||
|  | func (*MemoryEvents) Descriptor() ([]byte, []int) { | ||||||
|  | 	return fileDescriptor_2fc6005842049e6b, []int{4} | ||||||
|  | } | ||||||
|  | func (m *MemoryEvents) XXX_Unmarshal(b []byte) error { | ||||||
|  | 	return m.Unmarshal(b) | ||||||
|  | } | ||||||
|  | func (m *MemoryEvents) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { | ||||||
|  | 	if deterministic { | ||||||
|  | 		return xxx_messageInfo_MemoryEvents.Marshal(b, m, deterministic) | ||||||
|  | 	} else { | ||||||
|  | 		b = b[:cap(b)] | ||||||
|  | 		n, err := m.MarshalTo(b) | ||||||
|  | 		if err != nil { | ||||||
|  | 			return nil, err | ||||||
|  | 		} | ||||||
|  | 		return b[:n], nil | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | func (m *MemoryEvents) XXX_Merge(src proto.Message) { | ||||||
|  | 	xxx_messageInfo_MemoryEvents.Merge(m, src) | ||||||
|  | } | ||||||
|  | func (m *MemoryEvents) XXX_Size() int { | ||||||
|  | 	return m.Size() | ||||||
|  | } | ||||||
|  | func (m *MemoryEvents) XXX_DiscardUnknown() { | ||||||
|  | 	xxx_messageInfo_MemoryEvents.DiscardUnknown(m) | ||||||
|  | } | ||||||
|  |  | ||||||
|  | var xxx_messageInfo_MemoryEvents proto.InternalMessageInfo | ||||||
|  |  | ||||||
| type RdmaStat struct { | type RdmaStat struct { | ||||||
| 	Current              []*RdmaEntry `protobuf:"bytes,1,rep,name=current,proto3" json:"current,omitempty"` | 	Current              []*RdmaEntry `protobuf:"bytes,1,rep,name=current,proto3" json:"current,omitempty"` | ||||||
| 	Limit                []*RdmaEntry `protobuf:"bytes,2,rep,name=limit,proto3" json:"limit,omitempty"` | 	Limit                []*RdmaEntry `protobuf:"bytes,2,rep,name=limit,proto3" json:"limit,omitempty"` | ||||||
| @@ -236,7 +280,7 @@ type RdmaStat struct { | |||||||
| func (m *RdmaStat) Reset()      { *m = RdmaStat{} } | func (m *RdmaStat) Reset()      { *m = RdmaStat{} } | ||||||
| func (*RdmaStat) ProtoMessage() {} | func (*RdmaStat) ProtoMessage() {} | ||||||
| func (*RdmaStat) Descriptor() ([]byte, []int) { | func (*RdmaStat) Descriptor() ([]byte, []int) { | ||||||
| 	return fileDescriptor_2fc6005842049e6b, []int{4} | 	return fileDescriptor_2fc6005842049e6b, []int{5} | ||||||
| } | } | ||||||
| func (m *RdmaStat) XXX_Unmarshal(b []byte) error { | func (m *RdmaStat) XXX_Unmarshal(b []byte) error { | ||||||
| 	return m.Unmarshal(b) | 	return m.Unmarshal(b) | ||||||
| @@ -277,7 +321,7 @@ type RdmaEntry struct { | |||||||
| func (m *RdmaEntry) Reset()      { *m = RdmaEntry{} } | func (m *RdmaEntry) Reset()      { *m = RdmaEntry{} } | ||||||
| func (*RdmaEntry) ProtoMessage() {} | func (*RdmaEntry) ProtoMessage() {} | ||||||
| func (*RdmaEntry) Descriptor() ([]byte, []int) { | func (*RdmaEntry) Descriptor() ([]byte, []int) { | ||||||
| 	return fileDescriptor_2fc6005842049e6b, []int{5} | 	return fileDescriptor_2fc6005842049e6b, []int{6} | ||||||
| } | } | ||||||
| func (m *RdmaEntry) XXX_Unmarshal(b []byte) error { | func (m *RdmaEntry) XXX_Unmarshal(b []byte) error { | ||||||
| 	return m.Unmarshal(b) | 	return m.Unmarshal(b) | ||||||
| @@ -316,7 +360,7 @@ type IOStat struct { | |||||||
| func (m *IOStat) Reset()      { *m = IOStat{} } | func (m *IOStat) Reset()      { *m = IOStat{} } | ||||||
| func (*IOStat) ProtoMessage() {} | func (*IOStat) ProtoMessage() {} | ||||||
| func (*IOStat) Descriptor() ([]byte, []int) { | func (*IOStat) Descriptor() ([]byte, []int) { | ||||||
| 	return fileDescriptor_2fc6005842049e6b, []int{6} | 	return fileDescriptor_2fc6005842049e6b, []int{7} | ||||||
| } | } | ||||||
| func (m *IOStat) XXX_Unmarshal(b []byte) error { | func (m *IOStat) XXX_Unmarshal(b []byte) error { | ||||||
| 	return m.Unmarshal(b) | 	return m.Unmarshal(b) | ||||||
| @@ -360,7 +404,7 @@ type IOEntry struct { | |||||||
| func (m *IOEntry) Reset()      { *m = IOEntry{} } | func (m *IOEntry) Reset()      { *m = IOEntry{} } | ||||||
| func (*IOEntry) ProtoMessage() {} | func (*IOEntry) ProtoMessage() {} | ||||||
| func (*IOEntry) Descriptor() ([]byte, []int) { | func (*IOEntry) Descriptor() ([]byte, []int) { | ||||||
| 	return fileDescriptor_2fc6005842049e6b, []int{7} | 	return fileDescriptor_2fc6005842049e6b, []int{8} | ||||||
| } | } | ||||||
| func (m *IOEntry) XXX_Unmarshal(b []byte) error { | func (m *IOEntry) XXX_Unmarshal(b []byte) error { | ||||||
| 	return m.Unmarshal(b) | 	return m.Unmarshal(b) | ||||||
| @@ -401,7 +445,7 @@ type HugeTlbStat struct { | |||||||
| func (m *HugeTlbStat) Reset()      { *m = HugeTlbStat{} } | func (m *HugeTlbStat) Reset()      { *m = HugeTlbStat{} } | ||||||
| func (*HugeTlbStat) ProtoMessage() {} | func (*HugeTlbStat) ProtoMessage() {} | ||||||
| func (*HugeTlbStat) Descriptor() ([]byte, []int) { | func (*HugeTlbStat) Descriptor() ([]byte, []int) { | ||||||
| 	return fileDescriptor_2fc6005842049e6b, []int{8} | 	return fileDescriptor_2fc6005842049e6b, []int{9} | ||||||
| } | } | ||||||
| func (m *HugeTlbStat) XXX_Unmarshal(b []byte) error { | func (m *HugeTlbStat) XXX_Unmarshal(b []byte) error { | ||||||
| 	return m.Unmarshal(b) | 	return m.Unmarshal(b) | ||||||
| @@ -435,6 +479,7 @@ func init() { | |||||||
| 	proto.RegisterType((*PidsStat)(nil), "io.containerd.cgroups.v2.PidsStat") | 	proto.RegisterType((*PidsStat)(nil), "io.containerd.cgroups.v2.PidsStat") | ||||||
| 	proto.RegisterType((*CPUStat)(nil), "io.containerd.cgroups.v2.CPUStat") | 	proto.RegisterType((*CPUStat)(nil), "io.containerd.cgroups.v2.CPUStat") | ||||||
| 	proto.RegisterType((*MemoryStat)(nil), "io.containerd.cgroups.v2.MemoryStat") | 	proto.RegisterType((*MemoryStat)(nil), "io.containerd.cgroups.v2.MemoryStat") | ||||||
|  | 	proto.RegisterType((*MemoryEvents)(nil), "io.containerd.cgroups.v2.MemoryEvents") | ||||||
| 	proto.RegisterType((*RdmaStat)(nil), "io.containerd.cgroups.v2.RdmaStat") | 	proto.RegisterType((*RdmaStat)(nil), "io.containerd.cgroups.v2.RdmaStat") | ||||||
| 	proto.RegisterType((*RdmaEntry)(nil), "io.containerd.cgroups.v2.RdmaEntry") | 	proto.RegisterType((*RdmaEntry)(nil), "io.containerd.cgroups.v2.RdmaEntry") | ||||||
| 	proto.RegisterType((*IOStat)(nil), "io.containerd.cgroups.v2.IOStat") | 	proto.RegisterType((*IOStat)(nil), "io.containerd.cgroups.v2.IOStat") | ||||||
| @@ -447,77 +492,82 @@ func init() { | |||||||
| } | } | ||||||
|  |  | ||||||
| var fileDescriptor_2fc6005842049e6b = []byte{ | var fileDescriptor_2fc6005842049e6b = []byte{ | ||||||
| 	// 1118 bytes of a gzipped FileDescriptorProto | 	// 1198 bytes of a gzipped FileDescriptorProto | ||||||
| 	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x56, 0x4d, 0x6f, 0x1c, 0x45, | 	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x56, 0x4d, 0x73, 0xd4, 0x46, | ||||||
| 	0x10, 0xcd, 0xda, 0x9b, 0xfd, 0xe8, 0xb5, 0x13, 0xa7, 0xe3, 0x84, 0x4e, 0x42, 0xd6, 0xf6, 0x86, | 	0x13, 0x66, 0xed, 0xc5, 0xeb, 0xed, 0xb5, 0xc1, 0x0c, 0x86, 0x57, 0xc0, 0xcb, 0xda, 0x5e, 0x02, | ||||||
| 	0xa0, 0x20, 0xc1, 0x2e, 0x32, 0x5f, 0x02, 0x05, 0x21, 0x27, 0x10, 0x05, 0x09, 0x13, 0x6b, 0xe2, | 	0x45, 0xaa, 0x92, 0xdd, 0x94, 0xf3, 0x55, 0x49, 0x91, 0x4a, 0x19, 0x02, 0x45, 0x8a, 0x10, 0x5c, | ||||||
| 	0x15, 0xc7, 0x51, 0xef, 0x4c, 0x7b, 0x76, 0xec, 0xf9, 0x52, 0x77, 0x8f, 0xcd, 0xe6, 0xc4, 0x81, | 	0x02, 0x57, 0x8e, 0xaa, 0x59, 0x69, 0x2c, 0x0d, 0x96, 0x34, 0xaa, 0x99, 0x91, 0x1d, 0x73, 0xca, | ||||||
| 	0x2b, 0xe2, 0x6f, 0xe5, 0x06, 0x47, 0x4e, 0x88, 0xf8, 0xc4, 0xcf, 0x40, 0x55, 0xd5, 0xb3, 0x33, | 	0x21, 0xd7, 0x54, 0x7e, 0x4d, 0xfe, 0x03, 0xb7, 0xe4, 0x98, 0x53, 0x2a, 0xf8, 0x97, 0xa4, 0xba, | ||||||
| 	0x1c, 0x6c, 0xb8, 0x75, 0xbd, 0x7a, 0xaf, 0xa6, 0xfa, 0xf5, 0x76, 0xf5, 0xb2, 0x8f, 0xa3, 0xd8, | 	0x67, 0x64, 0x29, 0x07, 0x43, 0x6e, 0xd3, 0x4f, 0x3f, 0xdd, 0xea, 0x8f, 0x99, 0x6e, 0xc1, 0x27, | ||||||
| 	0xce, 0xcb, 0xd9, 0x38, 0xc8, 0xd3, 0x49, 0x90, 0x67, 0x56, 0xc6, 0x99, 0xd2, 0xe1, 0x24, 0x88, | 	0xa9, 0xb4, 0x59, 0x3d, 0x9f, 0xc6, 0xaa, 0x98, 0xc5, 0xaa, 0xb4, 0x5c, 0x96, 0x42, 0x27, 0xb3, | ||||||
| 	0x74, 0x5e, 0x16, 0x66, 0x72, 0xba, 0x3b, 0x31, 0x56, 0x5a, 0x33, 0x49, 0x95, 0xd5, 0x71, 0x60, | 	0x38, 0xd5, 0xaa, 0xae, 0xcc, 0xec, 0x70, 0x7b, 0x66, 0x2c, 0xb7, 0x66, 0x56, 0x08, 0xab, 0x65, | ||||||
| 	0xc6, 0x85, 0xce, 0x6d, 0xce, 0x45, 0x9c, 0x8f, 0x6b, 0xf6, 0xd8, 0xb1, 0xc7, 0xa7, 0xbb, 0x77, | 	0x6c, 0xa6, 0x95, 0x56, 0x56, 0xb1, 0x40, 0xaa, 0x69, 0xcb, 0x9e, 0x7a, 0xf6, 0xf4, 0x70, 0xfb, | ||||||
| 	0x37, 0xa3, 0x3c, 0xca, 0x91, 0x34, 0x81, 0x15, 0xf1, 0x47, 0x7f, 0xaf, 0xb0, 0xee, 0x3e, 0x55, | 	0xfa, 0x7a, 0xaa, 0x52, 0x45, 0xa4, 0x19, 0x9e, 0x1c, 0x7f, 0xf2, 0xdb, 0x22, 0x0c, 0x9e, 0x3a, | ||||||
| 	0xe0, 0x9f, 0xb2, 0x76, 0x11, 0x87, 0x46, 0xb4, 0xb6, 0x5b, 0x8f, 0x06, 0xbb, 0xa3, 0xf1, 0x45, | 	0x0f, 0xec, 0x33, 0xe8, 0x57, 0x32, 0x31, 0x41, 0x6f, 0xb3, 0x77, 0x77, 0xb4, 0x3d, 0x99, 0x9e, | ||||||
| 	0xa5, 0xc6, 0x07, 0x71, 0x68, 0x5e, 0x5a, 0x69, 0x3d, 0xe4, 0xf3, 0xc7, 0x6c, 0x35, 0x28, 0x4a, | 	0xe5, 0x6a, 0xba, 0x2b, 0x13, 0xf3, 0xdc, 0x72, 0x1b, 0x12, 0x9f, 0xdd, 0x83, 0xc5, 0xb8, 0xaa, | ||||||
| 	0xb1, 0x82, 0xb2, 0x9d, 0x8b, 0x65, 0x4f, 0x0f, 0xa6, 0xa0, 0x7a, 0xd2, 0x3d, 0xff, 0x73, 0x6b, | 	0x83, 0x05, 0x32, 0xdb, 0x3a, 0xdb, 0xec, 0xc1, 0xee, 0x1e, 0x5a, 0xdd, 0x1f, 0x9c, 0xfc, 0xb5, | ||||||
| 	0xf5, 0xe9, 0xc1, 0xd4, 0x03, 0x19, 0x7f, 0xcc, 0x3a, 0xa9, 0x4a, 0x73, 0xbd, 0x10, 0x6d, 0x2c, | 	0xb1, 0xf8, 0x60, 0x77, 0x2f, 0x44, 0x33, 0x76, 0x0f, 0x96, 0x0a, 0x51, 0x28, 0x7d, 0x1c, 0xf4, | ||||||
| 	0xf0, 0xce, 0xc5, 0x05, 0xf6, 0x91, 0x87, 0x5f, 0x76, 0x1a, 0xe8, 0x59, 0x87, 0xa9, 0x14, 0x57, | 	0xc9, 0xc1, 0x7b, 0x67, 0x3b, 0x78, 0x4a, 0x3c, 0xfa, 0xb2, 0xb7, 0xc1, 0x98, 0x75, 0x52, 0xf0, | ||||||
| 	0xff, 0xab, 0x67, 0x2f, 0x4c, 0x25, 0xf5, 0x0c, 0x7c, 0xfe, 0x21, 0x5b, 0x89, 0x73, 0xd1, 0x41, | 	0xe0, 0xfc, 0xbb, 0x62, 0x0e, 0x93, 0x82, 0xbb, 0x98, 0x91, 0xcf, 0x3e, 0x82, 0x05, 0xa9, 0x82, | ||||||
| 	0xd5, 0xf6, 0xc5, 0xaa, 0x6f, 0x5f, 0xa0, 0x66, 0x25, 0xce, 0xf9, 0x57, 0xac, 0x3b, 0x2f, 0x23, | 	0x25, 0xb2, 0xda, 0x3c, 0xdb, 0xea, 0xdb, 0x67, 0x64, 0xb3, 0x20, 0x15, 0xfb, 0x1a, 0x06, 0x59, | ||||||
| 	0x65, 0x93, 0x99, 0xe8, 0x6e, 0xaf, 0x3e, 0x1a, 0xec, 0x3e, 0xbc, 0x58, 0xf6, 0xbc, 0x8c, 0xd4, | 	0x9d, 0x0a, 0x9b, 0xcf, 0x83, 0xc1, 0xe6, 0xe2, 0xdd, 0xd1, 0xf6, 0xed, 0xb3, 0xcd, 0x1e, 0xd7, | ||||||
| 	0x61, 0x32, 0x43, 0x6d, 0xa5, 0x1a, 0x7d, 0xc1, 0x7a, 0x95, 0x71, 0x5c, 0xb0, 0x6e, 0x50, 0x6a, | 	0xa9, 0x78, 0x91, 0xcf, 0xc9, 0xb6, 0xb1, 0x62, 0x4f, 0x60, 0xd5, 0x05, 0x1d, 0x89, 0x43, 0x51, | ||||||
| 	0xad, 0x32, 0x8b, 0x6e, 0xb7, 0xbd, 0x2a, 0xe4, 0x9b, 0xec, 0x6a, 0x12, 0xa7, 0xb1, 0x45, 0x3b, | 	0x5a, 0x13, 0x2c, 0xd3, 0xd7, 0xef, 0xbc, 0x2b, 0xdf, 0x87, 0xc4, 0x0e, 0x57, 0x8a, 0x8e, 0x34, | ||||||
| 	0xdb, 0x1e, 0x05, 0xa3, 0xdf, 0x5a, 0xac, 0xeb, 0xec, 0xe3, 0xf7, 0x19, 0x2b, 0x8d, 0x8c, 0x94, | 	0xf9, 0x12, 0x96, 0x9b, 0x2e, 0xb0, 0x00, 0x06, 0x71, 0xad, 0xb5, 0x28, 0x2d, 0xb5, 0xae, 0x1f, | ||||||
| 	0x5f, 0x1a, 0x15, 0x38, 0x79, 0x1f, 0x91, 0xa9, 0x51, 0x01, 0xbf, 0xc7, 0xfa, 0xa5, 0x51, 0x9a, | 	0x36, 0x22, 0x5b, 0x87, 0xf3, 0xb9, 0x2c, 0xa4, 0xa5, 0xde, 0xf4, 0x43, 0x27, 0x4c, 0x7e, 0xef, | ||||||
| 	0xb2, 0x54, 0xa4, 0x07, 0x00, 0x26, 0xb7, 0xd8, 0xc0, 0x2c, 0x8c, 0x55, 0x29, 0xa5, 0x57, 0x31, | 	0xc1, 0xc0, 0xf7, 0x82, 0xdd, 0x04, 0xa8, 0x0d, 0x4f, 0x45, 0x54, 0x1b, 0x11, 0x7b, 0xf3, 0x21, | ||||||
| 	0xcd, 0x08, 0x42, 0xc2, 0x7d, 0xc6, 0x32, 0xed, 0x17, 0x4a, 0xc7, 0x79, 0x68, 0xf0, 0x44, 0xda, | 	0x21, 0x7b, 0x46, 0xc4, 0xec, 0x06, 0x0c, 0x6b, 0x23, 0xb4, 0xd3, 0x3a, 0x27, 0xcb, 0x08, 0x90, | ||||||
| 	0x5e, 0x3f, 0xd3, 0x07, 0x04, 0xf0, 0x1d, 0xb6, 0x96, 0x69, 0xdf, 0xce, 0x75, 0x6e, 0x6d, 0xa2, | 	0x72, 0x03, 0x46, 0xe6, 0xd8, 0x58, 0x51, 0x38, 0xf5, 0x22, 0xa9, 0xc1, 0x41, 0x44, 0xb8, 0x09, | ||||||
| 	0x42, 0xb4, 0xbd, 0xed, 0x0d, 0x32, 0x7d, 0x58, 0x41, 0xfc, 0x21, 0xbb, 0xb6, 0xcc, 0xd3, 0x57, | 	0x50, 0xea, 0xa8, 0x12, 0x5a, 0xaa, 0xc4, 0x50, 0x7b, 0xfb, 0xe1, 0xb0, 0xd4, 0xbb, 0x0e, 0x60, | ||||||
| 	0x3a, 0x48, 0x5a, 0x5f, 0xa2, 0xf0, 0xa1, 0xd1, 0xaf, 0x7d, 0xc6, 0xea, 0xf3, 0xe4, 0x9c, 0xb5, | 	0x5b, 0xb0, 0x52, 0xea, 0xc8, 0x66, 0x5a, 0x59, 0x9b, 0x8b, 0x84, 0x7a, 0xd8, 0x0f, 0x47, 0xa5, | ||||||
| 	0x65, 0x96, 0x67, 0x6e, 0x3b, 0xb8, 0x06, 0xec, 0x28, 0x4e, 0x94, 0xdb, 0x04, 0xae, 0xa1, 0x81, | 	0x7e, 0xd1, 0x40, 0xec, 0x36, 0x5c, 0x38, 0xd5, 0xbb, 0xaf, 0x2c, 0x11, 0x69, 0xf5, 0x14, 0xc5, | ||||||
| 	0x13, 0xa5, 0x33, 0x95, 0xf8, 0xc6, 0xca, 0xe0, 0xc4, 0xed, 0x60, 0x40, 0xd8, 0x4b, 0x80, 0x40, | 	0x0f, 0x4d, 0x7e, 0x1d, 0x02, 0xb4, 0x97, 0x83, 0x31, 0xe8, 0xf3, 0x52, 0x95, 0x3e, 0x1d, 0x3a, | ||||||
| 	0x66, 0x12, 0x39, 0x73, 0xcd, 0xe3, 0x1a, 0xb1, 0x3c, 0x38, 0x71, 0xfd, 0xe2, 0x1a, 0x9c, 0x36, | 	0x23, 0xb6, 0x2f, 0x73, 0xe1, 0x93, 0xa0, 0x33, 0x06, 0x70, 0x20, 0x74, 0x29, 0xf2, 0xc8, 0x58, | ||||||
| 	0xf3, 0x54, 0xa5, 0xae, 0x3f, 0x0a, 0xc0, 0x21, 0xf8, 0x90, 0x9f, 0xca, 0xa2, 0x50, 0xa1, 0xe8, | 	0x1e, 0x1f, 0xf8, 0x0c, 0x46, 0x0e, 0x7b, 0x8e, 0x10, 0x9a, 0x99, 0x9c, 0xcf, 0x7d, 0xf0, 0x74, | ||||||
| 	0x92, 0x43, 0x00, 0xed, 0x23, 0x02, 0x0e, 0x21, 0x21, 0x8c, 0xb5, 0x5d, 0x88, 0x1e, 0x39, 0x04, | 	0x26, 0x4c, 0xc5, 0x07, 0x3e, 0x5e, 0x3a, 0x63, 0xa5, 0x4d, 0x56, 0x88, 0xc2, 0xc7, 0xe7, 0x04, | ||||||
| 	0xc8, 0xd7, 0x00, 0xc0, 0xf6, 0x31, 0x7d, 0xa6, 0x63, 0xab, 0x66, 0xd0, 0x62, 0x9f, 0xb6, 0x0f, | 	0xac, 0x10, 0x7e, 0x28, 0x2a, 0x78, 0x55, 0x89, 0x24, 0x18, 0xb8, 0x0a, 0x21, 0xf4, 0x94, 0x10, | ||||||
| 	0xe8, 0x0f, 0x15, 0xc8, 0xef, 0xb0, 0x1e, 0xec, 0xd1, 0xb7, 0xf3, 0x42, 0x30, 0xfa, 0x05, 0x40, | 	0xac, 0x10, 0x11, 0x12, 0xa9, 0xed, 0x31, 0x5d, 0x88, 0x7e, 0x38, 0x44, 0xe4, 0x1b, 0x04, 0x30, | ||||||
| 	0x7c, 0x38, 0x2f, 0xf8, 0x03, 0xb6, 0x1e, 0x67, 0x32, 0xb0, 0xf1, 0xa9, 0xf2, 0xd1, 0x93, 0x01, | 	0x7d, 0x52, 0x1f, 0x69, 0x69, 0xc5, 0x1c, 0x43, 0x1c, 0xba, 0xf4, 0x11, 0xfd, 0xa1, 0x01, 0xd9, | ||||||
| 	0xe6, 0xd7, 0x2a, 0x70, 0x0f, 0xbc, 0xd9, 0x62, 0x83, 0x26, 0x65, 0x8d, 0xda, 0x6c, 0x10, 0x9a, | 	0x35, 0x58, 0xc6, 0x1c, 0x23, 0x9b, 0x55, 0x01, 0xb8, 0x1b, 0x80, 0xf2, 0x8b, 0xac, 0x62, 0xb7, | ||||||
| 	0x55, 0xd0, 0xc5, 0xf5, 0x7f, 0x57, 0x79, 0x06, 0x6e, 0xd6, 0x55, 0x90, 0x72, 0xad, 0x59, 0x05, | 	0x60, 0x55, 0x96, 0x3c, 0xb6, 0xf2, 0x50, 0x44, 0x54, 0x93, 0x11, 0xe9, 0x57, 0x1a, 0x70, 0x07, | ||||||
| 	0x09, 0xdb, 0x6c, 0x50, 0x66, 0xea, 0x34, 0x0e, 0xac, 0x9c, 0x25, 0x4a, 0x5c, 0x27, 0xb7, 0x1b, | 	0x6b, 0xb3, 0x01, 0xa3, 0x2e, 0x65, 0xc5, 0x85, 0xd9, 0x21, 0x74, 0xbd, 0x50, 0x15, 0x57, 0xff, | ||||||
| 	0x10, 0x7f, 0x8f, 0x6d, 0x80, 0xc3, 0xbe, 0x56, 0x41, 0x22, 0xe3, 0x14, 0x69, 0x1b, 0x48, 0xbb, | 	0xed, 0xe5, 0x11, 0x56, 0xb3, 0xf5, 0x42, 0x94, 0x0b, 0x5d, 0x2f, 0x44, 0xd8, 0x84, 0x51, 0x5d, | ||||||
| 	0x0e, 0xb8, 0x57, 0xc3, 0xfc, 0x03, 0xc6, 0x91, 0x5a, 0x66, 0x4d, 0xf2, 0x0d, 0x24, 0xdf, 0x80, | 	0x8a, 0x43, 0x19, 0x5b, 0x3e, 0xcf, 0x45, 0x70, 0xd1, 0x55, 0xbb, 0x03, 0xb1, 0xf7, 0x61, 0x0d, | ||||||
| 	0xcc, 0xb4, 0x99, 0x80, 0x3b, 0x52, 0x44, 0x47, 0xb2, 0x4c, 0xac, 0xe0, 0xe4, 0x90, 0x0b, 0xf9, | 	0x2b, 0x1c, 0x69, 0x11, 0xe7, 0x5c, 0x16, 0x44, 0x5b, 0x23, 0xda, 0x45, 0xc4, 0xc3, 0x16, 0x66, | ||||||
| 	0x90, 0xb1, 0x22, 0x4a, 0xe5, 0x31, 0x25, 0x6f, 0x52, 0xd7, 0x35, 0x02, 0x1f, 0x3a, 0xcb, 0xf5, | 	0x1f, 0x02, 0x23, 0x6a, 0x5d, 0x76, 0xc9, 0x97, 0x88, 0x7c, 0x09, 0x35, 0x7b, 0x5d, 0x05, 0xbe, | ||||||
| 	0x49, 0x9c, 0x45, 0x46, 0x59, 0x5f, 0x2b, 0xe2, 0x6d, 0xd2, 0x87, 0xea, 0x8c, 0x47, 0x09, 0x3e, | 	0x91, 0x2a, 0xdd, 0xe7, 0x75, 0x6e, 0x03, 0xe6, 0x2a, 0xe4, 0x45, 0x36, 0x06, 0xa8, 0xd2, 0x82, | ||||||
| 	0x61, 0x37, 0x1b, 0x74, 0xdc, 0xbd, 0xb4, 0x4a, 0xdc, 0x42, 0x7e, 0xa3, 0xd2, 0x9e, 0xcb, 0xf0, | 	0xbf, 0x74, 0xca, 0xcb, 0x2e, 0xea, 0x16, 0xc1, 0x0f, 0x1d, 0x29, 0x7d, 0x20, 0xcb, 0xd4, 0x08, | ||||||
| 	0x4f, 0xd8, 0xed, 0x86, 0x20, 0xcb, 0x43, 0xe5, 0xfa, 0x16, 0xb7, 0x51, 0x73, 0xab, 0xce, 0x7e, | 	0x1b, 0x69, 0xe1, 0x78, 0xeb, 0xee, 0x43, 0xad, 0x26, 0x74, 0x0a, 0x36, 0x83, 0xcb, 0x1d, 0x3a, | ||||||
| 	0x5f, 0x27, 0xf9, 0x5d, 0xd6, 0x2b, 0x22, 0xad, 0x8e, 0xe2, 0x24, 0x11, 0x6f, 0xd1, 0xc5, 0xac, | 	0x65, 0xcf, 0xad, 0x08, 0xae, 0x10, 0xbf, 0xe3, 0x69, 0xc7, 0x6b, 0xd8, 0xa7, 0x70, 0xb5, 0x63, | ||||||
| 	0x62, 0x7e, 0x9b, 0x75, 0x8a, 0xc8, 0x04, 0x32, 0x13, 0x02, 0x33, 0x2e, 0x22, 0x13, 0x8c, 0x55, | 	0x50, 0xaa, 0x44, 0xf8, 0xb8, 0x83, 0xab, 0x64, 0x73, 0xa5, 0xd5, 0x7e, 0xdf, 0x2a, 0xd9, 0x75, | ||||||
| 	0x32, 0x11, 0x77, 0x2a, 0x13, 0x30, 0x24, 0x13, 0x96, 0xcd, 0xde, 0xad, 0x4c, 0xa8, 0x10, 0x3e, | 	0x58, 0xae, 0x52, 0x2d, 0xf6, 0x65, 0x9e, 0x07, 0xff, 0x73, 0x0f, 0xb3, 0x91, 0xd9, 0x55, 0x58, | ||||||
| 	0x62, 0x6b, 0x45, 0x14, 0xaa, 0x25, 0xe3, 0x1e, 0x9d, 0x7f, 0x13, 0xa3, 0x1a, 0x89, 0x7c, 0xb5, | 	0xaa, 0x52, 0x13, 0xf3, 0x32, 0x08, 0x48, 0xe3, 0x25, 0x57, 0x04, 0x63, 0x05, 0xcf, 0x83, 0x6b, | ||||||
| 	0x38, 0xd2, 0x4a, 0x89, 0xb7, 0xab, 0x1a, 0x15, 0x02, 0xc7, 0x5f, 0x47, 0xa1, 0xb8, 0x4f, 0xc7, | 	0x4d, 0x11, 0x48, 0x74, 0x45, 0x38, 0x0d, 0xf6, 0x7a, 0x53, 0x84, 0x06, 0x61, 0x13, 0x58, 0xa9, | ||||||
| 	0xdf, 0x80, 0xf8, 0xbb, 0xec, 0xba, 0x9d, 0x17, 0x3e, 0x1a, 0xe9, 0xcb, 0x24, 0xc9, 0x03, 0x31, | 	0xd2, 0x44, 0x9c, 0x32, 0x6e, 0xb8, 0xfe, 0x77, 0x31, 0xe7, 0x23, 0xe7, 0xaf, 0x8e, 0xf7, 0xb5, | ||||||
| 	0xac, 0xae, 0x7b, 0xf1, 0x0c, 0xd0, 0x3d, 0x00, 0xf9, 0xfb, 0x8c, 0x03, 0x2f, 0xc8, 0x93, 0x44, | 	0x10, 0xc1, 0xff, 0x1b, 0x1f, 0x0d, 0x82, 0xed, 0x6f, 0xa5, 0x24, 0xb8, 0xe9, 0xda, 0xdf, 0x81, | ||||||
| 	0x16, 0x46, 0x39, 0xea, 0x16, 0x52, 0x37, 0xec, 0xbc, 0x78, 0xea, 0x12, 0xc4, 0xde, 0x64, 0x57, | 	0xd8, 0x1d, 0xb8, 0x68, 0xb3, 0x2a, 0xa2, 0x42, 0x46, 0x3c, 0xcf, 0x55, 0x1c, 0x8c, 0x9b, 0xe7, | ||||||
| 	0x71, 0xa0, 0x89, 0x6d, 0xba, 0x9a, 0x18, 0xc0, 0xaf, 0x95, 0x06, 0x1f, 0x0d, 0xc8, 0x1d, 0x6a, | 	0x5e, 0x3d, 0x42, 0x74, 0x07, 0x41, 0xf6, 0x01, 0x30, 0xe4, 0xc5, 0x2a, 0xcf, 0x79, 0x65, 0x84, | ||||||
| 	0x17, 0xa1, 0xef, 0x00, 0x81, 0xab, 0x69, 0xce, 0x64, 0xe1, 0x93, 0x76, 0x44, 0x57, 0x13, 0x90, | 	0xa7, 0x6e, 0x10, 0x75, 0xcd, 0x66, 0xd5, 0x03, 0xaf, 0x70, 0xec, 0x75, 0x38, 0x4f, 0x03, 0x2d, | ||||||
| 	0x29, 0xea, 0xab, 0x34, 0xc9, 0x1f, 0xd4, 0x69, 0x54, 0x8f, 0x7e, 0x6e, 0xb1, 0x5e, 0xf5, 0x4a, | 	0xd8, 0x74, 0x4f, 0x93, 0x04, 0xbc, 0xad, 0x6e, 0xf0, 0xb9, 0x01, 0xb9, 0xe5, 0xc2, 0x25, 0xe8, | ||||||
| 	0xf0, 0x2f, 0x9b, 0x03, 0x1a, 0xa6, 0xfd, 0x83, 0xcb, 0x9f, 0x96, 0x6f, 0x32, 0xab, 0x17, 0xf5, | 	0x3b, 0x44, 0xf0, 0x69, 0x9a, 0x23, 0x5e, 0x45, 0xce, 0x76, 0xe2, 0x9e, 0x26, 0x22, 0x7b, 0x64, | ||||||
| 	0x14, 0xff, 0xbc, 0x9e, 0xe2, 0xff, 0x5b, 0xec, 0x46, 0xbd, 0x62, 0xfd, 0x25, 0x06, 0x3f, 0x8b, | 	0xdf, 0xa8, 0x9d, 0xf9, 0xad, 0x56, 0x4d, 0xd6, 0x13, 0x03, 0x2b, 0xdd, 0xe9, 0xcd, 0xd6, 0x60, | ||||||
| 	0x10, 0xee, 0x9a, 0xc2, 0xc1, 0xd8, 0xf7, 0x5c, 0x04, 0x56, 0xcc, 0x03, 0xe9, 0xcf, 0x65, 0x16, | 	0x31, 0x57, 0x47, 0x7e, 0x22, 0xe1, 0x11, 0xa7, 0x48, 0x26, 0xd3, 0xac, 0x19, 0x48, 0x78, 0x46, | ||||||
| 	0x26, 0xca, 0xe0, 0x84, 0x5c, 0xf7, 0xd8, 0x3c, 0x90, 0xcf, 0x09, 0xa9, 0x08, 0xf9, 0xec, 0x58, | 	0x56, 0xc1, 0x7f, 0xf4, 0x73, 0x08, 0x8f, 0x88, 0x28, 0x55, 0xf8, 0xf1, 0x83, 0x47, 0x7c, 0xec, | ||||||
| 	0x05, 0xd6, 0xe0, 0x98, 0x24, 0xc2, 0x0b, 0x42, 0x46, 0x7b, 0xac, 0x43, 0x8f, 0x1b, 0xff, 0xac, | 	0x4a, 0x15, 0xd1, 0x01, 0x36, 0xde, 0x4d, 0xa0, 0x81, 0x52, 0xc5, 0x13, 0x99, 0xe7, 0x93, 0x9f, | ||||||
| 	0x32, 0x9b, 0x36, 0xba, 0x73, 0xd9, 0x6b, 0xe8, 0x3a, 0x45, 0xfe, 0xe8, 0x97, 0x16, 0xeb, 0x3a, | 	0x7b, 0xb0, 0xdc, 0xec, 0x39, 0xf6, 0x55, 0x77, 0x2b, 0xe0, 0xbe, 0xba, 0xf5, 0xf6, 0xe5, 0xf8, | ||||||
| 	0x08, 0x4e, 0x2c, 0x95, 0xc7, 0xb9, 0x76, 0x03, 0x9c, 0x02, 0x44, 0xe3, 0x2c, 0xd7, 0xd5, 0x63, | 	0xb0, 0xb4, 0xfa, 0xb8, 0x5d, 0x1d, 0x5f, 0xb4, 0xab, 0xe3, 0x3f, 0x1b, 0xfb, 0xfd, 0x22, 0x60, | ||||||
| 	0x86, 0x01, 0x6c, 0x4a, 0xcf, 0x16, 0x56, 0x19, 0x37, 0xbd, 0x5d, 0x04, 0xf8, 0x19, 0xe1, 0x34, | 	0x78, 0x8a, 0xe1, 0x5d, 0x4c, 0xf0, 0x81, 0x0b, 0xca, 0x7d, 0x18, 0x7a, 0x09, 0xeb, 0x9f, 0xc5, | ||||||
| 	0xba, 0x5d, 0x04, 0xc3, 0x5b, 0xc7, 0xb9, 0xa9, 0x86, 0x37, 0xac, 0x01, 0x3b, 0x03, 0x8c, 0x66, | 	0x3c, 0xca, 0x78, 0x99, 0xe4, 0xc2, 0x50, 0x15, 0x56, 0x43, 0xc8, 0x62, 0xfe, 0xd8, 0x21, 0x0d, | ||||||
| 	0x37, 0xae, 0x47, 0x53, 0x36, 0x68, 0x3c, 0xbc, 0x97, 0xbc, 0xb1, 0x1b, 0x6c, 0x35, 0x95, 0x3f, | 	0x41, 0xcd, 0x5f, 0x8a, 0xd8, 0x1a, 0xaa, 0x89, 0x23, 0x3c, 0x73, 0xc8, 0x64, 0x07, 0x96, 0xdc, | ||||||
| 	0xba, 0xa6, 0x60, 0x89, 0x57, 0x53, 0x46, 0xca, 0xc4, 0xaf, 0x14, 0x36, 0xd5, 0xf7, 0x96, 0xf1, | 	0x7a, 0x66, 0x9f, 0x37, 0x1d, 0x76, 0x89, 0x6e, 0xbd, 0x6d, 0x9f, 0xfb, 0x48, 0x89, 0x3f, 0xf9, | ||||||
| 	0x13, 0xf1, 0xfa, 0xcd, 0xf0, 0xca, 0x1f, 0x6f, 0x86, 0x57, 0x7e, 0x3a, 0x1f, 0xb6, 0x5e, 0x9f, | 	0xa5, 0x07, 0x03, 0x0f, 0xe1, 0x35, 0x29, 0xf8, 0x4b, 0xa5, 0x7d, 0x8f, 0x9c, 0x40, 0xa8, 0x2c, | ||||||
| 	0x0f, 0x5b, 0xbf, 0x9f, 0x0f, 0x5b, 0x7f, 0x9d, 0x0f, 0x5b, 0xb3, 0x0e, 0xfe, 0x87, 0xfa, 0xe8, | 	0x95, 0x6e, 0x36, 0x28, 0x09, 0x98, 0x94, 0x9e, 0x1f, 0x5b, 0x61, 0x7c, 0xab, 0xbc, 0x84, 0xf8, | ||||||
| 	0x9f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x79, 0xd2, 0xcd, 0xe2, 0xab, 0x09, 0x00, 0x00, | 	0x91, 0xc3, 0x5d, 0xc3, 0xbc, 0x84, 0xbd, 0xd6, 0x52, 0x99, 0x66, 0x63, 0xe0, 0x19, 0xb1, 0x23, | ||||||
|  | 	0xc4, 0xdc, 0xc2, 0xa0, 0xf3, 0x64, 0x0f, 0x46, 0x9d, 0x5f, 0x87, 0xb7, 0x2c, 0x76, 0x7f, 0x51, | ||||||
|  | 	0x16, 0xda, 0x8b, 0x82, 0xf3, 0x80, 0xa7, 0xc2, 0xc8, 0x57, 0x82, 0x82, 0x1a, 0x86, 0xa7, 0xf2, | ||||||
|  | 	0xfd, 0xe0, 0xf5, 0x9b, 0xf1, 0xb9, 0x3f, 0xdf, 0x8c, 0xcf, 0xfd, 0x74, 0x32, 0xee, 0xbd, 0x3e, | ||||||
|  | 	0x19, 0xf7, 0xfe, 0x38, 0x19, 0xf7, 0xfe, 0x3e, 0x19, 0xf7, 0xe6, 0x4b, 0xf4, 0x17, 0xf8, 0xf1, | ||||||
|  | 	0x3f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x4f, 0x2b, 0x30, 0xd6, 0x6d, 0x0a, 0x00, 0x00, | ||||||
| } | } | ||||||
|  |  | ||||||
| func (m *Metrics) Marshal() (dAtA []byte, err error) { | func (m *Metrics) Marshal() (dAtA []byte, err error) { | ||||||
| @@ -597,6 +647,16 @@ func (m *Metrics) MarshalTo(dAtA []byte) (int, error) { | |||||||
| 			i += n | 			i += n | ||||||
| 		} | 		} | ||||||
| 	} | 	} | ||||||
|  | 	if m.MemoryEvents != nil { | ||||||
|  | 		dAtA[i] = 0x42 | ||||||
|  | 		i++ | ||||||
|  | 		i = encodeVarintMetrics(dAtA, i, uint64(m.MemoryEvents.Size())) | ||||||
|  | 		n6, err := m.MemoryEvents.MarshalTo(dAtA[i:]) | ||||||
|  | 		if err != nil { | ||||||
|  | 			return 0, err | ||||||
|  | 		} | ||||||
|  | 		i += n6 | ||||||
|  | 	} | ||||||
| 	if m.XXX_unrecognized != nil { | 	if m.XXX_unrecognized != nil { | ||||||
| 		i += copy(dAtA[i:], m.XXX_unrecognized) | 		i += copy(dAtA[i:], m.XXX_unrecognized) | ||||||
| 	} | 	} | ||||||
| @@ -921,6 +981,52 @@ func (m *MemoryStat) MarshalTo(dAtA []byte) (int, error) { | |||||||
| 	return i, nil | 	return i, nil | ||||||
| } | } | ||||||
|  |  | ||||||
|  | func (m *MemoryEvents) Marshal() (dAtA []byte, err error) { | ||||||
|  | 	size := m.Size() | ||||||
|  | 	dAtA = make([]byte, size) | ||||||
|  | 	n, err := m.MarshalTo(dAtA) | ||||||
|  | 	if err != nil { | ||||||
|  | 		return nil, err | ||||||
|  | 	} | ||||||
|  | 	return dAtA[:n], nil | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (m *MemoryEvents) MarshalTo(dAtA []byte) (int, error) { | ||||||
|  | 	var i int | ||||||
|  | 	_ = i | ||||||
|  | 	var l int | ||||||
|  | 	_ = l | ||||||
|  | 	if m.Low != 0 { | ||||||
|  | 		dAtA[i] = 0x8 | ||||||
|  | 		i++ | ||||||
|  | 		i = encodeVarintMetrics(dAtA, i, uint64(m.Low)) | ||||||
|  | 	} | ||||||
|  | 	if m.High != 0 { | ||||||
|  | 		dAtA[i] = 0x10 | ||||||
|  | 		i++ | ||||||
|  | 		i = encodeVarintMetrics(dAtA, i, uint64(m.High)) | ||||||
|  | 	} | ||||||
|  | 	if m.Max != 0 { | ||||||
|  | 		dAtA[i] = 0x18 | ||||||
|  | 		i++ | ||||||
|  | 		i = encodeVarintMetrics(dAtA, i, uint64(m.Max)) | ||||||
|  | 	} | ||||||
|  | 	if m.Oom != 0 { | ||||||
|  | 		dAtA[i] = 0x20 | ||||||
|  | 		i++ | ||||||
|  | 		i = encodeVarintMetrics(dAtA, i, uint64(m.Oom)) | ||||||
|  | 	} | ||||||
|  | 	if m.OomKill != 0 { | ||||||
|  | 		dAtA[i] = 0x28 | ||||||
|  | 		i++ | ||||||
|  | 		i = encodeVarintMetrics(dAtA, i, uint64(m.OomKill)) | ||||||
|  | 	} | ||||||
|  | 	if m.XXX_unrecognized != nil { | ||||||
|  | 		i += copy(dAtA[i:], m.XXX_unrecognized) | ||||||
|  | 	} | ||||||
|  | 	return i, nil | ||||||
|  | } | ||||||
|  |  | ||||||
| func (m *RdmaStat) Marshal() (dAtA []byte, err error) { | func (m *RdmaStat) Marshal() (dAtA []byte, err error) { | ||||||
| 	size := m.Size() | 	size := m.Size() | ||||||
| 	dAtA = make([]byte, size) | 	dAtA = make([]byte, size) | ||||||
| @@ -1165,6 +1271,10 @@ func (m *Metrics) Size() (n int) { | |||||||
| 			n += 1 + l + sovMetrics(uint64(l)) | 			n += 1 + l + sovMetrics(uint64(l)) | ||||||
| 		} | 		} | ||||||
| 	} | 	} | ||||||
|  | 	if m.MemoryEvents != nil { | ||||||
|  | 		l = m.MemoryEvents.Size() | ||||||
|  | 		n += 1 + l + sovMetrics(uint64(l)) | ||||||
|  | 	} | ||||||
| 	if m.XXX_unrecognized != nil { | 	if m.XXX_unrecognized != nil { | ||||||
| 		n += len(m.XXX_unrecognized) | 		n += len(m.XXX_unrecognized) | ||||||
| 	} | 	} | ||||||
| @@ -1336,6 +1446,33 @@ func (m *MemoryStat) Size() (n int) { | |||||||
| 	return n | 	return n | ||||||
| } | } | ||||||
|  |  | ||||||
|  | func (m *MemoryEvents) Size() (n int) { | ||||||
|  | 	if m == nil { | ||||||
|  | 		return 0 | ||||||
|  | 	} | ||||||
|  | 	var l int | ||||||
|  | 	_ = l | ||||||
|  | 	if m.Low != 0 { | ||||||
|  | 		n += 1 + sovMetrics(uint64(m.Low)) | ||||||
|  | 	} | ||||||
|  | 	if m.High != 0 { | ||||||
|  | 		n += 1 + sovMetrics(uint64(m.High)) | ||||||
|  | 	} | ||||||
|  | 	if m.Max != 0 { | ||||||
|  | 		n += 1 + sovMetrics(uint64(m.Max)) | ||||||
|  | 	} | ||||||
|  | 	if m.Oom != 0 { | ||||||
|  | 		n += 1 + sovMetrics(uint64(m.Oom)) | ||||||
|  | 	} | ||||||
|  | 	if m.OomKill != 0 { | ||||||
|  | 		n += 1 + sovMetrics(uint64(m.OomKill)) | ||||||
|  | 	} | ||||||
|  | 	if m.XXX_unrecognized != nil { | ||||||
|  | 		n += len(m.XXX_unrecognized) | ||||||
|  | 	} | ||||||
|  | 	return n | ||||||
|  | } | ||||||
|  |  | ||||||
| func (m *RdmaStat) Size() (n int) { | func (m *RdmaStat) Size() (n int) { | ||||||
| 	if m == nil { | 	if m == nil { | ||||||
| 		return 0 | 		return 0 | ||||||
| @@ -1476,6 +1613,7 @@ func (this *Metrics) String() string { | |||||||
| 		`Rdma:` + strings.Replace(fmt.Sprintf("%v", this.Rdma), "RdmaStat", "RdmaStat", 1) + `,`, | 		`Rdma:` + strings.Replace(fmt.Sprintf("%v", this.Rdma), "RdmaStat", "RdmaStat", 1) + `,`, | ||||||
| 		`Io:` + strings.Replace(fmt.Sprintf("%v", this.Io), "IOStat", "IOStat", 1) + `,`, | 		`Io:` + strings.Replace(fmt.Sprintf("%v", this.Io), "IOStat", "IOStat", 1) + `,`, | ||||||
| 		`Hugetlb:` + strings.Replace(fmt.Sprintf("%v", this.Hugetlb), "HugeTlbStat", "HugeTlbStat", 1) + `,`, | 		`Hugetlb:` + strings.Replace(fmt.Sprintf("%v", this.Hugetlb), "HugeTlbStat", "HugeTlbStat", 1) + `,`, | ||||||
|  | 		`MemoryEvents:` + strings.Replace(fmt.Sprintf("%v", this.MemoryEvents), "MemoryEvents", "MemoryEvents", 1) + `,`, | ||||||
| 		`XXX_unrecognized:` + fmt.Sprintf("%v", this.XXX_unrecognized) + `,`, | 		`XXX_unrecognized:` + fmt.Sprintf("%v", this.XXX_unrecognized) + `,`, | ||||||
| 		`}`, | 		`}`, | ||||||
| 	}, "") | 	}, "") | ||||||
| @@ -1554,6 +1692,21 @@ func (this *MemoryStat) String() string { | |||||||
| 	}, "") | 	}, "") | ||||||
| 	return s | 	return s | ||||||
| } | } | ||||||
|  | func (this *MemoryEvents) String() string { | ||||||
|  | 	if this == nil { | ||||||
|  | 		return "nil" | ||||||
|  | 	} | ||||||
|  | 	s := strings.Join([]string{`&MemoryEvents{`, | ||||||
|  | 		`Low:` + fmt.Sprintf("%v", this.Low) + `,`, | ||||||
|  | 		`High:` + fmt.Sprintf("%v", this.High) + `,`, | ||||||
|  | 		`Max:` + fmt.Sprintf("%v", this.Max) + `,`, | ||||||
|  | 		`Oom:` + fmt.Sprintf("%v", this.Oom) + `,`, | ||||||
|  | 		`OomKill:` + fmt.Sprintf("%v", this.OomKill) + `,`, | ||||||
|  | 		`XXX_unrecognized:` + fmt.Sprintf("%v", this.XXX_unrecognized) + `,`, | ||||||
|  | 		`}`, | ||||||
|  | 	}, "") | ||||||
|  | 	return s | ||||||
|  | } | ||||||
| func (this *RdmaStat) String() string { | func (this *RdmaStat) String() string { | ||||||
| 	if this == nil { | 	if this == nil { | ||||||
| 		return "nil" | 		return "nil" | ||||||
| @@ -1870,6 +2023,42 @@ func (m *Metrics) Unmarshal(dAtA []byte) error { | |||||||
| 				return err | 				return err | ||||||
| 			} | 			} | ||||||
| 			iNdEx = postIndex | 			iNdEx = postIndex | ||||||
|  | 		case 8: | ||||||
|  | 			if wireType != 2 { | ||||||
|  | 				return fmt.Errorf("proto: wrong wireType = %d for field MemoryEvents", wireType) | ||||||
|  | 			} | ||||||
|  | 			var msglen int | ||||||
|  | 			for shift := uint(0); ; shift += 7 { | ||||||
|  | 				if shift >= 64 { | ||||||
|  | 					return ErrIntOverflowMetrics | ||||||
|  | 				} | ||||||
|  | 				if iNdEx >= l { | ||||||
|  | 					return io.ErrUnexpectedEOF | ||||||
|  | 				} | ||||||
|  | 				b := dAtA[iNdEx] | ||||||
|  | 				iNdEx++ | ||||||
|  | 				msglen |= int(b&0x7F) << shift | ||||||
|  | 				if b < 0x80 { | ||||||
|  | 					break | ||||||
|  | 				} | ||||||
|  | 			} | ||||||
|  | 			if msglen < 0 { | ||||||
|  | 				return ErrInvalidLengthMetrics | ||||||
|  | 			} | ||||||
|  | 			postIndex := iNdEx + msglen | ||||||
|  | 			if postIndex < 0 { | ||||||
|  | 				return ErrInvalidLengthMetrics | ||||||
|  | 			} | ||||||
|  | 			if postIndex > l { | ||||||
|  | 				return io.ErrUnexpectedEOF | ||||||
|  | 			} | ||||||
|  | 			if m.MemoryEvents == nil { | ||||||
|  | 				m.MemoryEvents = &MemoryEvents{} | ||||||
|  | 			} | ||||||
|  | 			if err := m.MemoryEvents.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { | ||||||
|  | 				return err | ||||||
|  | 			} | ||||||
|  | 			iNdEx = postIndex | ||||||
| 		default: | 		default: | ||||||
| 			iNdEx = preIndex | 			iNdEx = preIndex | ||||||
| 			skippy, err := skipMetrics(dAtA[iNdEx:]) | 			skippy, err := skipMetrics(dAtA[iNdEx:]) | ||||||
| @@ -2874,6 +3063,155 @@ func (m *MemoryStat) Unmarshal(dAtA []byte) error { | |||||||
| 	} | 	} | ||||||
| 	return nil | 	return nil | ||||||
| } | } | ||||||
|  | func (m *MemoryEvents) Unmarshal(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 ErrIntOverflowMetrics | ||||||
|  | 			} | ||||||
|  | 			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: MemoryEvents: wiretype end group for non-group") | ||||||
|  | 		} | ||||||
|  | 		if fieldNum <= 0 { | ||||||
|  | 			return fmt.Errorf("proto: MemoryEvents: illegal tag %d (wire type %d)", fieldNum, wire) | ||||||
|  | 		} | ||||||
|  | 		switch fieldNum { | ||||||
|  | 		case 1: | ||||||
|  | 			if wireType != 0 { | ||||||
|  | 				return fmt.Errorf("proto: wrong wireType = %d for field Low", wireType) | ||||||
|  | 			} | ||||||
|  | 			m.Low = 0 | ||||||
|  | 			for shift := uint(0); ; shift += 7 { | ||||||
|  | 				if shift >= 64 { | ||||||
|  | 					return ErrIntOverflowMetrics | ||||||
|  | 				} | ||||||
|  | 				if iNdEx >= l { | ||||||
|  | 					return io.ErrUnexpectedEOF | ||||||
|  | 				} | ||||||
|  | 				b := dAtA[iNdEx] | ||||||
|  | 				iNdEx++ | ||||||
|  | 				m.Low |= uint64(b&0x7F) << shift | ||||||
|  | 				if b < 0x80 { | ||||||
|  | 					break | ||||||
|  | 				} | ||||||
|  | 			} | ||||||
|  | 		case 2: | ||||||
|  | 			if wireType != 0 { | ||||||
|  | 				return fmt.Errorf("proto: wrong wireType = %d for field High", wireType) | ||||||
|  | 			} | ||||||
|  | 			m.High = 0 | ||||||
|  | 			for shift := uint(0); ; shift += 7 { | ||||||
|  | 				if shift >= 64 { | ||||||
|  | 					return ErrIntOverflowMetrics | ||||||
|  | 				} | ||||||
|  | 				if iNdEx >= l { | ||||||
|  | 					return io.ErrUnexpectedEOF | ||||||
|  | 				} | ||||||
|  | 				b := dAtA[iNdEx] | ||||||
|  | 				iNdEx++ | ||||||
|  | 				m.High |= uint64(b&0x7F) << shift | ||||||
|  | 				if b < 0x80 { | ||||||
|  | 					break | ||||||
|  | 				} | ||||||
|  | 			} | ||||||
|  | 		case 3: | ||||||
|  | 			if wireType != 0 { | ||||||
|  | 				return fmt.Errorf("proto: wrong wireType = %d for field Max", wireType) | ||||||
|  | 			} | ||||||
|  | 			m.Max = 0 | ||||||
|  | 			for shift := uint(0); ; shift += 7 { | ||||||
|  | 				if shift >= 64 { | ||||||
|  | 					return ErrIntOverflowMetrics | ||||||
|  | 				} | ||||||
|  | 				if iNdEx >= l { | ||||||
|  | 					return io.ErrUnexpectedEOF | ||||||
|  | 				} | ||||||
|  | 				b := dAtA[iNdEx] | ||||||
|  | 				iNdEx++ | ||||||
|  | 				m.Max |= uint64(b&0x7F) << shift | ||||||
|  | 				if b < 0x80 { | ||||||
|  | 					break | ||||||
|  | 				} | ||||||
|  | 			} | ||||||
|  | 		case 4: | ||||||
|  | 			if wireType != 0 { | ||||||
|  | 				return fmt.Errorf("proto: wrong wireType = %d for field Oom", wireType) | ||||||
|  | 			} | ||||||
|  | 			m.Oom = 0 | ||||||
|  | 			for shift := uint(0); ; shift += 7 { | ||||||
|  | 				if shift >= 64 { | ||||||
|  | 					return ErrIntOverflowMetrics | ||||||
|  | 				} | ||||||
|  | 				if iNdEx >= l { | ||||||
|  | 					return io.ErrUnexpectedEOF | ||||||
|  | 				} | ||||||
|  | 				b := dAtA[iNdEx] | ||||||
|  | 				iNdEx++ | ||||||
|  | 				m.Oom |= uint64(b&0x7F) << shift | ||||||
|  | 				if b < 0x80 { | ||||||
|  | 					break | ||||||
|  | 				} | ||||||
|  | 			} | ||||||
|  | 		case 5: | ||||||
|  | 			if wireType != 0 { | ||||||
|  | 				return fmt.Errorf("proto: wrong wireType = %d for field OomKill", wireType) | ||||||
|  | 			} | ||||||
|  | 			m.OomKill = 0 | ||||||
|  | 			for shift := uint(0); ; shift += 7 { | ||||||
|  | 				if shift >= 64 { | ||||||
|  | 					return ErrIntOverflowMetrics | ||||||
|  | 				} | ||||||
|  | 				if iNdEx >= l { | ||||||
|  | 					return io.ErrUnexpectedEOF | ||||||
|  | 				} | ||||||
|  | 				b := dAtA[iNdEx] | ||||||
|  | 				iNdEx++ | ||||||
|  | 				m.OomKill |= uint64(b&0x7F) << shift | ||||||
|  | 				if b < 0x80 { | ||||||
|  | 					break | ||||||
|  | 				} | ||||||
|  | 			} | ||||||
|  | 		default: | ||||||
|  | 			iNdEx = preIndex | ||||||
|  | 			skippy, err := skipMetrics(dAtA[iNdEx:]) | ||||||
|  | 			if err != nil { | ||||||
|  | 				return err | ||||||
|  | 			} | ||||||
|  | 			if skippy < 0 { | ||||||
|  | 				return ErrInvalidLengthMetrics | ||||||
|  | 			} | ||||||
|  | 			if (iNdEx + skippy) < 0 { | ||||||
|  | 				return ErrInvalidLengthMetrics | ||||||
|  | 			} | ||||||
|  | 			if (iNdEx + skippy) > l { | ||||||
|  | 				return io.ErrUnexpectedEOF | ||||||
|  | 			} | ||||||
|  | 			m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) | ||||||
|  | 			iNdEx += skippy | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	if iNdEx > l { | ||||||
|  | 		return io.ErrUnexpectedEOF | ||||||
|  | 	} | ||||||
|  | 	return nil | ||||||
|  | } | ||||||
| func (m *RdmaStat) Unmarshal(dAtA []byte) error { | func (m *RdmaStat) Unmarshal(dAtA []byte) error { | ||||||
| 	l := len(dAtA) | 	l := len(dAtA) | ||||||
| 	iNdEx := 0 | 	iNdEx := 0 | ||||||
|   | |||||||
							
								
								
									
										9
									
								
								vendor/github.com/containerd/cgroups/v2/stats/metrics.proto
									
									
									
										generated
									
									
										vendored
									
									
								
							
							
						
						
									
										9
									
								
								vendor/github.com/containerd/cgroups/v2/stats/metrics.proto
									
									
									
										generated
									
									
										vendored
									
									
								
							| @@ -11,6 +11,7 @@ message Metrics { | |||||||
| 	RdmaStat rdma = 5; | 	RdmaStat rdma = 5; | ||||||
| 	IOStat io = 6; | 	IOStat io = 6; | ||||||
| 	repeated HugeTlbStat hugetlb = 7; | 	repeated HugeTlbStat hugetlb = 7; | ||||||
|  | 	MemoryEvents memory_events = 8; | ||||||
| } | } | ||||||
|  |  | ||||||
| message PidsStat { | message PidsStat { | ||||||
| @@ -65,6 +66,14 @@ message MemoryStat { | |||||||
| 	uint64 swap_limit = 35; | 	uint64 swap_limit = 35; | ||||||
| } | } | ||||||
|  |  | ||||||
|  | message MemoryEvents { | ||||||
|  | 	uint64 low = 1; | ||||||
|  | 	uint64 high = 2; | ||||||
|  | 	uint64 max = 3; | ||||||
|  | 	uint64 oom = 4; | ||||||
|  | 	uint64 oom_kill = 5; | ||||||
|  | } | ||||||
|  |  | ||||||
| message RdmaStat { | message RdmaStat { | ||||||
| 	repeated RdmaEntry current = 1; | 	repeated RdmaEntry current = 1; | ||||||
| 	repeated RdmaEntry limit = 2; | 	repeated RdmaEntry limit = 2; | ||||||
|   | |||||||
		Reference in New Issue
	
	Block a user
	 Akihiro Suda
					Akihiro Suda