pull out common parts of docker and raw container handlers for use by rkt handler

This commit is contained in:
Shaya Potter 2016-02-25 16:32:11 -08:00
parent 4a8ffcfc7e
commit 5e04a224ae
9 changed files with 303 additions and 224 deletions

View File

@ -16,7 +16,7 @@
// an array of ContainerHint structs, each with a container's id and networkInterface // an array of ContainerHint structs, each with a container's id and networkInterface
// This allows collecting stats about network interfaces configured outside docker // This allows collecting stats about network interfaces configured outside docker
// and lxc // and lxc
package raw package common
import ( import (
"encoding/json" "encoding/json"
@ -25,7 +25,7 @@ import (
"os" "os"
) )
var argContainerHints = flag.String("container_hints", "/etc/cadvisor/container_hints.json", "location of the container hints file") var ArgContainerHints = flag.String("container_hints", "/etc/cadvisor/container_hints.json", "location of the container hints file")
type containerHints struct { type containerHints struct {
AllHosts []containerHint `json:"all_hosts,omitempty"` AllHosts []containerHint `json:"all_hosts,omitempty"`
@ -34,10 +34,10 @@ type containerHints struct {
type containerHint struct { type containerHint struct {
FullName string `json:"full_path,omitempty"` FullName string `json:"full_path,omitempty"`
NetworkInterface *networkInterface `json:"network_interface,omitempty"` NetworkInterface *networkInterface `json:"network_interface,omitempty"`
Mounts []mount `json:"mounts,omitempty"` Mounts []Mount `json:"mounts,omitempty"`
} }
type mount struct { type Mount struct {
HostDir string `json:"host_dir,omitempty"` HostDir string `json:"host_dir,omitempty"`
ContainerDir string `json:"container_dir,omitempty"` ContainerDir string `json:"container_dir,omitempty"`
} }
@ -47,7 +47,7 @@ type networkInterface struct {
VethChild string `json:"veth_child,omitempty"` VethChild string `json:"veth_child,omitempty"`
} }
func getContainerHintsFromFile(containerHintsFile string) (containerHints, error) { func GetContainerHintsFromFile(containerHintsFile string) (containerHints, error) {
dat, err := ioutil.ReadFile(containerHintsFile) dat, err := ioutil.ReadFile(containerHintsFile)
if os.IsNotExist(err) { if os.IsNotExist(err) {
return containerHints{}, nil return containerHints{}, nil

View File

@ -12,14 +12,14 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package raw package common
import ( import (
"testing" "testing"
) )
func TestGetContainerHintsFromFile(t *testing.T) { func TestGetContainerHintsFromFile(t *testing.T) {
cHints, err := getContainerHintsFromFile("test_resources/container_hints.json") cHints, err := GetContainerHintsFromFile("test_resources/container_hints.json")
if err != nil { if err != nil {
t.Fatalf("Error in unmarshalling: %s", err) t.Fatalf("Error in unmarshalling: %s", err)
@ -50,8 +50,8 @@ func TestGetContainerHintsFromFile(t *testing.T) {
} }
func TestFileNotExist(t *testing.T) { func TestFileNotExist(t *testing.T) {
_, err := getContainerHintsFromFile("/file_does_not_exist.json") _, err := GetContainerHintsFromFile("/file_does_not_exist.json")
if err != nil { if err != nil {
t.Fatalf("getContainerHintsFromFile must not error for blank file: %s", err) t.Fatalf("GetContainerHintsFromFile must not error for blank file: %s", err)
} }
} }

View File

@ -13,7 +13,7 @@
// limitations under the License. // limitations under the License.
// Handler for Docker containers. // Handler for Docker containers.
package docker package common
import ( import (
"sync" "sync"
@ -24,10 +24,10 @@ import (
"github.com/golang/glog" "github.com/golang/glog"
) )
type fsHandler interface { type FsHandler interface {
start() Start()
usage() (uint64, uint64) Usage() (baseUsageBytes uint64, totalUsageBytes uint64)
stop() Stop()
} }
type realFsHandler struct { type realFsHandler struct {
@ -50,9 +50,9 @@ const (
maxDuBackoffFactor = 20 maxDuBackoffFactor = 20
) )
var _ fsHandler = &realFsHandler{} var _ FsHandler = &realFsHandler{}
func newFsHandler(period time.Duration, rootfs, extraDir string, fsInfo fs.FsInfo) fsHandler { func NewFsHandler(period time.Duration, rootfs, extraDir string, fsInfo fs.FsInfo) FsHandler {
return &realFsHandler{ return &realFsHandler{
lastUpdate: time.Time{}, lastUpdate: time.Time{},
usageBytes: 0, usageBytes: 0,
@ -119,15 +119,15 @@ func (fh *realFsHandler) trackUsage() {
} }
} }
func (fh *realFsHandler) start() { func (fh *realFsHandler) Start() {
go fh.trackUsage() go fh.trackUsage()
} }
func (fh *realFsHandler) stop() { func (fh *realFsHandler) Stop() {
close(fh.stopChan) close(fh.stopChan)
} }
func (fh *realFsHandler) usage() (baseUsageBytes, totalUsageBytes uint64) { func (fh *realFsHandler) Usage() (baseUsageBytes, totalUsageBytes uint64) {
fh.RLock() fh.RLock()
defer fh.RUnlock() defer fh.RUnlock()
return fh.baseUsageBytes, fh.usageBytes return fh.baseUsageBytes, fh.usageBytes

239
container/common/helpers.go Normal file
View File

@ -0,0 +1,239 @@
// Copyright 2016 Google Inc. All Rights Reserved.
//
// 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 common
import (
"fmt"
"io/ioutil"
"os"
"path"
"strconv"
"strings"
"time"
info "github.com/google/cadvisor/info/v1"
"github.com/google/cadvisor/utils"
"github.com/google/cadvisor/utils/machine"
"github.com/golang/glog"
)
func DebugInfo(watches map[string][]string) map[string][]string {
out := make(map[string][]string)
lines := make([]string, 0, len(watches))
for containerName, cgroupWatches := range watches {
lines = append(lines, fmt.Sprintf("%s:", containerName))
for _, cg := range cgroupWatches {
lines = append(lines, fmt.Sprintf("\t%s", cg))
}
}
out["Inotify watches"] = lines
return out
}
type AbstractContainerHandler interface {
GetCgroupPaths() map[string]string
GetMachineInfoFactory() info.MachineInfoFactory
GetName() string
GetRootNetworkDevices() ([]info.NetInfo, error)
GetExternalMounts() []Mount
HasNetwork() bool
HasFilesystem() bool
}
func GetSpec(handler AbstractContainerHandler) (info.ContainerSpec, error) {
cgroupPaths := handler.GetCgroupPaths()
machineInfoFactory := handler.GetMachineInfoFactory()
name := handler.GetName()
externalMounts := handler.GetExternalMounts()
var spec info.ContainerSpec
// The raw driver assumes unified hierarchy containers.
// Get the lowest creation time from all hierarchies as the container creation time.
now := time.Now()
lowestTime := now
for _, cgroupPath := range cgroupPaths {
// The modified time of the cgroup directory changes whenever a subcontainer is created.
// eg. /docker will have creation time matching the creation of latest docker container.
// Use clone_children as a workaround as it isn't usually modified. It is only likely changed
// immediately after creating a container.
cgroupPath = path.Join(cgroupPath, "cgroup.clone_children")
fi, err := os.Stat(cgroupPath)
if err == nil && fi.ModTime().Before(lowestTime) {
lowestTime = fi.ModTime()
}
}
if lowestTime != now {
spec.CreationTime = lowestTime
}
// Get machine info.
mi, err := machineInfoFactory.GetMachineInfo()
if err != nil {
return spec, err
}
// CPU.
cpuRoot, ok := cgroupPaths["cpu"]
if ok {
if utils.FileExists(cpuRoot) {
spec.HasCpu = true
spec.Cpu.Limit = readUInt64(cpuRoot, "cpu.shares")
spec.Cpu.Period = readUInt64(cpuRoot, "cpu.cfs_period_us")
quota := readString(cpuRoot, "cpu.cfs_quota_us")
if quota != "" && quota != "-1" {
val, err := strconv.ParseUint(quota, 10, 64)
if err != nil {
glog.Errorf("GetSpec: Failed to parse CPUQuota from %q: %s", path.Join(cpuRoot, "cpu.cfs_quota_us"), err)
}
spec.Cpu.Quota = val
}
}
}
// Cpu Mask.
// This will fail for non-unified hierarchies. We'll return the whole machine mask in that case.
cpusetRoot, ok := cgroupPaths["cpuset"]
if ok {
if utils.FileExists(cpusetRoot) {
spec.HasCpu = true
mask := readString(cpusetRoot, "cpuset.cpus")
spec.Cpu.Mask = utils.FixCpuMask(mask, mi.NumCores)
}
}
// Memory
if name == "/" {
// Get memory and swap limits of the running machine
memLimit, err := machine.GetMachineMemoryCapacity()
if err != nil {
glog.Warningf("failed to obtain memory limit for machine container")
spec.HasMemory = false
} else {
spec.Memory.Limit = uint64(memLimit)
// Spec is marked to have memory only if the memory limit is set
spec.HasMemory = true
}
swapLimit, err := machine.GetMachineSwapCapacity()
if err != nil {
glog.Warningf("failed to obtain swap limit for machine container")
} else {
spec.Memory.SwapLimit = uint64(swapLimit)
}
} else {
memoryRoot, ok := cgroupPaths["memory"]
if ok {
if utils.FileExists(memoryRoot) {
spec.HasMemory = true
spec.Memory.Limit = readUInt64(memoryRoot, "memory.limit_in_bytes")
spec.Memory.SwapLimit = readUInt64(memoryRoot, "memory.memsw.limit_in_bytes")
}
}
}
spec.HasFilesystem = name == "/" || externalMounts != nil || handler.HasFilesystem()
spec.HasNetwork = handler.HasNetwork()
if blkioRoot, ok := cgroupPaths["blkio"]; ok && utils.FileExists(blkioRoot) {
spec.HasDiskIo = true
}
// Check physical network devices for root container.
nd, err := handler.GetRootNetworkDevices()
if err != nil {
return spec, err
}
spec.HasNetwork = spec.HasNetwork || len(nd) != 0
return spec, nil
}
func readString(dirpath string, file string) string {
cgroupFile := path.Join(dirpath, file)
// Ignore non-existent files
if !utils.FileExists(cgroupFile) {
return ""
}
// Read
out, err := ioutil.ReadFile(cgroupFile)
if err != nil {
glog.Errorf("readString: Failed to read %q: %s", cgroupFile, err)
return ""
}
return strings.TrimSpace(string(out))
}
func readUInt64(dirpath string, file string) uint64 {
out := readString(dirpath, file)
if out == "" {
return 0
}
val, err := strconv.ParseUint(out, 10, 64)
if err != nil {
glog.Errorf("readUInt64: Failed to parse int %q from file %q: %s", out, path.Join(dirpath, file), err)
return 0
}
return val
}
// Lists all directories under "path" and outputs the results as children of "parent".
func ListDirectories(dirpath string, parent string, recursive bool, output map[string]struct{}) error {
// Ignore if this hierarchy does not exist.
if !utils.FileExists(dirpath) {
return nil
}
entries, err := ioutil.ReadDir(dirpath)
if err != nil {
return err
}
for _, entry := range entries {
// We only grab directories.
if entry.IsDir() {
name := path.Join(parent, entry.Name())
output[name] = struct{}{}
// List subcontainers if asked to.
if recursive {
err := ListDirectories(path.Join(dirpath, entry.Name()), name, true, output)
if err != nil {
return err
}
}
}
}
return nil
}
func MakeCgroupPaths(mountPoints map[string]string, name string) map[string]string {
cgroupPaths := make(map[string]string, len(mountPoints))
for key, val := range mountPoints {
cgroupPaths[key] = path.Join(val, name)
}
return cgroupPaths
}

View File

@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package raw package common
import ( import (
"sync" "sync"

View File

@ -24,6 +24,7 @@ import (
"time" "time"
"github.com/google/cadvisor/container" "github.com/google/cadvisor/container"
"github.com/google/cadvisor/container/common"
containerlibcontainer "github.com/google/cadvisor/container/libcontainer" containerlibcontainer "github.com/google/cadvisor/container/libcontainer"
"github.com/google/cadvisor/fs" "github.com/google/cadvisor/fs"
info "github.com/google/cadvisor/info/v1" info "github.com/google/cadvisor/info/v1"
@ -80,7 +81,7 @@ type dockerContainerHandler struct {
networkMode string networkMode string
// Filesystem handler. // Filesystem handler.
fsHandler fsHandler fsHandler common.FsHandler
ignoreMetrics container.MetricSet ignoreMetrics container.MetricSet
} }
@ -169,7 +170,7 @@ func newDockerContainerHandler(
} }
if !ignoreMetrics.Has(container.DiskUsageMetrics) { if !ignoreMetrics.Has(container.DiskUsageMetrics) {
handler.fsHandler = newFsHandler(time.Minute, rootfsStorageDir, otherStorageDir, fsInfo) handler.fsHandler = common.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.
@ -200,15 +201,14 @@ func newDockerContainerHandler(
} }
func (self *dockerContainerHandler) Start() { func (self *dockerContainerHandler) Start() {
// Start the filesystem handler.
if self.fsHandler != nil { if self.fsHandler != nil {
self.fsHandler.start() self.fsHandler.Start()
} }
} }
func (self *dockerContainerHandler) Cleanup() { func (self *dockerContainerHandler) Cleanup() {
if self.fsHandler != nil { if self.fsHandler != nil {
self.fsHandler.stop() self.fsHandler.Stop()
} }
} }
@ -317,6 +317,7 @@ func (self *dockerContainerHandler) getFsStats(stats *info.ContainerStats) error
if err != nil { if err != nil {
return err return err
} }
var ( var (
limit uint64 limit uint64
fsType string fsType string
@ -333,7 +334,7 @@ func (self *dockerContainerHandler) getFsStats(stats *info.ContainerStats) error
fsStat := info.FsStats{Device: deviceInfo.Device, Type: fsType, Limit: limit} fsStat := info.FsStats{Device: deviceInfo.Device, Type: fsType, Limit: limit}
fsStat.BaseUsage, fsStat.Usage = self.fsHandler.usage() fsStat.BaseUsage, fsStat.Usage = self.fsHandler.Usage()
stats.Filesystem = append(stats.Filesystem, fsStat) stats.Filesystem = append(stats.Filesystem, fsStat)
return nil return nil

View File

@ -19,6 +19,7 @@ import (
"fmt" "fmt"
"github.com/google/cadvisor/container" "github.com/google/cadvisor/container"
"github.com/google/cadvisor/container/common"
"github.com/google/cadvisor/container/libcontainer" "github.com/google/cadvisor/container/libcontainer"
"github.com/google/cadvisor/fs" "github.com/google/cadvisor/fs"
info "github.com/google/cadvisor/info/v1" info "github.com/google/cadvisor/info/v1"
@ -39,7 +40,7 @@ type rawFactory struct {
fsInfo fs.FsInfo fsInfo fs.FsInfo
// Watcher for inotify events. // Watcher for inotify events.
watcher *InotifyWatcher watcher *common.InotifyWatcher
// List of metrics to be ignored. // List of metrics to be ignored.
ignoreMetrics map[container.MetricKind]struct{} ignoreMetrics map[container.MetricKind]struct{}
@ -64,20 +65,7 @@ func (self *rawFactory) CanHandleAndAccept(name string) (bool, bool, error) {
} }
func (self *rawFactory) DebugInfo() map[string][]string { func (self *rawFactory) DebugInfo() map[string][]string {
out := make(map[string][]string) return common.DebugInfo(self.watcher.GetWatches())
// Get information about inotify watches.
watches := self.watcher.GetWatches()
lines := make([]string, 0, len(watches))
for containerName, cgroupWatches := range watches {
lines = append(lines, fmt.Sprintf("%s:", containerName))
for _, cg := range cgroupWatches {
lines = append(lines, fmt.Sprintf("\t%s", cg))
}
}
out["Inotify watches"] = lines
return out
} }
func Register(machineInfoFactory info.MachineInfoFactory, fsInfo fs.FsInfo, ignoreMetrics map[container.MetricKind]struct{}) error { func Register(machineInfoFactory info.MachineInfoFactory, fsInfo fs.FsInfo, ignoreMetrics map[container.MetricKind]struct{}) error {
@ -89,7 +77,7 @@ func Register(machineInfoFactory info.MachineInfoFactory, fsInfo fs.FsInfo, igno
return fmt.Errorf("failed to find supported cgroup mounts for the raw factory") return fmt.Errorf("failed to find supported cgroup mounts for the raw factory")
} }
watcher, err := NewInotifyWatcher() watcher, err := common.NewInotifyWatcher()
if err != nil { if err != nil {
return err return err
} }

View File

@ -20,16 +20,14 @@ import (
"io/ioutil" "io/ioutil"
"os" "os"
"path" "path"
"strconv"
"strings" "strings"
"time"
"github.com/google/cadvisor/container" "github.com/google/cadvisor/container"
"github.com/google/cadvisor/container/common"
"github.com/google/cadvisor/container/libcontainer" "github.com/google/cadvisor/container/libcontainer"
"github.com/google/cadvisor/fs" "github.com/google/cadvisor/fs"
info "github.com/google/cadvisor/info/v1" info "github.com/google/cadvisor/info/v1"
"github.com/google/cadvisor/utils" "github.com/google/cadvisor/utils"
"github.com/google/cadvisor/utils/machine"
"github.com/golang/glog" "github.com/golang/glog"
"github.com/opencontainers/runc/libcontainer/cgroups" "github.com/opencontainers/runc/libcontainer/cgroups"
@ -45,7 +43,7 @@ type rawContainerHandler struct {
machineInfoFactory info.MachineInfoFactory machineInfoFactory info.MachineInfoFactory
// Inotify event watcher. // Inotify event watcher.
watcher *InotifyWatcher watcher *common.InotifyWatcher
// Signal for watcher thread to stop. // Signal for watcher thread to stop.
stopWatcher chan error stopWatcher chan error
@ -58,7 +56,7 @@ type rawContainerHandler struct {
cgroupManager cgroups.Manager cgroupManager cgroups.Manager
fsInfo fs.FsInfo fsInfo fs.FsInfo
externalMounts []mount externalMounts []common.Mount
rootFs string rootFs string
@ -66,14 +64,34 @@ type rawContainerHandler struct {
ignoreMetrics container.MetricSet ignoreMetrics container.MetricSet
} }
func newRawContainerHandler(name string, cgroupSubsystems *libcontainer.CgroupSubsystems, machineInfoFactory info.MachineInfoFactory, fsInfo fs.FsInfo, watcher *InotifyWatcher, rootFs string, ignoreMetrics container.MetricSet) (container.ContainerHandler, error) { func (self *rawContainerHandler) GetCgroupPaths() map[string]string {
// Create the cgroup paths. return self.cgroupPaths
cgroupPaths := make(map[string]string, len(cgroupSubsystems.MountPoints))
for key, val := range cgroupSubsystems.MountPoints {
cgroupPaths[key] = path.Join(val, name)
} }
cHints, err := getContainerHintsFromFile(*argContainerHints) func (self *rawContainerHandler) GetMachineInfoFactory() info.MachineInfoFactory {
return self.machineInfoFactory
}
func (self *rawContainerHandler) GetName() string {
return self.name
}
func (self *rawContainerHandler) GetExternalMounts() []common.Mount {
return self.externalMounts
}
func (self *rawContainerHandler) HasNetwork() bool {
return false
}
func (self *rawContainerHandler) HasFilesystem() bool {
return false
}
func newRawContainerHandler(name string, cgroupSubsystems *libcontainer.CgroupSubsystems, machineInfoFactory info.MachineInfoFactory, fsInfo fs.FsInfo, watcher *common.InotifyWatcher, rootFs string, ignoreMetrics container.MetricSet) (container.ContainerHandler, error) {
cgroupPaths := common.MakeCgroupPaths(cgroupSubsystems.MountPoints, name)
cHints, err := common.GetContainerHintsFromFile(*common.ArgContainerHints)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -86,7 +104,7 @@ func newRawContainerHandler(name string, cgroupSubsystems *libcontainer.CgroupSu
Paths: cgroupPaths, Paths: cgroupPaths,
} }
var externalMounts []mount var externalMounts []common.Mount
for _, container := range cHints.AllHosts { for _, container := range cHints.AllHosts {
if name == container.FullName { if name == container.FullName {
externalMounts = container.Mounts externalMounts = container.Mounts
@ -116,38 +134,6 @@ func (self *rawContainerHandler) ContainerReference() (info.ContainerReference,
}, nil }, nil
} }
func readString(dirpath string, file string) string {
cgroupFile := path.Join(dirpath, file)
// Ignore non-existent files
if !utils.FileExists(cgroupFile) {
return ""
}
// Read
out, err := ioutil.ReadFile(cgroupFile)
if err != nil {
glog.Errorf("raw driver: Failed to read %q: %s", cgroupFile, err)
return ""
}
return strings.TrimSpace(string(out))
}
func readUInt64(dirpath string, file string) uint64 {
out := readString(dirpath, file)
if out == "" {
return 0
}
val, err := strconv.ParseUint(out, 10, 64)
if err != nil {
glog.Errorf("raw driver: Failed to parse int %q from file %q: %s", out, path.Join(dirpath, file), err)
return 0
}
return val
}
func (self *rawContainerHandler) GetRootNetworkDevices() ([]info.NetInfo, error) { func (self *rawContainerHandler) GetRootNetworkDevices() ([]info.NetInfo, error) {
nd := []info.NetInfo{} nd := []info.NetInfo{}
if self.name == "/" { if self.name == "/" {
@ -167,113 +153,7 @@ func (self *rawContainerHandler) Start() {}
func (self *rawContainerHandler) Cleanup() {} func (self *rawContainerHandler) Cleanup() {}
func (self *rawContainerHandler) GetSpec() (info.ContainerSpec, error) { func (self *rawContainerHandler) GetSpec() (info.ContainerSpec, error) {
var spec info.ContainerSpec return common.GetSpec(self)
// The raw driver assumes unified hierarchy containers.
// Get the lowest creation time from all hierarchies as the container creation time.
now := time.Now()
lowestTime := now
for _, cgroupPath := range self.cgroupPaths {
// The modified time of the cgroup directory changes whenever a subcontainer is created.
// eg. /docker will have creation time matching the creation of latest docker container.
// Use clone_children as a workaround as it isn't usually modified. It is only likely changed
// immediately after creating a container.
cgroupPath = path.Join(cgroupPath, "cgroup.clone_children")
fi, err := os.Stat(cgroupPath)
if err == nil && fi.ModTime().Before(lowestTime) {
lowestTime = fi.ModTime()
}
}
if lowestTime != now {
spec.CreationTime = lowestTime
}
// Get machine info.
mi, err := self.machineInfoFactory.GetMachineInfo()
if err != nil {
return spec, err
}
// CPU.
cpuRoot, ok := self.cgroupPaths["cpu"]
if ok {
if utils.FileExists(cpuRoot) {
spec.HasCpu = true
spec.Cpu.Limit = readUInt64(cpuRoot, "cpu.shares")
spec.Cpu.Period = readUInt64(cpuRoot, "cpu.cfs_period_us")
quota := readString(cpuRoot, "cpu.cfs_quota_us")
if quota != "" && quota != "-1" {
val, err := strconv.ParseUint(quota, 10, 64)
if err != nil {
glog.Errorf("raw driver: Failed to parse CPUQuota from %q: %s", path.Join(cpuRoot, "cpu.cfs_quota_us"), err)
}
spec.Cpu.Quota = val
}
}
}
// Cpu Mask.
// This will fail for non-unified hierarchies. We'll return the whole machine mask in that case.
cpusetRoot, ok := self.cgroupPaths["cpuset"]
if ok {
if utils.FileExists(cpusetRoot) {
spec.HasCpu = true
mask := readString(cpusetRoot, "cpuset.cpus")
spec.Cpu.Mask = utils.FixCpuMask(mask, mi.NumCores)
}
}
// Memory
if self.name == "/" {
// Get memory and swap limits of the running machine
memLimit, err := machine.GetMachineMemoryCapacity()
if err != nil {
glog.Warningf("failed to obtain memory limit for machine container")
spec.HasMemory = false
} else {
spec.Memory.Limit = uint64(memLimit)
// Spec is marked to have memory only if the memory limit is set
spec.HasMemory = true
}
swapLimit, err := machine.GetMachineSwapCapacity()
if err != nil {
glog.Warningf("failed to obtain swap limit for machine container")
} else {
spec.Memory.SwapLimit = uint64(swapLimit)
}
} else {
memoryRoot, ok := self.cgroupPaths["memory"]
if ok {
if utils.FileExists(memoryRoot) {
spec.HasMemory = true
spec.Memory.Limit = readUInt64(memoryRoot, "memory.limit_in_bytes")
spec.Memory.SwapLimit = readUInt64(memoryRoot, "memory.memsw.limit_in_bytes")
}
}
}
// Fs.
if self.name == "/" || self.externalMounts != nil {
spec.HasFilesystem = true
}
// DiskIo.
if blkioRoot, ok := self.cgroupPaths["blkio"]; ok && utils.FileExists(blkioRoot) {
spec.HasDiskIo = true
}
// Check physical network devices for root container.
nd, err := self.GetRootNetworkDevices()
if err != nil {
return spec, err
}
if len(nd) != 0 {
spec.HasNetwork = true
}
return spec, nil
} }
func (self *rawContainerHandler) getFsStats(stats *info.ContainerStats) error { func (self *rawContainerHandler) getFsStats(stats *info.ContainerStats) error {
@ -367,39 +247,10 @@ func (self *rawContainerHandler) GetContainerLabels() map[string]string {
return map[string]string{} return map[string]string{}
} }
// Lists all directories under "path" and outputs the results as children of "parent".
func listDirectories(dirpath string, parent string, recursive bool, output map[string]struct{}) error {
// Ignore if this hierarchy does not exist.
if !utils.FileExists(dirpath) {
return nil
}
entries, err := ioutil.ReadDir(dirpath)
if err != nil {
return err
}
for _, entry := range entries {
// We only grab directories.
if entry.IsDir() {
name := path.Join(parent, entry.Name())
output[name] = struct{}{}
// List subcontainers if asked to.
if recursive {
err := listDirectories(path.Join(dirpath, entry.Name()), name, true, output)
if err != nil {
return err
}
}
}
}
return nil
}
func (self *rawContainerHandler) ListContainers(listType container.ListType) ([]info.ContainerReference, error) { func (self *rawContainerHandler) ListContainers(listType container.ListType) ([]info.ContainerReference, error) {
containers := make(map[string]struct{}) containers := make(map[string]struct{})
for _, cgroupPath := range self.cgroupPaths { for _, cgroupPath := range self.cgroupPaths {
err := listDirectories(cgroupPath, self.name, listType == container.ListRecursive, containers) err := common.ListDirectories(cgroupPath, self.name, listType == container.ListRecursive, containers)
if err != nil { if err != nil {
return nil, err return nil, err
} }