Merge pull request #745 from rjnagal/summary
Separate in-memory cache from storage drivers.
This commit is contained in:
commit
c3f15e3b6a
33
cache/cache.go
vendored
Normal file
33
cache/cache.go
vendored
Normal file
@ -0,0 +1,33 @@
|
|||||||
|
// Copyright 2015 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 cache
|
||||||
|
|
||||||
|
import info "github.com/google/cadvisor/info/v1"
|
||||||
|
|
||||||
|
type Cache interface {
|
||||||
|
AddStats(ref info.ContainerReference, stats *info.ContainerStats) error
|
||||||
|
|
||||||
|
// Read most recent stats. numStats indicates max number of stats
|
||||||
|
// returned. The returned stats must be consecutive observed stats. If
|
||||||
|
// numStats < 0, then return all stats stored in the storage. The
|
||||||
|
// returned stats should be sorted in time increasing order, i.e. Most
|
||||||
|
// recent stats should be the last.
|
||||||
|
RecentStats(containerName string, numStats int) ([]*info.ContainerStats, error)
|
||||||
|
|
||||||
|
// Close will clear the state of the storage driver. The elements
|
||||||
|
// stored in the underlying storage may or may not be deleted depending
|
||||||
|
// on the implementation of the storage driver.
|
||||||
|
Close() error
|
||||||
|
}
|
@ -25,16 +25,16 @@ import (
|
|||||||
"github.com/google/cadvisor/utils"
|
"github.com/google/cadvisor/utils"
|
||||||
)
|
)
|
||||||
|
|
||||||
// TODO(vmarmol): See about refactoring this class, we have an unecessary redirection of containerStorage and InMemoryStorage.
|
// TODO(vmarmol): See about refactoring this class, we have an unecessary redirection of containerCache and InMemoryCache.
|
||||||
// containerStorage is used to store per-container information
|
// containerCache is used to store per-container information
|
||||||
type containerStorage struct {
|
type containerCache struct {
|
||||||
ref info.ContainerReference
|
ref info.ContainerReference
|
||||||
recentStats *utils.TimedStore
|
recentStats *utils.TimedStore
|
||||||
maxAge time.Duration
|
maxAge time.Duration
|
||||||
lock sync.RWMutex
|
lock sync.RWMutex
|
||||||
}
|
}
|
||||||
|
|
||||||
func (self *containerStorage) AddStats(stats *info.ContainerStats) error {
|
func (self *containerCache) AddStats(stats *info.ContainerStats) error {
|
||||||
self.lock.Lock()
|
self.lock.Lock()
|
||||||
defer self.lock.Unlock()
|
defer self.lock.Unlock()
|
||||||
|
|
||||||
@ -43,7 +43,7 @@ func (self *containerStorage) AddStats(stats *info.ContainerStats) error {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (self *containerStorage) RecentStats(start, end time.Time, maxStats int) ([]*info.ContainerStats, error) {
|
func (self *containerCache) RecentStats(start, end time.Time, maxStats int) ([]*info.ContainerStats, error) {
|
||||||
self.lock.RLock()
|
self.lock.RLock()
|
||||||
defer self.lock.RUnlock()
|
defer self.lock.RUnlock()
|
||||||
result := self.recentStats.InTimeRange(start, end, maxStats)
|
result := self.recentStats.InTimeRange(start, end, maxStats)
|
||||||
@ -54,31 +54,31 @@ func (self *containerStorage) RecentStats(start, end time.Time, maxStats int) ([
|
|||||||
return converted, nil
|
return converted, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func newContainerStore(ref info.ContainerReference, maxAge time.Duration) *containerStorage {
|
func newContainerStore(ref info.ContainerReference, maxAge time.Duration) *containerCache {
|
||||||
return &containerStorage{
|
return &containerCache{
|
||||||
ref: ref,
|
ref: ref,
|
||||||
recentStats: utils.NewTimedStore(maxAge, -1),
|
recentStats: utils.NewTimedStore(maxAge, -1),
|
||||||
maxAge: maxAge,
|
maxAge: maxAge,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
type InMemoryStorage struct {
|
type InMemoryCache struct {
|
||||||
lock sync.RWMutex
|
lock sync.RWMutex
|
||||||
containerStorageMap map[string]*containerStorage
|
containerCacheMap map[string]*containerCache
|
||||||
maxAge time.Duration
|
maxAge time.Duration
|
||||||
backend storage.StorageDriver
|
backend storage.StorageDriver
|
||||||
}
|
}
|
||||||
|
|
||||||
func (self *InMemoryStorage) AddStats(ref info.ContainerReference, stats *info.ContainerStats) error {
|
func (self *InMemoryCache) AddStats(ref info.ContainerReference, stats *info.ContainerStats) error {
|
||||||
var cstore *containerStorage
|
var cstore *containerCache
|
||||||
var ok bool
|
var ok bool
|
||||||
|
|
||||||
func() {
|
func() {
|
||||||
self.lock.Lock()
|
self.lock.Lock()
|
||||||
defer self.lock.Unlock()
|
defer self.lock.Unlock()
|
||||||
if cstore, ok = self.containerStorageMap[ref.Name]; !ok {
|
if cstore, ok = self.containerCacheMap[ref.Name]; !ok {
|
||||||
cstore = newContainerStore(ref, self.maxAge)
|
cstore = newContainerStore(ref, self.maxAge)
|
||||||
self.containerStorageMap[ref.Name] = cstore
|
self.containerCacheMap[ref.Name] = cstore
|
||||||
}
|
}
|
||||||
}()
|
}()
|
||||||
|
|
||||||
@ -93,13 +93,13 @@ func (self *InMemoryStorage) AddStats(ref info.ContainerReference, stats *info.C
|
|||||||
return cstore.AddStats(stats)
|
return cstore.AddStats(stats)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (self *InMemoryStorage) RecentStats(name string, start, end time.Time, maxStats int) ([]*info.ContainerStats, error) {
|
func (self *InMemoryCache) RecentStats(name string, start, end time.Time, maxStats int) ([]*info.ContainerStats, error) {
|
||||||
var cstore *containerStorage
|
var cstore *containerCache
|
||||||
var ok bool
|
var ok bool
|
||||||
err := func() error {
|
err := func() error {
|
||||||
self.lock.RLock()
|
self.lock.RLock()
|
||||||
defer self.lock.RUnlock()
|
defer self.lock.RUnlock()
|
||||||
if cstore, ok = self.containerStorageMap[name]; !ok {
|
if cstore, ok = self.containerCacheMap[name]; !ok {
|
||||||
return fmt.Errorf("unable to find data for container %v", name)
|
return fmt.Errorf("unable to find data for container %v", name)
|
||||||
}
|
}
|
||||||
return nil
|
return nil
|
||||||
@ -111,9 +111,9 @@ func (self *InMemoryStorage) RecentStats(name string, start, end time.Time, maxS
|
|||||||
return cstore.RecentStats(start, end, maxStats)
|
return cstore.RecentStats(start, end, maxStats)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (self *InMemoryStorage) Close() error {
|
func (self *InMemoryCache) Close() error {
|
||||||
self.lock.Lock()
|
self.lock.Lock()
|
||||||
self.containerStorageMap = make(map[string]*containerStorage, 32)
|
self.containerCacheMap = make(map[string]*containerCache, 32)
|
||||||
self.lock.Unlock()
|
self.lock.Unlock()
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
@ -121,11 +121,11 @@ func (self *InMemoryStorage) Close() error {
|
|||||||
func New(
|
func New(
|
||||||
maxAge time.Duration,
|
maxAge time.Duration,
|
||||||
backend storage.StorageDriver,
|
backend storage.StorageDriver,
|
||||||
) *InMemoryStorage {
|
) *InMemoryCache {
|
||||||
ret := &InMemoryStorage{
|
ret := &InMemoryCache{
|
||||||
containerStorageMap: make(map[string]*containerStorage, 32),
|
containerCacheMap: make(map[string]*containerCache, 32),
|
||||||
maxAge: maxAge,
|
maxAge: maxAge,
|
||||||
backend: backend,
|
backend: backend,
|
||||||
}
|
}
|
||||||
return ret
|
return ret
|
||||||
}
|
}
|
@ -40,58 +40,58 @@ func makeStat(i int) *info.ContainerStats {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func getRecentStats(t *testing.T, memoryStorage *InMemoryStorage, numStats int) []*info.ContainerStats {
|
func getRecentStats(t *testing.T, memoryCache *InMemoryCache, numStats int) []*info.ContainerStats {
|
||||||
stats, err := memoryStorage.RecentStats(containerName, zero, zero, numStats)
|
stats, err := memoryCache.RecentStats(containerName, zero, zero, numStats)
|
||||||
require.Nil(t, err)
|
require.Nil(t, err)
|
||||||
return stats
|
return stats
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestAddStats(t *testing.T) {
|
func TestAddStats(t *testing.T) {
|
||||||
memoryStorage := New(60*time.Second, nil)
|
memoryCache := New(60*time.Second, nil)
|
||||||
|
|
||||||
assert := assert.New(t)
|
assert := assert.New(t)
|
||||||
assert.Nil(memoryStorage.AddStats(containerRef, makeStat(0)))
|
assert.Nil(memoryCache.AddStats(containerRef, makeStat(0)))
|
||||||
assert.Nil(memoryStorage.AddStats(containerRef, makeStat(1)))
|
assert.Nil(memoryCache.AddStats(containerRef, makeStat(1)))
|
||||||
assert.Nil(memoryStorage.AddStats(containerRef, makeStat(2)))
|
assert.Nil(memoryCache.AddStats(containerRef, makeStat(2)))
|
||||||
assert.Nil(memoryStorage.AddStats(containerRef, makeStat(0)))
|
assert.Nil(memoryCache.AddStats(containerRef, makeStat(0)))
|
||||||
containerRef2 := info.ContainerReference{
|
containerRef2 := info.ContainerReference{
|
||||||
Name: "/container2",
|
Name: "/container2",
|
||||||
}
|
}
|
||||||
assert.Nil(memoryStorage.AddStats(containerRef2, makeStat(0)))
|
assert.Nil(memoryCache.AddStats(containerRef2, makeStat(0)))
|
||||||
assert.Nil(memoryStorage.AddStats(containerRef2, makeStat(1)))
|
assert.Nil(memoryCache.AddStats(containerRef2, makeStat(1)))
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestRecentStatsNoRecentStats(t *testing.T) {
|
func TestRecentStatsNoRecentStats(t *testing.T) {
|
||||||
memoryStorage := makeWithStats(0)
|
memoryCache := makeWithStats(0)
|
||||||
|
|
||||||
_, err := memoryStorage.RecentStats(containerName, zero, zero, 60)
|
_, err := memoryCache.RecentStats(containerName, zero, zero, 60)
|
||||||
assert.NotNil(t, err)
|
assert.NotNil(t, err)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Make an instance of InMemoryStorage with n stats.
|
// Make an instance of InMemoryCache with n stats.
|
||||||
func makeWithStats(n int) *InMemoryStorage {
|
func makeWithStats(n int) *InMemoryCache {
|
||||||
memoryStorage := New(60*time.Second, nil)
|
memoryCache := New(60*time.Second, nil)
|
||||||
|
|
||||||
for i := 0; i < n; i++ {
|
for i := 0; i < n; i++ {
|
||||||
memoryStorage.AddStats(containerRef, makeStat(i))
|
memoryCache.AddStats(containerRef, makeStat(i))
|
||||||
}
|
}
|
||||||
return memoryStorage
|
return memoryCache
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestRecentStatsGetZeroStats(t *testing.T) {
|
func TestRecentStatsGetZeroStats(t *testing.T) {
|
||||||
memoryStorage := makeWithStats(10)
|
memoryCache := makeWithStats(10)
|
||||||
|
|
||||||
assert.Len(t, getRecentStats(t, memoryStorage, 0), 0)
|
assert.Len(t, getRecentStats(t, memoryCache, 0), 0)
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestRecentStatsGetSomeStats(t *testing.T) {
|
func TestRecentStatsGetSomeStats(t *testing.T) {
|
||||||
memoryStorage := makeWithStats(10)
|
memoryCache := makeWithStats(10)
|
||||||
|
|
||||||
assert.Len(t, getRecentStats(t, memoryStorage, 5), 5)
|
assert.Len(t, getRecentStats(t, memoryCache, 5), 5)
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestRecentStatsGetAllStats(t *testing.T) {
|
func TestRecentStatsGetAllStats(t *testing.T) {
|
||||||
memoryStorage := makeWithStats(10)
|
memoryCache := makeWithStats(10)
|
||||||
|
|
||||||
assert.Len(t, getRecentStats(t, memoryStorage, -1), 10)
|
assert.Len(t, getRecentStats(t, memoryCache, -1), 10)
|
||||||
}
|
}
|
@ -28,11 +28,11 @@ import (
|
|||||||
|
|
||||||
"github.com/docker/docker/pkg/units"
|
"github.com/docker/docker/pkg/units"
|
||||||
"github.com/golang/glog"
|
"github.com/golang/glog"
|
||||||
|
"github.com/google/cadvisor/cache/memory"
|
||||||
"github.com/google/cadvisor/collector"
|
"github.com/google/cadvisor/collector"
|
||||||
"github.com/google/cadvisor/container"
|
"github.com/google/cadvisor/container"
|
||||||
info "github.com/google/cadvisor/info/v1"
|
info "github.com/google/cadvisor/info/v1"
|
||||||
"github.com/google/cadvisor/info/v2"
|
"github.com/google/cadvisor/info/v2"
|
||||||
"github.com/google/cadvisor/storage/memory"
|
|
||||||
"github.com/google/cadvisor/summary"
|
"github.com/google/cadvisor/summary"
|
||||||
"github.com/google/cadvisor/utils/cpuload"
|
"github.com/google/cadvisor/utils/cpuload"
|
||||||
)
|
)
|
||||||
@ -56,7 +56,7 @@ type containerInfo struct {
|
|||||||
type containerData struct {
|
type containerData struct {
|
||||||
handler container.ContainerHandler
|
handler container.ContainerHandler
|
||||||
info containerInfo
|
info containerInfo
|
||||||
memoryStorage *memory.InMemoryStorage
|
memoryCache *memory.InMemoryCache
|
||||||
lock sync.Mutex
|
lock sync.Mutex
|
||||||
loadReader cpuload.CpuLoadReader
|
loadReader cpuload.CpuLoadReader
|
||||||
summaryReader *summary.StatsSummary
|
summaryReader *summary.StatsSummary
|
||||||
@ -215,8 +215,8 @@ func (c *containerData) GetProcessList() ([]v2.ProcessInfo, error) {
|
|||||||
return processes, nil
|
return processes, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func newContainerData(containerName string, memoryStorage *memory.InMemoryStorage, handler container.ContainerHandler, loadReader cpuload.CpuLoadReader, logUsage bool, collectorManager collector.CollectorManager) (*containerData, error) {
|
func newContainerData(containerName string, memoryCache *memory.InMemoryCache, handler container.ContainerHandler, loadReader cpuload.CpuLoadReader, logUsage bool, collectorManager collector.CollectorManager) (*containerData, error) {
|
||||||
if memoryStorage == nil {
|
if memoryCache == nil {
|
||||||
return nil, fmt.Errorf("nil memory storage")
|
return nil, fmt.Errorf("nil memory storage")
|
||||||
}
|
}
|
||||||
if handler == nil {
|
if handler == nil {
|
||||||
@ -229,7 +229,7 @@ func newContainerData(containerName string, memoryStorage *memory.InMemoryStorag
|
|||||||
|
|
||||||
cont := &containerData{
|
cont := &containerData{
|
||||||
handler: handler,
|
handler: handler,
|
||||||
memoryStorage: memoryStorage,
|
memoryCache: memoryCache,
|
||||||
housekeepingInterval: *HousekeepingInterval,
|
housekeepingInterval: *HousekeepingInterval,
|
||||||
loadReader: loadReader,
|
loadReader: loadReader,
|
||||||
logUsage: logUsage,
|
logUsage: logUsage,
|
||||||
@ -256,7 +256,7 @@ func newContainerData(containerName string, memoryStorage *memory.InMemoryStorag
|
|||||||
func (self *containerData) nextHousekeeping(lastHousekeeping time.Time) time.Time {
|
func (self *containerData) nextHousekeeping(lastHousekeeping time.Time) time.Time {
|
||||||
if *allowDynamicHousekeeping {
|
if *allowDynamicHousekeeping {
|
||||||
var empty time.Time
|
var empty time.Time
|
||||||
stats, err := self.memoryStorage.RecentStats(self.info.Name, empty, empty, 2)
|
stats, err := self.memoryCache.RecentStats(self.info.Name, empty, empty, 2)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
if self.allowErrorLogging() {
|
if self.allowErrorLogging() {
|
||||||
glog.Warningf("Failed to get RecentStats(%q) while determining the next housekeeping: %v", self.info.Name, err)
|
glog.Warningf("Failed to get RecentStats(%q) while determining the next housekeeping: %v", self.info.Name, err)
|
||||||
@ -311,7 +311,7 @@ func (c *containerData) housekeeping() {
|
|||||||
if c.logUsage {
|
if c.logUsage {
|
||||||
const numSamples = 60
|
const numSamples = 60
|
||||||
var empty time.Time
|
var empty time.Time
|
||||||
stats, err := c.memoryStorage.RecentStats(c.info.Name, empty, empty, numSamples)
|
stats, err := c.memoryCache.RecentStats(c.info.Name, empty, empty, numSamples)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
if c.allowErrorLogging() {
|
if c.allowErrorLogging() {
|
||||||
glog.Infof("[%s] Failed to get recent stats for logging usage: %v", c.info.Name, err)
|
glog.Infof("[%s] Failed to get recent stats for logging usage: %v", c.info.Name, err)
|
||||||
@ -434,7 +434,7 @@ func (c *containerData) updateStats() error {
|
|||||||
}
|
}
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
err = c.memoryStorage.AddStats(ref, stats)
|
err = c.memoryCache.AddStats(ref, stats)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
@ -22,11 +22,11 @@ import (
|
|||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/google/cadvisor/cache/memory"
|
||||||
"github.com/google/cadvisor/collector"
|
"github.com/google/cadvisor/collector"
|
||||||
"github.com/google/cadvisor/container"
|
"github.com/google/cadvisor/container"
|
||||||
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/storage/memory"
|
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
"github.com/stretchr/testify/require"
|
"github.com/stretchr/testify/require"
|
||||||
)
|
)
|
||||||
@ -34,25 +34,25 @@ import (
|
|||||||
const containerName = "/container"
|
const containerName = "/container"
|
||||||
|
|
||||||
// Create a containerData instance for a test.
|
// Create a containerData instance for a test.
|
||||||
func setupContainerData(t *testing.T, spec info.ContainerSpec) (*containerData, *container.MockContainerHandler, *memory.InMemoryStorage) {
|
func setupContainerData(t *testing.T, spec info.ContainerSpec) (*containerData, *container.MockContainerHandler, *memory.InMemoryCache) {
|
||||||
mockHandler := container.NewMockContainerHandler(containerName)
|
mockHandler := container.NewMockContainerHandler(containerName)
|
||||||
mockHandler.On("GetSpec").Return(
|
mockHandler.On("GetSpec").Return(
|
||||||
spec,
|
spec,
|
||||||
nil,
|
nil,
|
||||||
)
|
)
|
||||||
memoryStorage := memory.New(60, nil)
|
memoryCache := memory.New(60, nil)
|
||||||
ret, err := newContainerData(containerName, memoryStorage, mockHandler, nil, false, &collector.FakeCollectorManager{})
|
ret, err := newContainerData(containerName, memoryCache, mockHandler, nil, false, &collector.FakeCollectorManager{})
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
return ret, mockHandler, memoryStorage
|
return ret, mockHandler, memoryCache
|
||||||
}
|
}
|
||||||
|
|
||||||
// Create a containerData instance for a test and add a default GetSpec mock.
|
// Create a containerData instance for a test and add a default GetSpec mock.
|
||||||
func newTestContainerData(t *testing.T) (*containerData, *container.MockContainerHandler, *memory.InMemoryStorage) {
|
func newTestContainerData(t *testing.T) (*containerData, *container.MockContainerHandler, *memory.InMemoryCache) {
|
||||||
spec := itest.GenerateRandomContainerSpec(4)
|
spec := itest.GenerateRandomContainerSpec(4)
|
||||||
ret, mockHandler, memoryStorage := setupContainerData(t, spec)
|
ret, mockHandler, memoryCache := setupContainerData(t, spec)
|
||||||
return ret, mockHandler, memoryStorage
|
return ret, mockHandler, memoryCache
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestUpdateSubcontainers(t *testing.T) {
|
func TestUpdateSubcontainers(t *testing.T) {
|
||||||
@ -116,9 +116,9 @@ func TestUpdateSubcontainersWithErrorOnDeadContainer(t *testing.T) {
|
|||||||
mockHandler.AssertExpectations(t)
|
mockHandler.AssertExpectations(t)
|
||||||
}
|
}
|
||||||
|
|
||||||
func checkNumStats(t *testing.T, memoryStorage *memory.InMemoryStorage, numStats int) {
|
func checkNumStats(t *testing.T, memoryCache *memory.InMemoryCache, numStats int) {
|
||||||
var empty time.Time
|
var empty time.Time
|
||||||
stats, err := memoryStorage.RecentStats(containerName, empty, empty, -1)
|
stats, err := memoryCache.RecentStats(containerName, empty, empty, -1)
|
||||||
require.Nil(t, err)
|
require.Nil(t, err)
|
||||||
assert.Len(t, stats, numStats)
|
assert.Len(t, stats, numStats)
|
||||||
}
|
}
|
||||||
@ -127,7 +127,7 @@ func TestUpdateStats(t *testing.T) {
|
|||||||
statsList := itest.GenerateRandomStats(1, 4, 1*time.Second)
|
statsList := itest.GenerateRandomStats(1, 4, 1*time.Second)
|
||||||
stats := statsList[0]
|
stats := statsList[0]
|
||||||
|
|
||||||
cd, mockHandler, memoryStorage := newTestContainerData(t)
|
cd, mockHandler, memoryCache := newTestContainerData(t)
|
||||||
mockHandler.On("GetStats").Return(
|
mockHandler.On("GetStats").Return(
|
||||||
stats,
|
stats,
|
||||||
nil,
|
nil,
|
||||||
@ -138,7 +138,7 @@ func TestUpdateStats(t *testing.T) {
|
|||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
|
|
||||||
checkNumStats(t, memoryStorage, 1)
|
checkNumStats(t, memoryCache, 1)
|
||||||
mockHandler.AssertExpectations(t)
|
mockHandler.AssertExpectations(t)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -27,6 +27,7 @@ import (
|
|||||||
|
|
||||||
"github.com/docker/libcontainer/cgroups"
|
"github.com/docker/libcontainer/cgroups"
|
||||||
"github.com/golang/glog"
|
"github.com/golang/glog"
|
||||||
|
"github.com/google/cadvisor/cache/memory"
|
||||||
"github.com/google/cadvisor/collector"
|
"github.com/google/cadvisor/collector"
|
||||||
"github.com/google/cadvisor/container"
|
"github.com/google/cadvisor/container"
|
||||||
"github.com/google/cadvisor/container/docker"
|
"github.com/google/cadvisor/container/docker"
|
||||||
@ -35,7 +36,6 @@ import (
|
|||||||
"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/info/v2"
|
"github.com/google/cadvisor/info/v2"
|
||||||
"github.com/google/cadvisor/storage/memory"
|
|
||||||
"github.com/google/cadvisor/utils/cpuload"
|
"github.com/google/cadvisor/utils/cpuload"
|
||||||
"github.com/google/cadvisor/utils/oomparser"
|
"github.com/google/cadvisor/utils/oomparser"
|
||||||
"github.com/google/cadvisor/utils/sysfs"
|
"github.com/google/cadvisor/utils/sysfs"
|
||||||
@ -113,8 +113,8 @@ type Manager interface {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// New takes a memory storage and returns a new manager.
|
// New takes a memory storage and returns a new manager.
|
||||||
func New(memoryStorage *memory.InMemoryStorage, sysfs sysfs.SysFs) (Manager, error) {
|
func New(memoryCache *memory.InMemoryCache, sysfs sysfs.SysFs) (Manager, error) {
|
||||||
if memoryStorage == nil {
|
if memoryCache == nil {
|
||||||
return nil, fmt.Errorf("manager requires memory storage")
|
return nil, fmt.Errorf("manager requires memory storage")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -133,7 +133,7 @@ func New(memoryStorage *memory.InMemoryStorage, sysfs sysfs.SysFs) (Manager, err
|
|||||||
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),
|
||||||
memoryStorage: memoryStorage,
|
memoryCache: memoryCache,
|
||||||
fsInfo: fsInfo,
|
fsInfo: fsInfo,
|
||||||
cadvisorContainer: selfContainer,
|
cadvisorContainer: selfContainer,
|
||||||
startupTime: time.Now(),
|
startupTime: time.Now(),
|
||||||
@ -169,7 +169,7 @@ type namespacedContainerName struct {
|
|||||||
type manager struct {
|
type manager struct {
|
||||||
containers map[namespacedContainerName]*containerData
|
containers map[namespacedContainerName]*containerData
|
||||||
containersLock sync.RWMutex
|
containersLock sync.RWMutex
|
||||||
memoryStorage *memory.InMemoryStorage
|
memoryCache *memory.InMemoryCache
|
||||||
fsInfo fs.FsInfo
|
fsInfo fs.FsInfo
|
||||||
machineInfo info.MachineInfo
|
machineInfo info.MachineInfo
|
||||||
versionInfo info.VersionInfo
|
versionInfo info.VersionInfo
|
||||||
@ -412,7 +412,7 @@ func (self *manager) containerDataToContainerInfo(cont *containerData, query *in
|
|||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
stats, err := self.memoryStorage.RecentStats(cinfo.Name, query.Start, query.End, query.NumStats)
|
stats, err := self.memoryCache.RecentStats(cinfo.Name, query.Start, query.End, query.NumStats)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
@ -597,7 +597,7 @@ func (self *manager) getRequestedContainers(containerName string, options v2.Req
|
|||||||
func (self *manager) GetFsInfo(label string) ([]v2.FsInfo, error) {
|
func (self *manager) GetFsInfo(label string) ([]v2.FsInfo, error) {
|
||||||
var empty time.Time
|
var empty time.Time
|
||||||
// Get latest data from filesystems hanging off root container.
|
// Get latest data from filesystems hanging off root container.
|
||||||
stats, err := self.memoryStorage.RecentStats("/", empty, empty, 1)
|
stats, err := self.memoryCache.RecentStats("/", empty, empty, 1)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
@ -696,7 +696,7 @@ func (m *manager) createContainer(containerName string) error {
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
logUsage := *logCadvisorUsage && containerName == m.cadvisorContainer
|
logUsage := *logCadvisorUsage && containerName == m.cadvisorContainer
|
||||||
cont, err := newContainerData(containerName, m.memoryStorage, handler, m.loadReader, logUsage, collectorManager)
|
cont, err := newContainerData(containerName, m.memoryCache, handler, m.loadReader, logUsage, collectorManager)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
@ -22,19 +22,19 @@ import (
|
|||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/google/cadvisor/cache/memory"
|
||||||
"github.com/google/cadvisor/collector"
|
"github.com/google/cadvisor/collector"
|
||||||
"github.com/google/cadvisor/container"
|
"github.com/google/cadvisor/container"
|
||||||
"github.com/google/cadvisor/container/docker"
|
"github.com/google/cadvisor/container/docker"
|
||||||
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/storage/memory"
|
|
||||||
"github.com/google/cadvisor/utils/sysfs/fakesysfs"
|
"github.com/google/cadvisor/utils/sysfs/fakesysfs"
|
||||||
)
|
)
|
||||||
|
|
||||||
// TODO(vmarmol): Refactor these tests.
|
// TODO(vmarmol): Refactor these tests.
|
||||||
|
|
||||||
func createManagerAndAddContainers(
|
func createManagerAndAddContainers(
|
||||||
memoryStorage *memory.InMemoryStorage,
|
memoryCache *memory.InMemoryCache,
|
||||||
sysfs *fakesysfs.FakeSysFs,
|
sysfs *fakesysfs.FakeSysFs,
|
||||||
containers []string,
|
containers []string,
|
||||||
f func(*container.MockContainerHandler),
|
f func(*container.MockContainerHandler),
|
||||||
@ -42,9 +42,9 @@ func createManagerAndAddContainers(
|
|||||||
) *manager {
|
) *manager {
|
||||||
container.ClearContainerHandlerFactories()
|
container.ClearContainerHandlerFactories()
|
||||||
mif := &manager{
|
mif := &manager{
|
||||||
containers: make(map[namespacedContainerName]*containerData),
|
containers: make(map[namespacedContainerName]*containerData),
|
||||||
quitChannels: make([]chan error, 0, 2),
|
quitChannels: make([]chan error, 0, 2),
|
||||||
memoryStorage: memoryStorage,
|
memoryCache: memoryCache,
|
||||||
}
|
}
|
||||||
for _, name := range containers {
|
for _, name := range containers {
|
||||||
mockHandler := container.NewMockContainerHandler(name)
|
mockHandler := container.NewMockContainerHandler(name)
|
||||||
@ -53,7 +53,7 @@ func createManagerAndAddContainers(
|
|||||||
spec,
|
spec,
|
||||||
nil,
|
nil,
|
||||||
).Once()
|
).Once()
|
||||||
cont, err := newContainerData(name, memoryStorage, mockHandler, nil, false, &collector.FakeCollectorManager{})
|
cont, err := newContainerData(name, memoryCache, mockHandler, nil, false, &collector.FakeCollectorManager{})
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
@ -82,10 +82,10 @@ func expectManagerWithContainers(containers []string, query *info.ContainerInfoR
|
|||||||
infosMap[container] = itest.GenerateRandomContainerInfo(container, 4, query, 1*time.Second)
|
infosMap[container] = itest.GenerateRandomContainerInfo(container, 4, query, 1*time.Second)
|
||||||
}
|
}
|
||||||
|
|
||||||
memoryStorage := memory.New(time.Duration(query.NumStats)*time.Second, nil)
|
memoryCache := memory.New(time.Duration(query.NumStats)*time.Second, nil)
|
||||||
sysfs := &fakesysfs.FakeSysFs{}
|
sysfs := &fakesysfs.FakeSysFs{}
|
||||||
m := createManagerAndAddContainers(
|
m := createManagerAndAddContainers(
|
||||||
memoryStorage,
|
memoryCache,
|
||||||
sysfs,
|
sysfs,
|
||||||
containers,
|
containers,
|
||||||
func(h *container.MockContainerHandler) {
|
func(h *container.MockContainerHandler) {
|
||||||
@ -95,7 +95,7 @@ func expectManagerWithContainers(containers []string, query *info.ContainerInfoR
|
|||||||
t.Error(err)
|
t.Error(err)
|
||||||
}
|
}
|
||||||
for _, stat := range cinfo.Stats {
|
for _, stat := range cinfo.Stats {
|
||||||
err = memoryStorage.AddStats(ref, stat)
|
err = memoryCache.AddStats(ref, stat)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Error(err)
|
t.Error(err)
|
||||||
}
|
}
|
||||||
|
@ -260,11 +260,6 @@ func (self *bigqueryStorage) AddStats(ref info.ContainerReference, stats *info.C
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Recent stats is not required to be implemented by any storage driver other than the in-memory cache.
|
|
||||||
func (self *bigqueryStorage) RecentStats(containerName string, numStats int) ([]*info.ContainerStats, error) {
|
|
||||||
return nil, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func (self *bigqueryStorage) Close() error {
|
func (self *bigqueryStorage) Close() error {
|
||||||
self.client.Close()
|
self.client.Close()
|
||||||
self.client = nil
|
self.client = nil
|
||||||
|
@ -229,8 +229,3 @@ func New(machineName,
|
|||||||
ret.readyToFlush = ret.defaultReadyToFlush
|
ret.readyToFlush = ret.defaultReadyToFlush
|
||||||
return ret, nil
|
return ret, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// RecentStats is only implemented by in-memory cache storage.
|
|
||||||
func (self *influxdbStorage) RecentStats(containerName string, numStats int) ([]*info.ContainerStats, error) {
|
|
||||||
return nil, nil
|
|
||||||
}
|
|
||||||
|
@ -89,11 +89,6 @@ func (self *redisStorage) AddStats(ref info.ContainerReference, stats *info.Cont
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// RecentStats is only implemented by in-memory cache storage.
|
|
||||||
func (self *redisStorage) RecentStats(containerName string, numStats int) ([]*info.ContainerStats, error) {
|
|
||||||
return nil, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func (self *redisStorage) Close() error {
|
func (self *redisStorage) Close() error {
|
||||||
return self.conn.Close()
|
return self.conn.Close()
|
||||||
}
|
}
|
||||||
|
@ -19,16 +19,6 @@ import info "github.com/google/cadvisor/info/v1"
|
|||||||
type StorageDriver interface {
|
type StorageDriver interface {
|
||||||
AddStats(ref info.ContainerReference, stats *info.ContainerStats) error
|
AddStats(ref info.ContainerReference, stats *info.ContainerStats) error
|
||||||
|
|
||||||
// TODO(rjnagal): RecentStats() is only required by in-memory cache
|
|
||||||
// storage. Refactor and remove from the interface.
|
|
||||||
//
|
|
||||||
// Read most recent stats. numStats indicates max number of stats
|
|
||||||
// returned. The returned stats must be consecutive observed stats. If
|
|
||||||
// numStats < 0, then return all stats stored in the storage. The
|
|
||||||
// returned stats should be sorted in time increasing order, i.e. Most
|
|
||||||
// recent stats should be the last.
|
|
||||||
RecentStats(containerName string, numStats int) ([]*info.ContainerStats, error)
|
|
||||||
|
|
||||||
// Close will clear the state of the storage driver. The elements
|
// Close will clear the state of the storage driver. The elements
|
||||||
// stored in the underlying storage may or may not be deleted depending
|
// stored in the underlying storage may or may not be deleted depending
|
||||||
// on the implementation of the storage driver.
|
// on the implementation of the storage driver.
|
||||||
|
@ -29,11 +29,6 @@ func (self *MockStorageDriver) AddStats(ref info.ContainerReference, stats *info
|
|||||||
return args.Error(0)
|
return args.Error(0)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (self *MockStorageDriver) RecentStats(containerName string, numStats int) ([]*info.ContainerStats, error) {
|
|
||||||
args := self.Called(containerName, numStats)
|
|
||||||
return args.Get(0).([]*info.ContainerStats), args.Error(1)
|
|
||||||
}
|
|
||||||
|
|
||||||
func (self *MockStorageDriver) Close() error {
|
func (self *MockStorageDriver) Close() error {
|
||||||
if self.MockCloseMethod {
|
if self.MockCloseMethod {
|
||||||
args := self.Called()
|
args := self.Called()
|
||||||
|
@ -147,126 +147,3 @@ func StorageDriverFillRandomStatsFunc(
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func StorageDriverTestRetrievePartialRecentStats(driver TestStorageDriver, t *testing.T) {
|
|
||||||
defer driver.Close()
|
|
||||||
N := 100
|
|
||||||
memTrace := make([]uint64, N)
|
|
||||||
cpuTrace := make([]uint64, N)
|
|
||||||
for i := 0; i < N; i++ {
|
|
||||||
memTrace[i] = uint64(i + 1)
|
|
||||||
cpuTrace[i] = uint64(1)
|
|
||||||
}
|
|
||||||
|
|
||||||
ref := info.ContainerReference{
|
|
||||||
Name: "container",
|
|
||||||
}
|
|
||||||
|
|
||||||
trace := buildTrace(cpuTrace, memTrace, 1*time.Second)
|
|
||||||
|
|
||||||
for _, stats := range trace {
|
|
||||||
driver.AddStats(ref, stats)
|
|
||||||
}
|
|
||||||
|
|
||||||
recentStats, err := driver.RecentStats(ref.Name, 10)
|
|
||||||
if err != nil {
|
|
||||||
t.Fatal(err)
|
|
||||||
}
|
|
||||||
if len(recentStats) == 0 {
|
|
||||||
t.Fatal("should at least store one stats")
|
|
||||||
}
|
|
||||||
|
|
||||||
if len(recentStats) > 10 {
|
|
||||||
t.Fatalf("returned %v stats, not 10.", len(recentStats))
|
|
||||||
}
|
|
||||||
|
|
||||||
actualRecentStats := trace[len(trace)-len(recentStats):]
|
|
||||||
|
|
||||||
// The returned stats should be sorted in time increasing order
|
|
||||||
for i, s := range actualRecentStats {
|
|
||||||
r := recentStats[i]
|
|
||||||
if !driver.StatsEq(s, r) {
|
|
||||||
t.Errorf("unexpected stats %+v with memory usage %v; should be %+v", r, r.Memory.Usage, s)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func StorageDriverTestRetrieveAllRecentStats(driver TestStorageDriver, t *testing.T) {
|
|
||||||
defer driver.Close()
|
|
||||||
N := 100
|
|
||||||
memTrace := make([]uint64, N)
|
|
||||||
cpuTrace := make([]uint64, N)
|
|
||||||
for i := 0; i < N; i++ {
|
|
||||||
memTrace[i] = uint64(i + 1)
|
|
||||||
cpuTrace[i] = uint64(1)
|
|
||||||
}
|
|
||||||
|
|
||||||
ref := info.ContainerReference{
|
|
||||||
Name: "container",
|
|
||||||
}
|
|
||||||
|
|
||||||
trace := buildTrace(cpuTrace, memTrace, 1*time.Second)
|
|
||||||
|
|
||||||
for _, stats := range trace {
|
|
||||||
driver.AddStats(ref, stats)
|
|
||||||
}
|
|
||||||
|
|
||||||
recentStats, err := driver.RecentStats(ref.Name, -1)
|
|
||||||
if err != nil {
|
|
||||||
t.Fatal(err)
|
|
||||||
}
|
|
||||||
if len(recentStats) == 0 {
|
|
||||||
t.Fatal("should at least store one stats")
|
|
||||||
}
|
|
||||||
if len(recentStats) > N {
|
|
||||||
t.Fatalf("returned %v stats, not %d.", len(recentStats), N)
|
|
||||||
}
|
|
||||||
|
|
||||||
actualRecentStats := trace[len(trace)-len(recentStats):]
|
|
||||||
|
|
||||||
// The returned stats should be sorted in time increasing order
|
|
||||||
for i, s := range actualRecentStats {
|
|
||||||
r := recentStats[i]
|
|
||||||
if !driver.StatsEq(s, r) {
|
|
||||||
t.Errorf("unexpected stats %+v with memory usage %v", r, r.Memory.Usage)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func StorageDriverTestNoRecentStats(driver TestStorageDriver, t *testing.T) {
|
|
||||||
defer driver.Close()
|
|
||||||
nonExistContainer := "somerandomecontainer"
|
|
||||||
stats, _ := driver.RecentStats(nonExistContainer, -1)
|
|
||||||
if len(stats) > 0 {
|
|
||||||
t.Errorf("RecentStats() returns %v stats on non exist container", len(stats))
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func StorageDriverTestRetrieveZeroRecentStats(driver TestStorageDriver, t *testing.T) {
|
|
||||||
defer driver.Close()
|
|
||||||
N := 100
|
|
||||||
memTrace := make([]uint64, N)
|
|
||||||
cpuTrace := make([]uint64, N)
|
|
||||||
for i := 0; i < N; i++ {
|
|
||||||
memTrace[i] = uint64(i + 1)
|
|
||||||
cpuTrace[i] = uint64(1)
|
|
||||||
}
|
|
||||||
|
|
||||||
ref := info.ContainerReference{
|
|
||||||
Name: "container",
|
|
||||||
}
|
|
||||||
|
|
||||||
trace := buildTrace(cpuTrace, memTrace, 1*time.Second)
|
|
||||||
|
|
||||||
for _, stats := range trace {
|
|
||||||
driver.AddStats(ref, stats)
|
|
||||||
}
|
|
||||||
|
|
||||||
recentStats, err := driver.RecentStats(ref.Name, 0)
|
|
||||||
if err != nil {
|
|
||||||
t.Fatal(err)
|
|
||||||
}
|
|
||||||
if len(recentStats) > 0 {
|
|
||||||
t.Errorf("RecentStats() returns %v stats when requests for 0 stats", len(recentStats))
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
@ -21,10 +21,10 @@ import (
|
|||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/golang/glog"
|
"github.com/golang/glog"
|
||||||
|
"github.com/google/cadvisor/cache/memory"
|
||||||
"github.com/google/cadvisor/storage"
|
"github.com/google/cadvisor/storage"
|
||||||
"github.com/google/cadvisor/storage/bigquery"
|
"github.com/google/cadvisor/storage/bigquery"
|
||||||
"github.com/google/cadvisor/storage/influxdb"
|
"github.com/google/cadvisor/storage/influxdb"
|
||||||
"github.com/google/cadvisor/storage/memory"
|
|
||||||
"github.com/google/cadvisor/storage/redis"
|
"github.com/google/cadvisor/storage/redis"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -38,8 +38,8 @@ var argDbBufferDuration = flag.Duration("storage_driver_buffer_duration", 60*tim
|
|||||||
var storageDuration = flag.Duration("storage_duration", 2*time.Minute, "How long to keep data stored (Default: 2min).")
|
var storageDuration = flag.Duration("storage_duration", 2*time.Minute, "How long to keep data stored (Default: 2min).")
|
||||||
|
|
||||||
// Creates a memory storage with an optional backend storage option.
|
// Creates a memory storage with an optional backend storage option.
|
||||||
func NewMemoryStorage(backendStorageName string) (*memory.InMemoryStorage, error) {
|
func NewMemoryStorage(backendStorageName string) (*memory.InMemoryCache, error) {
|
||||||
var storageDriver *memory.InMemoryStorage
|
var storageDriver *memory.InMemoryCache
|
||||||
var backendStorage storage.StorageDriver
|
var backendStorage storage.StorageDriver
|
||||||
var err error
|
var err error
|
||||||
switch backendStorageName {
|
switch backendStorageName {
|
||||||
|
@ -19,7 +19,7 @@ import (
|
|||||||
)
|
)
|
||||||
|
|
||||||
// Manages a buffer of usage samples.
|
// Manages a buffer of usage samples.
|
||||||
// This is similar to stats buffer in storage/memory.
|
// This is similar to stats buffer in cache/memory.
|
||||||
// The main difference is that we do not pre-allocate the buffer as most containers
|
// The main difference is that we do not pre-allocate the buffer as most containers
|
||||||
// won't live that long.
|
// won't live that long.
|
||||||
type SamplesBuffer struct {
|
type SamplesBuffer struct {
|
||||||
|
Loading…
Reference in New Issue
Block a user