Collect container-level GPU metrics using NVML.

When cAdvisor starts up, it would read the `vendor` files in
`/sys/bus/pci/devices/*` to see if any NVIDIA devices (vendor ID: 0x10de) are
attached to the node. If no NVIDIA devices are found, this code path would
become dormant for the rest of cAdvisor lifetime. If NVIDIA devices are found,
we would start a goroutine that would check for the presence of NVML by trying
to dynamically load it at regular intervals. We need to do this regular
checking instead of doing it just once because it may happen that cAdvisor is
started before the NVIDIA drivers and NVML are installed.  Once the NVML
dynamic loading succeeds, we would use NVML’s query methods to find out how
many devices exist on the node and create a map from their minor numbers to
their handles and cache that map. The goroutine would exit at this point.

If we detected the presence of NVML in the previous step, whenever a new
container is detected by cAdvisor, cAdvisor would read the `devices.list` file
from the container's devices cgroup. The `devices.list` file lists the
major:minor number of all the devices that the container is allowed to access.
If we find any device with major number 195 (which is the major number assigned
to NVIDIA devices), we would cache the list of corresponding minor numbers for
that container.

During every housekeeping operation, in addition to collecting all the existing
metrics, we will use the cached NVIDIA device minor numbers and the map from
minor numbers to device handles to get metrics for GPU devices attached to the
container.
This commit is contained in:
Rohit Agarwal 2017-11-06 11:54:59 -08:00
parent 318f28bef6
commit 4a35130019
7 changed files with 491 additions and 2 deletions

239
accelerators/nvidia.go Normal file
View File

@ -0,0 +1,239 @@
// Copyright 2017 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 accelerators
import (
"bufio"
"fmt"
"io/ioutil"
"os"
"path/filepath"
"strconv"
"strings"
"time"
info "github.com/google/cadvisor/info/v1"
"github.com/golang/glog"
"github.com/mindprince/gonvml"
)
type NvidiaManager struct {
// true if the NVML library (libnvidia-ml.so.1) was loaded successfully
nvmlInitialized bool
// nvidiaDevices is a map from device minor number to a handle that can be used to get metrics about the device
nvidiaDevices map[int]gonvml.Device
}
var sysFsPCIDevicesPath = "/sys/bus/pci/devices/"
const nvidiaVendorId = "0x10de"
// Setup initializes NVML if nvidia devices are present on the node.
func (nm *NvidiaManager) Setup() {
if !detectDevices(nvidiaVendorId) {
glog.Info("No NVIDIA devices found.")
return
}
go func() {
glog.Info("Starting goroutine to initialize NVML")
nm.initializeNVML()
if nm.nvmlInitialized {
return
}
// TODO: use globalHousekeepingInterval
for range time.Tick(time.Minute) {
nm.initializeNVML()
if nm.nvmlInitialized {
return
}
}
}()
}
// detectDevices returns true if a device with given pci id is present on the node.
func detectDevices(vendorId string) bool {
devices, err := ioutil.ReadDir(sysFsPCIDevicesPath)
if err != nil {
glog.Warningf("error reading %q: %v", sysFsPCIDevicesPath, err)
return false
}
for _, device := range devices {
vendorPath := filepath.Join(sysFsPCIDevicesPath, device.Name(), "vendor")
content, err := ioutil.ReadFile(vendorPath)
if err != nil {
glog.Infof("Error while reading %q: %v", vendorPath, err)
continue
}
if strings.EqualFold(strings.TrimSpace(string(content)), vendorId) {
glog.Infof("Found device with vendorId %q", vendorId)
return true
}
}
return false
}
// initializeNVML initializes the NVML library and sets up the nvmlDevices map.
func (nm *NvidiaManager) initializeNVML() {
if err := gonvml.Initialize(); err != nil {
// This is under a logging level because otherwise we may cause
// log spam if the drivers/nvml is not installed on the system.
glog.V(3).Infof("Could not initialize NVML: %v", err)
return
}
nm.nvmlInitialized = true
numDevices, err := gonvml.DeviceCount()
if err != nil {
glog.Warningf("GPU metrics would not be available. Failed to get the number of nvidia devices: %v", err)
return
}
glog.Infof("NVML initialized. Number of nvidia devices: %v", numDevices)
nm.nvidiaDevices = make(map[int]gonvml.Device, numDevices)
for i := 0; i < int(numDevices); i++ {
device, err := gonvml.DeviceHandleByIndex(uint(i))
if err != nil {
glog.Warningf("Failed to get nvidia device handle %d: %v", i, err)
continue
}
minorNumber, err := device.MinorNumber()
if err != nil {
glog.Warningf("Failed to get nvidia device minor number: %v", err)
continue
}
nm.nvidiaDevices[int(minorNumber)] = device
}
}
// Destroy shuts down NVML.
func (nm *NvidiaManager) Destroy() {
if nm.nvmlInitialized {
gonvml.Shutdown()
}
}
// GetCollector returns a collector that can fetch nvidia gpu metrics for nvidia devices
// present in the devices.list file in the given devicesCgroupPath.
func (nm *NvidiaManager) GetCollector(devicesCgroupPath string) (AcceleratorCollector, error) {
nc := &NvidiaCollector{}
if !nm.nvmlInitialized || len(nm.nvidiaDevices) == 0 {
return nc, nil
}
nvidiaMinorNumbers, err := parseDevicesCgroup(devicesCgroupPath)
if err != nil {
return nc, err
}
for _, minor := range nvidiaMinorNumbers {
device, ok := nm.nvidiaDevices[minor]
if !ok {
return nc, fmt.Errorf("nvidia device minor number %d not found in cached devices", minor)
}
nc.Devices = append(nc.Devices, device)
}
return nc, nil
}
// parseDevicesCgroup parses the devices cgroup devices.list file for the container
// and returns a list of minor numbers corresponding to NVIDIA GPU devices that the
// container is allowed to access. In cases where the container has access to all
// devices or all NVIDIA devices but the devices are not enumerated separately in
// the devices.list file, we return an empty list.
// This is defined as a variable to help in testing.
var parseDevicesCgroup = func(devicesCgroupPath string) ([]int, error) {
// Always return a non-nil slice
nvidiaMinorNumbers := []int{}
devicesList := filepath.Join(devicesCgroupPath, "devices.list")
f, err := os.Open(devicesList)
if err != nil {
return nvidiaMinorNumbers, fmt.Errorf("error while opening devices cgroup file %q: %v", devicesList, err)
}
defer f.Close()
s := bufio.NewScanner(f)
// See https://www.kernel.org/doc/Documentation/cgroup-v1/devices.txt for the file format
for s.Scan() {
text := s.Text()
fields := strings.Fields(text)
if len(fields) != 3 {
return nvidiaMinorNumbers, fmt.Errorf("invalid devices cgroup entry %q: must contain three whitespace-separated fields", text)
}
// Split the second field to find out major:minor numbers
majorMinor := strings.Split(fields[1], ":")
if len(majorMinor) != 2 {
return nvidiaMinorNumbers, fmt.Errorf("invalid devices cgroup entry %q: second field should have one colon", text)
}
// NVIDIA graphics devices are character devices with major number 195.
// https://github.com/torvalds/linux/blob/v4.13/Documentation/admin-guide/devices.txt#L2583
if fields[0] == "c" && majorMinor[0] == "195" {
minorNumber, err := strconv.Atoi(majorMinor[1])
if err != nil {
return nvidiaMinorNumbers, fmt.Errorf("invalid devices cgroup entry %q: minor number is not integer", text)
}
// We don't want devices like nvidiactl (195:255) and nvidia-modeset (195:254)
if minorNumber < 128 {
nvidiaMinorNumbers = append(nvidiaMinorNumbers, minorNumber)
}
// We are ignoring the "195:*" case
// where the container has access to all NVIDIA devices on the machine.
}
// We are ignoring the "*:*" case
// where the container has access to all devices on the machine.
}
return nvidiaMinorNumbers, nil
}
type NvidiaCollector struct {
// Exposed for testing
Devices []gonvml.Device
}
// UpdateStats updates the stats for NVIDIA GPUs (if any) attached to the container.
func (nc *NvidiaCollector) UpdateStats(stats *info.ContainerStats) error {
for _, device := range nc.Devices {
model, err := device.Name()
if err != nil {
return fmt.Errorf("error while getting gpu name: %v", err)
}
uuid, err := device.UUID()
if err != nil {
return fmt.Errorf("error while getting gpu uuid: %v", err)
}
memoryTotal, memoryUsed, err := device.MemoryInfo()
if err != nil {
return fmt.Errorf("error while getting gpu memory info: %v", err)
}
//TODO: Use housekeepingInterval
utilizationGPU, err := device.AverageGPUUtilization(10 * time.Second)
if err != nil {
return fmt.Errorf("error while getting gpu utilization: %v", err)
}
stats.Accelerators = append(stats.Accelerators, info.AcceleratorStats{
Make: "nvidia",
Model: model,
ID: uuid,
MemoryTotal: memoryTotal,
MemoryUsed: memoryUsed,
DutyCycle: uint64(utilizationGPU),
})
}
return nil
}

168
accelerators/nvidia_test.go Normal file
View File

@ -0,0 +1,168 @@
// Copyright 2017 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 accelerators
import (
"io/ioutil"
"os"
"path/filepath"
"testing"
"github.com/mindprince/gonvml"
"github.com/stretchr/testify/assert"
)
func updateFile(t *testing.T, fn string, content []byte) {
if err := ioutil.WriteFile(fn, content, 0666); err != nil {
t.Fatalf("Error writing to temporary file for testing: %v", err)
}
}
func TestDetectDevices(t *testing.T) {
sysFsPCIDevicesPath = "/non-existent-path"
detected := detectDevices("0x10de")
assert.False(t, detected)
var err error
// Create temporary directory to represent sysfs pci devices path
if sysFsPCIDevicesPath, err = ioutil.TempDir("", "sys-bus-pci-devices"); err != nil {
t.Fatalf("Error creating temporary directory for testing: %v", err)
}
defer os.RemoveAll(sysFsPCIDevicesPath)
device0 := filepath.Join(sysFsPCIDevicesPath, "device0")
device1 := filepath.Join(sysFsPCIDevicesPath, "device1")
device2 := filepath.Join(sysFsPCIDevicesPath, "device2")
for _, device := range []string{device0, device1, device2} {
if err = os.Mkdir(device, 0777); err != nil {
t.Fatalf("Error creating temporary directory for testing: %v", err)
}
}
// device0 directory is present to make sure that
// we handle bad device directories case correctly.
// A valid vendor file but different than what's being detected.
updateFile(t, filepath.Join(device1, "vendor"), []byte("0x8086\n"))
detected = detectDevices("0x10de")
assert.False(t, detected)
// vendor file for device being detected
updateFile(t, filepath.Join(device2, "vendor"), []byte("0x10de\n"))
detected = detectDevices("0x10de")
assert.True(t, detected)
}
func TestGetCollector(t *testing.T) {
// Mock parseDevicesCgroup.
originalParser := parseDevicesCgroup
mockParser := func(_ string) ([]int, error) {
return []int{2, 3}, nil
}
parseDevicesCgroup = mockParser
defer func() {
parseDevicesCgroup = originalParser
}()
nm := &NvidiaManager{}
// When nvmlInitialized is false, empty collector should be returned.
ac, err := nm.GetCollector("does-not-matter")
assert.Nil(t, err)
assert.NotNil(t, ac)
nc, ok := ac.(*NvidiaCollector)
assert.True(t, ok)
assert.Equal(t, 0, len(nc.Devices))
// When nvidiaDevices is empty, empty collector should be returned.
nm.nvmlInitialized = true
ac, err = nm.GetCollector("does-not-matter")
assert.Nil(t, err)
assert.NotNil(t, ac)
nc, ok = ac.(*NvidiaCollector)
assert.True(t, ok)
assert.Equal(t, 0, len(nc.Devices))
// nvidiaDevices contains devices but they are different than what
// is returned by parseDevicesCgroup. We should get an error.
nm.nvidiaDevices = map[int]gonvml.Device{0: {}, 1: {}}
ac, err = nm.GetCollector("does-not-matter")
assert.NotNil(t, err)
assert.NotNil(t, ac)
nc, ok = ac.(*NvidiaCollector)
assert.True(t, ok)
assert.Equal(t, 0, len(nc.Devices))
// nvidiaDevices contains devices returned by parseDevicesCgroup.
// No error should be returned and collectors devices array should be
// correctly initialized.
nm.nvidiaDevices[2] = gonvml.Device{}
nm.nvidiaDevices[3] = gonvml.Device{}
ac, err = nm.GetCollector("does-not-matter")
assert.Nil(t, err)
assert.NotNil(t, ac)
nc, ok = ac.(*NvidiaCollector)
assert.True(t, ok)
assert.Equal(t, 2, len(nc.Devices))
}
func TestParseDevicesCgroup(t *testing.T) {
// Test case for empty devices cgroup path
nvidiaMinorNumbers, err := parseDevicesCgroup("")
assert.NotNil(t, err)
assert.Equal(t, []int{}, nvidiaMinorNumbers)
// Test case for non-existent devices cgroup
nvidiaMinorNumbers, err = parseDevicesCgroup("/non-existent-path")
assert.NotNil(t, err)
assert.Equal(t, []int{}, nvidiaMinorNumbers)
// Create temporary directory to represent devices cgroup.
tmpDir, err := ioutil.TempDir("", "devices-cgroup")
if err != nil {
t.Fatalf("Error creating temporary directory for testing: %v", err)
}
defer os.RemoveAll(tmpDir)
tmpfn := filepath.Join(tmpDir, "devices.list")
// Test case when devices.list file has more than three fields.
updateFile(t, tmpfn, []byte("c 1:2 rwm badformat\n"))
nvidiaMinorNumbers, err = parseDevicesCgroup(tmpDir)
assert.NotNil(t, err)
assert.Equal(t, []int{}, nvidiaMinorNumbers)
// Test case when devices.list file's second field is not major:minor.
updateFile(t, tmpfn, []byte("c badformat rwm\n"))
nvidiaMinorNumbers, err = parseDevicesCgroup(tmpDir)
assert.NotNil(t, err)
assert.Equal(t, []int{}, nvidiaMinorNumbers)
// Test case with nvidia devices present
updateFile(t, tmpfn, []byte("c 195:0 rwm\nc 195:255 rwm\nc 195:1 rwm"))
nvidiaMinorNumbers, err = parseDevicesCgroup(tmpDir)
assert.Nil(t, err)
assert.Equal(t, []int{0, 1}, nvidiaMinorNumbers) // Note that 255 is not supposed to be returned.
// Test case with a common devices.list file
updateFile(t, tmpfn, []byte("a *:* rwm\n"))
nvidiaMinorNumbers, err = parseDevicesCgroup(tmpDir)
assert.Nil(t, err)
assert.Equal(t, []int{}, nvidiaMinorNumbers)
// Test case for empty devices.list file
updateFile(t, tmpfn, []byte(""))
nvidiaMinorNumbers, err = parseDevicesCgroup(tmpDir)
assert.Nil(t, err)
assert.Equal(t, []int{}, nvidiaMinorNumbers)
}

32
accelerators/types.go Normal file
View File

@ -0,0 +1,32 @@
// Copyright 2017 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 accelerators
import info "github.com/google/cadvisor/info/v1"
// This is supposed to store global state about an accelerator metrics collector.
// cadvisor manager will call Setup() when it starts and Destroy() when it stops.
// For each container detected by the cadvisor manager, it will call
// GetCollector() with the devices cgroup path for that container.
// GetCollector() is supposed to return an object that can update
// accelerator stats for that container.
type AcceleratorManager interface {
Setup()
Destroy()
GetCollector(deviceCgroup string) (AcceleratorCollector, error)
}
type AcceleratorCollector interface {
UpdateStats(*info.ContainerStats) error
}

View File

@ -84,6 +84,7 @@ var supportedSubsystems map[string]struct{} = map[string]struct{}{
"memory": {},
"cpuset": {},
"blkio": {},
"devices": {},
}
// Get cgroup and networking stats of the specified container

View File

@ -29,6 +29,7 @@ import (
"sync"
"time"
"github.com/google/cadvisor/accelerators"
"github.com/google/cadvisor/cache/memory"
"github.com/google/cadvisor/collector"
"github.com/google/cadvisor/container"
@ -78,6 +79,9 @@ type containerData struct {
// Runs custom metric collectors.
collectorManager collector.CollectorManager
// nvidiaCollector updates stats for Nvidia GPUs attached to the container.
nvidiaCollector accelerators.AcceleratorCollector
}
// jitter returns a time.Duration between duration and duration + maxFactor * duration,
@ -557,6 +561,12 @@ func (c *containerData) updateStats() error {
}
}
var nvidiaStatsErr error
if c.nvidiaCollector != nil {
// This updates the Accelerators field of the stats struct
nvidiaStatsErr = c.nvidiaCollector.UpdateStats(stats)
}
ref, err := c.handler.ContainerReference()
if err != nil {
// Ignore errors if the container is dead.
@ -572,6 +582,9 @@ func (c *containerData) updateStats() error {
if statsErr != nil {
return statsErr
}
if nvidiaStatsErr != nil {
return nvidiaStatsErr
}
return customStatsErr
}

View File

@ -29,6 +29,8 @@ import (
info "github.com/google/cadvisor/info/v1"
itest "github.com/google/cadvisor/info/v1/test"
"github.com/google/cadvisor/accelerators"
"github.com/mindprince/gonvml"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
@ -205,3 +207,22 @@ func TestGetInfo(t *testing.T) {
t.Errorf("received wrong container name: received %v; should be %v", info.Name, mockHandler.Name)
}
}
func TestUpdateNvidiaStats(t *testing.T) {
cd, _, _ := newTestContainerData(t)
stats := info.ContainerStats{}
// When there are no devices, we should not get an error and stats should not change.
cd.nvidiaCollector = &accelerators.NvidiaCollector{}
err := cd.nvidiaCollector.UpdateStats(&stats)
assert.Nil(t, err)
assert.Equal(t, info.ContainerStats{}, stats)
// This is an impossible situation (there are devices but nvml is not initialized).
// Here I am testing that the CGo gonvml library doesn't panic when passed bad
// input and instead returns an error.
cd.nvidiaCollector = &accelerators.NvidiaCollector{Devices: []gonvml.Device{{}, {}}}
err = cd.nvidiaCollector.UpdateStats(&stats)
assert.NotNil(t, err)
assert.Equal(t, info.ContainerStats{}, stats)
}

View File

@ -18,6 +18,7 @@ package manager
import (
"flag"
"fmt"
"net/http"
"os"
"path"
"strconv"
@ -25,6 +26,7 @@ import (
"sync"
"time"
"github.com/google/cadvisor/accelerators"
"github.com/google/cadvisor/cache/memory"
"github.com/google/cadvisor/collector"
"github.com/google/cadvisor/container"
@ -45,8 +47,6 @@ import (
"github.com/google/cadvisor/utils/sysfs"
"github.com/google/cadvisor/version"
"net/http"
"github.com/golang/glog"
"github.com/opencontainers/runc/libcontainer/cgroups"
)
@ -212,6 +212,7 @@ func New(memoryCache *memory.InMemoryCache, sysfs sysfs.SysFs, maxHousekeepingIn
containerWatchers: []watcher.ContainerWatcher{},
eventsChannel: eventsChannel,
collectorHttpClient: collectorHttpClient,
nvidiaManager: &accelerators.NvidiaManager{},
}
machineInfo, err := machine.Info(sysfs, fsInfo, inHostNamespace)
@ -257,6 +258,7 @@ type manager struct {
containerWatchers []watcher.ContainerWatcher
eventsChannel chan watcher.ContainerEvent
collectorHttpClient *http.Client
nvidiaManager accelerators.AcceleratorManager
}
// Start the container manager.
@ -309,6 +311,9 @@ func (self *manager) Start() error {
return nil
}
// Setup collection of nvidia GPU metrics if any of them are attached to the machine.
self.nvidiaManager.Setup()
// Create root and then recover all containers.
err = self.createContainer("/", watcher.Raw)
if err != nil {
@ -338,6 +343,7 @@ func (self *manager) Start() error {
}
func (self *manager) Stop() error {
defer self.nvidiaManager.Destroy()
// Stop and wait on all quit channels.
for i, c := range self.quitChannels {
// Send the exit signal and wait on the thread to exit (by closing the channel).
@ -917,6 +923,15 @@ func (m *manager) createContainerLocked(containerName string, watchSource watche
if err != nil {
return err
}
devicesCgroupPath, err := handler.GetCgroupPath("devices")
if err != nil {
glog.Infof("Error getting devices cgroup path: %v", err)
} else {
cont.nvidiaCollector, err = m.nvidiaManager.GetCollector(devicesCgroupPath)
if err != nil {
glog.Infof("GPU metrics may be unavailable/incomplete for container %q: %v", cont.info.Name, err)
}
}
// Add collectors
labels := handler.GetContainerLabels()