Support opt out for metrics.

Signed-off-by: Vishnu kannan <vishnuk@google.com>
This commit is contained in:
Vishnu kannan 2016-02-19 17:37:39 -08:00
parent 39fbd9fc6a
commit 36415f465a
8 changed files with 139 additions and 52 deletions

View File

@ -86,6 +86,8 @@ type dockerFactory struct {
fsInfo fs.FsInfo fsInfo fs.FsInfo
dockerVersion []int dockerVersion []int
ignoreMetrics container.MetricSet
} }
func (self *dockerFactory) String() string { func (self *dockerFactory) String() string {
@ -111,6 +113,7 @@ func (self *dockerFactory) NewContainerHandler(name string, inHostNamespace bool
inHostNamespace, inHostNamespace,
metadataEnvs, metadataEnvs,
self.dockerVersion, self.dockerVersion,
self.ignoreMetrics,
) )
return return
} }
@ -178,7 +181,7 @@ func parseDockerVersion(full_version_string string) ([]int, error) {
} }
// Register root container before running this function! // Register root container before running this function!
func Register(factory info.MachineInfoFactory, fsInfo fs.FsInfo) error { func Register(factory info.MachineInfoFactory, fsInfo fs.FsInfo, ignoreMetrics container.MetricSet) error {
client, err := Client() client, err := Client()
if err != nil { if err != nil {
return fmt.Errorf("unable to communicate with docker daemon: %v", err) return fmt.Errorf("unable to communicate with docker daemon: %v", err)
@ -236,6 +239,7 @@ func Register(factory info.MachineInfoFactory, fsInfo fs.FsInfo) error {
machineInfoFactory: factory, machineInfoFactory: factory,
storageDriver: storageDriver(sd), storageDriver: storageDriver(sd),
storageDir: storageDir, storageDir: storageDir,
ignoreMetrics: ignoreMetrics,
} }
container.RegisterContainerHandlerFactory(f) container.RegisterContainerHandlerFactory(f)

View File

@ -81,6 +81,8 @@ type dockerContainerHandler struct {
// Filesystem handler. // Filesystem handler.
fsHandler fsHandler fsHandler fsHandler
ignoreMetrics container.MetricSet
} }
func getRwLayerID(containerID, storageDir string, sd storageDriver, dockerVersion []int) (string, error) { func getRwLayerID(containerID, storageDir string, sd storageDriver, dockerVersion []int) (string, error) {
@ -111,6 +113,7 @@ func newDockerContainerHandler(
inHostNamespace bool, inHostNamespace bool,
metadataEnvs []string, metadataEnvs []string,
dockerVersion []int, dockerVersion []int,
ignoreMetrics container.MetricSet,
) (container.ContainerHandler, error) { ) (container.ContainerHandler, error) {
// Create the cgroup paths. // Create the cgroup paths.
cgroupPaths := make(map[string]string, len(cgroupSubsystems.MountPoints)) cgroupPaths := make(map[string]string, len(cgroupSubsystems.MountPoints))
@ -161,8 +164,12 @@ func newDockerContainerHandler(
fsInfo: fsInfo, fsInfo: fsInfo,
rootFs: rootFs, rootFs: rootFs,
rootfsStorageDir: rootfsStorageDir, rootfsStorageDir: rootfsStorageDir,
fsHandler: newFsHandler(time.Minute, rootfsStorageDir, otherStorageDir, fsInfo),
envs: make(map[string]string), envs: make(map[string]string),
ignoreMetrics: ignoreMetrics,
}
if !ignoreMetrics.Has(container.DiskUsageMetrics) {
handler.fsHandler = newFsHandler(time.Minute, rootfsStorageDir, otherStorageDir, fsInfo)
} }
// We assume that if Inspect fails then the container is not known to docker. // We assume that if Inspect fails then the container is not known to docker.
@ -255,8 +262,11 @@ func libcontainerConfigToContainerSpec(config *libcontainerconfigs.Config, mi *i
return spec return spec
} }
func hasNet(networkMode string) bool { func (self *dockerContainerHandler) needNet() bool {
return !strings.HasPrefix(networkMode, "container:") if !self.ignoreMetrics.Has(container.NetworkUsageMetrics) {
return !strings.HasPrefix(self.networkMode, "container:")
}
return false
} }
func (self *dockerContainerHandler) GetSpec() (info.ContainerSpec, error) { func (self *dockerContainerHandler) GetSpec() (info.ContainerSpec, error) {
@ -272,22 +282,25 @@ func (self *dockerContainerHandler) GetSpec() (info.ContainerSpec, error) {
spec := libcontainerConfigToContainerSpec(libcontainerConfig, mi) spec := libcontainerConfigToContainerSpec(libcontainerConfig, mi)
spec.CreationTime = self.creationTime spec.CreationTime = self.creationTime
switch self.storageDriver { if !self.ignoreMetrics.Has(container.DiskUsageMetrics) {
case aufsStorageDriver, overlayStorageDriver, zfsStorageDriver: switch self.storageDriver {
spec.HasFilesystem = true case aufsStorageDriver, overlayStorageDriver, zfsStorageDriver:
default: spec.HasFilesystem = true
spec.HasFilesystem = false }
} }
spec.Labels = self.labels spec.Labels = self.labels
spec.Envs = self.envs spec.Envs = self.envs
spec.Image = self.image spec.Image = self.image
spec.HasNetwork = hasNet(self.networkMode) spec.HasNetwork = self.needNet()
return spec, err return spec, err
} }
func (self *dockerContainerHandler) getFsStats(stats *info.ContainerStats) error { func (self *dockerContainerHandler) getFsStats(stats *info.ContainerStats) error {
if self.ignoreMetrics.Has(container.DiskUsageMetrics) {
return nil
}
switch self.storageDriver { switch self.storageDriver {
case aufsStorageDriver, overlayStorageDriver, zfsStorageDriver: case aufsStorageDriver, overlayStorageDriver, zfsStorageDriver:
default: default:
@ -322,7 +335,7 @@ func (self *dockerContainerHandler) getFsStats(stats *info.ContainerStats) error
// TODO(vmarmol): Get from libcontainer API instead of cgroup manager when we don't have to support older Dockers. // TODO(vmarmol): Get from libcontainer API instead of cgroup manager when we don't have to support older Dockers.
func (self *dockerContainerHandler) GetStats() (*info.ContainerStats, error) { func (self *dockerContainerHandler) GetStats() (*info.ContainerStats, error) {
stats, err := containerlibcontainer.GetStats(self.cgroupManager, self.rootFs, self.pid) stats, err := containerlibcontainer.GetStats(self.cgroupManager, self.rootFs, self.pid, self.ignoreMetrics)
if err != nil { if err != nil {
return stats, err return stats, err
} }
@ -330,7 +343,7 @@ func (self *dockerContainerHandler) GetStats() (*info.ContainerStats, error) {
// includes containers running in Kubernetes pods that use the network of the // includes containers running in Kubernetes pods that use the network of the
// infrastructure container. This stops metrics being reported multiple times // infrastructure container. This stops metrics being reported multiple times
// for each container in a pod. // for each container in a pod.
if !hasNet(self.networkMode) { if !self.needNet() {
stats.Network = info.NetworkStats{} stats.Network = info.NetworkStats{}
} }

View File

@ -35,6 +35,35 @@ type ContainerHandlerFactory interface {
DebugInfo() map[string][]string DebugInfo() map[string][]string
} }
// MetricKind represents the kind of metrics that cAdvisor exposes.
type MetricKind string
const (
CpuUsageMetrics MetricKind = "cpu"
MemoryUsageMetrics MetricKind = "memory"
CpuLoadMetrics MetricKind = "cpuLoad"
DiskIOMetrics MetricKind = "diskIO"
DiskUsageMetrics MetricKind = "disk"
NetworkUsageMetrics MetricKind = "network"
NetworkTcpUsageMetrics MetricKind = "tcp"
AppMetrics MetricKind = "app"
)
func (mk MetricKind) String() string {
return string(mk)
}
type MetricSet map[MetricKind]struct{}
func (ms MetricSet) Has(mk MetricKind) bool {
_, exists := ms[mk]
return exists
}
func (ms MetricSet) Add(mk MetricKind) {
ms[mk] = struct{}{}
}
// TODO(vmarmol): Consider not making this global. // TODO(vmarmol): Consider not making this global.
// Global list of factories. // Global list of factories.
var ( var (

View File

@ -17,12 +17,14 @@ package libcontainer
import ( import (
"bufio" "bufio"
"fmt" "fmt"
"io/ioutil"
"os" "os"
"path" "path"
"strconv" "strconv"
"strings" "strings"
"time" "time"
"github.com/google/cadvisor/container"
info "github.com/google/cadvisor/info/v1" info "github.com/google/cadvisor/info/v1"
"github.com/golang/glog" "github.com/golang/glog"
@ -79,7 +81,7 @@ var supportedSubsystems map[string]struct{} = map[string]struct{}{
} }
// Get cgroup and networking stats of the specified container // Get cgroup and networking stats of the specified container
func GetStats(cgroupManager cgroups.Manager, rootFs string, pid int) (*info.ContainerStats, error) { func GetStats(cgroupManager cgroups.Manager, rootFs string, pid int, ignoreMetrics container.MetricSet) (*info.ContainerStats, error) {
cgroupStats, err := cgroupManager.GetStats() cgroupStats, err := cgroupManager.GetStats()
if err != nil { if err != nil {
return nil, err return nil, err
@ -91,27 +93,29 @@ func GetStats(cgroupManager cgroups.Manager, rootFs string, pid int) (*info.Cont
// If we know the pid then get network stats from /proc/<pid>/net/dev // If we know the pid then get network stats from /proc/<pid>/net/dev
if pid > 0 { if pid > 0 {
netStats, err := networkStatsFromProc(rootFs, pid) if !ignoreMetrics.Has(container.NetworkUsageMetrics) {
if err != nil { netStats, err := networkStatsFromProc(rootFs, pid)
glog.V(2).Infof("Unable to get network stats from pid %d: %v", pid, err) if err != nil {
} else { glog.V(2).Infof("Unable to get network stats from pid %d: %v", pid, err)
stats.Network.Interfaces = append(stats.Network.Interfaces, netStats...) } else {
stats.Network.Interfaces = append(stats.Network.Interfaces, netStats...)
}
} }
if !ignoreMetrics.Has(container.NetworkTcpUsageMetrics) {
t, err := tcpStatsFromProc(rootFs, pid, "net/tcp")
if err != nil {
glog.V(2).Infof("Unable to get tcp stats from pid %d: %v", pid, err)
} else {
stats.Network.Tcp = t
}
// Commenting out to disable: too CPU intensive t6, err := tcpStatsFromProc(rootFs, pid, "net/tcp6")
/*t, err := tcpStatsFromProc(rootFs, pid, "net/tcp") if err != nil {
if err != nil { glog.V(2).Infof("Unable to get tcp6 stats from pid %d: %v", pid, err)
glog.V(2).Infof("Unable to get tcp stats from pid %d: %v", pid, err) } else {
} else { stats.Network.Tcp6 = t6
stats.Network.Tcp = t }
} }
t6, err := tcpStatsFromProc(rootFs, pid, "net/tcp6")
if err != nil {
glog.V(2).Infof("Unable to get tcp6 stats from pid %d: %v", pid, err)
} else {
stats.Network.Tcp6 = t6
}*/
} }
// For backwards compatibility. // For backwards compatibility.
@ -211,7 +215,6 @@ func setInterfaceStatValues(fields []string, pointers []*uint64) error {
return nil return nil
} }
/*
func tcpStatsFromProc(rootFs string, pid int, file string) (info.TcpStat, error) { func tcpStatsFromProc(rootFs string, pid int, file string) (info.TcpStat, error) {
tcpStatsFile := path.Join(rootFs, "proc", strconv.Itoa(pid), file) tcpStatsFile := path.Join(rootFs, "proc", strconv.Itoa(pid), file)
@ -286,7 +289,6 @@ func scanTcpStats(tcpStatsFile string) (info.TcpStat, error) {
return stats, nil return stats, nil
} }
*/
func GetProcesses(cgroupManager cgroups.Manager) ([]int, error) { func GetProcesses(cgroupManager cgroups.Manager) ([]int, error) {
pids, err := cgroupManager.GetPids() pids, err := cgroupManager.GetPids()

View File

@ -40,6 +40,9 @@ type rawFactory struct {
// Watcher for inotify events. // Watcher for inotify events.
watcher *InotifyWatcher watcher *InotifyWatcher
// List of metrics to be ignored.
ignoreMetrics map[container.MetricKind]struct{}
} }
func (self *rawFactory) String() string { func (self *rawFactory) String() string {
@ -51,7 +54,7 @@ func (self *rawFactory) NewContainerHandler(name string, inHostNamespace bool) (
if !inHostNamespace { if !inHostNamespace {
rootFs = "/rootfs" rootFs = "/rootfs"
} }
return newRawContainerHandler(name, self.cgroupSubsystems, self.machineInfoFactory, self.fsInfo, self.watcher, rootFs) return newRawContainerHandler(name, self.cgroupSubsystems, self.machineInfoFactory, self.fsInfo, self.watcher, rootFs, self.ignoreMetrics)
} }
// The raw factory can handle any container. If --docker_only is set to false, non-docker containers are ignored. // The raw factory can handle any container. If --docker_only is set to false, non-docker containers are ignored.
@ -77,7 +80,7 @@ func (self *rawFactory) DebugInfo() map[string][]string {
return out return out
} }
func Register(machineInfoFactory info.MachineInfoFactory, fsInfo fs.FsInfo) error { func Register(machineInfoFactory info.MachineInfoFactory, fsInfo fs.FsInfo, ignoreMetrics map[container.MetricKind]struct{}) error {
cgroupSubsystems, err := libcontainer.GetCgroupSubsystems() cgroupSubsystems, err := libcontainer.GetCgroupSubsystems()
if err != nil { if err != nil {
return fmt.Errorf("failed to get cgroup subsystems: %v", err) return fmt.Errorf("failed to get cgroup subsystems: %v", err)
@ -97,6 +100,7 @@ func Register(machineInfoFactory info.MachineInfoFactory, fsInfo fs.FsInfo) erro
fsInfo: fsInfo, fsInfo: fsInfo,
cgroupSubsystems: &cgroupSubsystems, cgroupSubsystems: &cgroupSubsystems,
watcher: watcher, watcher: watcher,
ignoreMetrics: ignoreMetrics,
} }
container.RegisterContainerHandlerFactory(factory) container.RegisterContainerHandlerFactory(factory)
return nil return nil

View File

@ -57,16 +57,16 @@ type rawContainerHandler struct {
// Manager of this container's cgroups. // Manager of this container's cgroups.
cgroupManager cgroups.Manager cgroupManager cgroups.Manager
// Whether this container has network isolation enabled.
hasNetwork bool
fsInfo fs.FsInfo fsInfo fs.FsInfo
externalMounts []mount externalMounts []mount
rootFs string rootFs string
// Metrics to be ignored.
ignoreMetrics container.MetricSet
} }
func newRawContainerHandler(name string, cgroupSubsystems *libcontainer.CgroupSubsystems, machineInfoFactory info.MachineInfoFactory, fsInfo fs.FsInfo, watcher *InotifyWatcher, rootFs string) (container.ContainerHandler, error) { func newRawContainerHandler(name string, cgroupSubsystems *libcontainer.CgroupSubsystems, machineInfoFactory info.MachineInfoFactory, fsInfo fs.FsInfo, watcher *InotifyWatcher, rootFs string, ignoreMetrics container.MetricSet) (container.ContainerHandler, error) {
// Create the cgroup paths. // Create the cgroup paths.
cgroupPaths := make(map[string]string, len(cgroupSubsystems.MountPoints)) cgroupPaths := make(map[string]string, len(cgroupSubsystems.MountPoints))
for key, val := range cgroupSubsystems.MountPoints { for key, val := range cgroupSubsystems.MountPoints {
@ -86,15 +86,9 @@ func newRawContainerHandler(name string, cgroupSubsystems *libcontainer.CgroupSu
Paths: cgroupPaths, Paths: cgroupPaths,
} }
hasNetwork := false
var externalMounts []mount var externalMounts []mount
for _, container := range cHints.AllHosts { for _, container := range cHints.AllHosts {
if name == container.FullName { if name == container.FullName {
/*libcontainerState.NetworkState = network.NetworkState{
VethHost: container.NetworkInterface.VethHost,
VethChild: container.NetworkInterface.VethChild,
}
hasNetwork = true*/
externalMounts = container.Mounts externalMounts = container.Mounts
break break
} }
@ -108,10 +102,10 @@ func newRawContainerHandler(name string, cgroupSubsystems *libcontainer.CgroupSu
cgroupPaths: cgroupPaths, cgroupPaths: cgroupPaths,
cgroupManager: cgroupManager, cgroupManager: cgroupManager,
fsInfo: fsInfo, fsInfo: fsInfo,
hasNetwork: hasNetwork,
externalMounts: externalMounts, externalMounts: externalMounts,
watcher: watcher, watcher: watcher,
rootFs: rootFs, rootFs: rootFs,
ignoreMetrics: ignoreMetrics,
}, nil }, nil
} }
@ -266,9 +260,6 @@ func (self *rawContainerHandler) GetSpec() (info.ContainerSpec, error) {
spec.HasFilesystem = true spec.HasFilesystem = true
} }
//Network
spec.HasNetwork = self.hasNetwork
// DiskIo. // DiskIo.
if blkioRoot, ok := self.cgroupPaths["blkio"]; ok && utils.FileExists(blkioRoot) { if blkioRoot, ok := self.cgroupPaths["blkio"]; ok && utils.FileExists(blkioRoot) {
spec.HasDiskIo = true spec.HasDiskIo = true
@ -346,7 +337,7 @@ func (self *rawContainerHandler) getFsStats(stats *info.ContainerStats) error {
} }
func (self *rawContainerHandler) GetStats() (*info.ContainerStats, error) { func (self *rawContainerHandler) GetStats() (*info.ContainerStats, error) {
stats, err := libcontainer.GetStats(self.cgroupManager, self.rootFs, os.Getpid()) stats, err := libcontainer.GetStats(self.cgroupManager, self.rootFs, os.Getpid(), self.ignoreMetrics)
if err != nil { if err != nil {
return stats, err return stats, err
} }

View File

@ -50,6 +50,42 @@ var eventStorageAgeLimit = flag.String("event_storage_age_limit", "default=24h",
var eventStorageEventLimit = flag.String("event_storage_event_limit", "default=100000", "Max number of events to store (per type). Value is a comma separated list of key values, where the keys are event types (e.g.: creation, oom) or \"default\" and the value is an integer. Default is applied to all non-specified event types") var eventStorageEventLimit = flag.String("event_storage_event_limit", "default=100000", "Max number of events to store (per type). Value is a comma separated list of key values, where the keys are event types (e.g.: creation, oom) or \"default\" and the value is an integer. Default is applied to all non-specified event types")
var applicationMetricsCountLimit = flag.Int("application_metrics_count_limit", 100, "Max number of application metrics to store (per container)") var applicationMetricsCountLimit = flag.Int("application_metrics_count_limit", 100, "Max number of application metrics to store (per container)")
var (
// Metrics to be ignored.
ignoreMetrics metricSetValue = metricSetValue{container.MetricSet{}}
// List of metrics that can be ignored.
ignoreWhitelist = map[string]struct{}{
container.DiskUsageMetrics.String(): {},
container.NetworkUsageMetrics.String(): {},
container.NetworkTcpUsageMetrics.String(): {},
}
)
func init() {
flag.Var(&ignoreMetrics, "disable_metrics", "comma-separated list of metrics to be disabled. Options are `disk`, `network`, `tcp`. Note: tcp is disabled by default due to high CPU usage.")
// Tcp metrics are ignored by default.
flag.Set("disable_metrics", "tcp")
}
type metricSetValue struct {
container.MetricSet
}
func (ml *metricSetValue) String() string {
return fmt.Sprint(*ml)
}
func (ml *metricSetValue) Set(value string) error {
for _, metric := range strings.Split(value, ",") {
if _, exists := ignoreWhitelist[metric]; exists {
(*ml).Add(container.MetricKind(metric))
} else {
return fmt.Errorf("unsupported metric %q specified in disable_metrics", metric)
}
}
return nil
}
// The Manager interface defines operations for starting a manager and getting // The Manager interface defines operations for starting a manager and getting
// container and machine information. // container and machine information.
type Manager interface { type Manager interface {
@ -147,6 +183,7 @@ func New(memoryCache *memory.InMemoryCache, sysfs sysfs.SysFs, maxHousekeepingIn
if _, err := os.Stat("/rootfs/proc"); os.IsNotExist(err) { if _, err := os.Stat("/rootfs/proc"); os.IsNotExist(err) {
inHostNamespace = true inHostNamespace = true
} }
newManager := &manager{ newManager := &manager{
containers: make(map[namespacedContainerName]*containerData), containers: make(map[namespacedContainerName]*containerData),
quitChannels: make([]chan error, 0, 2), quitChannels: make([]chan error, 0, 2),
@ -157,6 +194,7 @@ func New(memoryCache *memory.InMemoryCache, sysfs sysfs.SysFs, maxHousekeepingIn
startupTime: time.Now(), startupTime: time.Now(),
maxHousekeepingInterval: maxHousekeepingInterval, maxHousekeepingInterval: maxHousekeepingInterval,
allowDynamicHousekeeping: allowDynamicHousekeeping, allowDynamicHousekeeping: allowDynamicHousekeeping,
ignoreMetrics: ignoreMetrics.MetricSet,
} }
machineInfo, err := getMachineInfo(sysfs, fsInfo, inHostNamespace) machineInfo, err := getMachineInfo(sysfs, fsInfo, inHostNamespace)
@ -199,18 +237,19 @@ type manager struct {
startupTime time.Time startupTime time.Time
maxHousekeepingInterval time.Duration maxHousekeepingInterval time.Duration
allowDynamicHousekeeping bool allowDynamicHousekeeping bool
ignoreMetrics container.MetricSet
} }
// Start the container manager. // Start the container manager.
func (self *manager) Start() error { func (self *manager) Start() error {
// Register Docker container factory. // Register Docker container factory.
err := docker.Register(self, self.fsInfo) err := docker.Register(self, self.fsInfo, self.ignoreMetrics)
if err != nil { if err != nil {
glog.Errorf("Docker container factory registration failed: %v.", err) glog.Errorf("Docker container factory registration failed: %v.", err)
} }
// Register the raw driver. // Register the raw driver.
err = raw.Register(self, self.fsInfo) err = raw.Register(self, self.fsInfo, self.ignoreMetrics)
if err != nil { if err != nil {
glog.Errorf("Registration of the raw container factory failed: %v", err) glog.Errorf("Registration of the raw container factory failed: %v", err)
} }

View File

@ -29,6 +29,7 @@ import (
info "github.com/google/cadvisor/info/v1" info "github.com/google/cadvisor/info/v1"
itest "github.com/google/cadvisor/info/v1/test" itest "github.com/google/cadvisor/info/v1/test"
"github.com/google/cadvisor/utils/sysfs/fakesysfs" "github.com/google/cadvisor/utils/sysfs/fakesysfs"
"github.com/stretchr/testify/assert"
) )
// TODO(vmarmol): Refactor these tests. // TODO(vmarmol): Refactor these tests.
@ -210,3 +211,7 @@ func TestNewNilManager(t *testing.T) {
t.Fatalf("Expected nil manager to return error") t.Fatalf("Expected nil manager to return error")
} }
} }
func TestTcpMetricsAreDisabledByDefault(t *testing.T) {
assert.True(t, ignoreMetrics.Has(container.NetworkTcpUsageMetrics))
}