Browse Source

Merge pull request #25737 from Microsoft/jjh-statistics

Windows: Add support for docker stats
John Howard 8 years ago
parent
commit
4a0419f536

+ 89 - 29
api/types/stats.go

@@ -4,7 +4,8 @@ package types
 
 import "time"
 
-// ThrottlingData stores CPU throttling stats of one running container
+// ThrottlingData stores CPU throttling stats of one running container.
+// Not used on Windows.
 type ThrottlingData struct {
 	// Number of periods with throttling active
 	Periods uint64 `json:"periods"`
@@ -17,42 +18,68 @@ type ThrottlingData struct {
 // CPUUsage stores All CPU stats aggregated since container inception.
 type CPUUsage struct {
 	// Total CPU time consumed.
-	// Units: nanoseconds.
+	// Units: nanoseconds (Linux)
+	// Units: 100's of nanoseconds (Windows)
 	TotalUsage uint64 `json:"total_usage"`
-	// Total CPU time consumed per core.
-	// Units: nanoseconds.
-	PercpuUsage []uint64 `json:"percpu_usage"`
-	// Time spent by tasks of the cgroup in kernel mode.
+
+	// Total CPU time consumed per core (Linux). Not used on Windows.
 	// Units: nanoseconds.
+	PercpuUsage []uint64 `json:"percpu_usage,omitempty"`
+
+	// Time spent by tasks of the cgroup in kernel mode (Linux).
+	// Time spent by all container processes in kernel mode (Windows).
+	// Units: nanoseconds (Linux).
+	// Units: 100's of nanoseconds (Windows). Not populated for Hyper-V Containers.
 	UsageInKernelmode uint64 `json:"usage_in_kernelmode"`
-	// Time spent by tasks of the cgroup in user mode.
-	// Units: nanoseconds.
+
+	// Time spent by tasks of the cgroup in user mode (Linux).
+	// Time spent by all container processes in user mode (Windows).
+	// Units: nanoseconds (Linux).
+	// Units: 100's of nanoseconds (Windows). Not populated for Hyper-V Containers
 	UsageInUsermode uint64 `json:"usage_in_usermode"`
 }
 
 // CPUStats aggregates and wraps all CPU related info of container
 type CPUStats struct {
-	CPUUsage       CPUUsage       `json:"cpu_usage"`
-	SystemUsage    uint64         `json:"system_cpu_usage"`
+	// CPU Usage. Linux and Windows.
+	CPUUsage CPUUsage `json:"cpu_usage"`
+
+	// System Usage. Linux only.
+	SystemUsage uint64 `json:"system_cpu_usage,omitempty"`
+
+	// Throttling Data. Linux only.
 	ThrottlingData ThrottlingData `json:"throttling_data,omitempty"`
 }
 
-// MemoryStats aggregates All memory stats since container inception
+// MemoryStats aggregates all memory stats since container inception on Linux.
+// Windows returns stats for commit and private working set only.
 type MemoryStats struct {
+	// Linux Memory Stats
+
 	// current res_counter usage for memory
-	Usage uint64 `json:"usage"`
+	Usage uint64 `json:"usage,omitempty"`
 	// maximum usage ever recorded.
-	MaxUsage uint64 `json:"max_usage"`
+	MaxUsage uint64 `json:"max_usage,omitempty"`
 	// TODO(vishh): Export these as stronger types.
 	// all the stats exported via memory.stat.
-	Stats map[string]uint64 `json:"stats"`
+	Stats map[string]uint64 `json:"stats,omitempty"`
 	// number of times memory usage hits limits.
-	Failcnt uint64 `json:"failcnt"`
-	Limit   uint64 `json:"limit"`
+	Failcnt uint64 `json:"failcnt,omitempty"`
+	Limit   uint64 `json:"limit,omitempty"`
+
+	// Windows Memory Stats
+	// See https://technet.microsoft.com/en-us/magazine/ff382715.aspx
+
+	// committed bytes
+	Commit uint64 `json:"commitbytes,omitempty"`
+	// peak committed bytes
+	CommitPeak uint64 `json:"commitpeakbytes,omitempty"`
+	// private working set
+	PrivateWorkingSet uint64 `json:"privateworkingset,omitempty"`
 }
 
 // BlkioStatEntry is one small entity to store a piece of Blkio stats
-// TODO Windows: This can be factored out
+// Not used on Windows.
 type BlkioStatEntry struct {
 	Major uint64 `json:"major"`
 	Minor uint64 `json:"minor"`
@@ -60,8 +87,10 @@ type BlkioStatEntry struct {
 	Value uint64 `json:"value"`
 }
 
-// BlkioStats stores All IO service stats for data read and write
-// TODO Windows: This can be factored out
+// BlkioStats stores All IO service stats for data read and write.
+// This is a Linux specific structure as the differences between expressing
+// block I/O on Windows and Linux are sufficiently significant to make
+// little sense attempting to morph into a combined structure.
 type BlkioStats struct {
 	// number of bytes transferred to and from the block device
 	IoServiceBytesRecursive []BlkioStatEntry `json:"io_service_bytes_recursive"`
@@ -74,17 +103,38 @@ type BlkioStats struct {
 	SectorsRecursive        []BlkioStatEntry `json:"sectors_recursive"`
 }
 
-// NetworkStats aggregates All network stats of one container
-// TODO Windows: This will require refactoring
+// StorageStats is the disk I/O stats for read/write on Windows.
+type StorageStats struct {
+	ReadCountNormalized  uint64 `json:"read_count_normalized,omitempty"`
+	ReadSizeBytes        uint64 `json:"read_size_bytes,omitempty"`
+	WriteCountNormalized uint64 `json:"write_count_normalized,omitempty"`
+	WriteSizeBytes       uint64 `json:"write_size_bytes,omitempty"`
+}
+
+// NetworkStats aggregates the network stats of one container
 type NetworkStats struct {
-	RxBytes   uint64 `json:"rx_bytes"`
+	// Bytes received. Windows and Linux.
+	RxBytes uint64 `json:"rx_bytes"`
+	// Packets received. Windows and Linux.
 	RxPackets uint64 `json:"rx_packets"`
-	RxErrors  uint64 `json:"rx_errors"`
+	// Received errors. Not used on Windows. Note that we dont `omitempty` this
+	// field as it is expected in the >=v1.21 API stats structure.
+	RxErrors uint64 `json:"rx_errors"`
+	// Incoming packets dropped. Windows and Linux.
 	RxDropped uint64 `json:"rx_dropped"`
-	TxBytes   uint64 `json:"tx_bytes"`
+	// Bytes sent. Windows and Linux.
+	TxBytes uint64 `json:"tx_bytes"`
+	// Packets sent. Windows and Linux.
 	TxPackets uint64 `json:"tx_packets"`
-	TxErrors  uint64 `json:"tx_errors"`
+	// Sent errors. Not used on Windows. Note that we dont `omitempty` this
+	// field as it is expected in the >=v1.21 API stats structure.
+	TxErrors uint64 `json:"tx_errors"`
+	// Outgoing packets dropped. Windows and Linux.
 	TxDropped uint64 `json:"tx_dropped"`
+	// Endpoint ID. Not used on Linux.
+	EndpointID string `json:"endpoint_id,omitempty"`
+	// Instance ID. Not used on Linux.
+	InstanceID string `json:"instance_id,omitempty"`
 }
 
 // PidsStats contains the stats of a container's pids
@@ -98,12 +148,22 @@ type PidsStats struct {
 
 // Stats is Ultimate struct aggregating all types of stats of one container
 type Stats struct {
-	Read        time.Time   `json:"read"`
-	PreCPUStats CPUStats    `json:"precpu_stats,omitempty"`
+	// Common stats
+	Read    time.Time `json:"read"`
+	PreRead time.Time `json:"preread"`
+
+	// Linux specific stats, not populated on Windows.
+	PidsStats  PidsStats  `json:"pids_stats,omitempty"`
+	BlkioStats BlkioStats `json:"blkio_stats,omitempty"`
+
+	// Windows specific stats, not populated on Linux.
+	NumProcs     uint32       `json:"num_procs"`
+	StorageStats StorageStats `json:"storage_stats,omitempty"`
+
+	// Shared stats
 	CPUStats    CPUStats    `json:"cpu_stats,omitempty"`
+	PreCPUStats CPUStats    `json:"precpu_stats,omitempty"` // "Pre"="Previous"
 	MemoryStats MemoryStats `json:"memory_stats,omitempty"`
-	BlkioStats  BlkioStats  `json:"blkio_stats,omitempty"`
-	PidsStats   PidsStats   `json:"pids_stats,omitempty"`
 }
 
 // StatsJSON is newly used Networks

+ 8 - 0
api/types/types.go

@@ -1,6 +1,7 @@
 package types
 
 import (
+	"io"
 	"os"
 	"time"
 
@@ -182,6 +183,13 @@ type ContainerPathStat struct {
 	LinkTarget string      `json:"linkTarget"`
 }
 
+// ContainerStats contains resonse of Remote API:
+// GET "/stats"
+type ContainerStats struct {
+	Body   io.ReadCloser `json:"body"`
+	OSType string        `json:"ostype"`
+}
+
 // ContainerProcessList contains response of Remote API:
 // GET "/containers/{name:.*}/top"
 type ContainerProcessList struct {

+ 9 - 1
cli/command/container/stats.go

@@ -187,7 +187,15 @@ func runStats(dockerCli *command.DockerCli, opts *statsOptions) error {
 			fmt.Fprint(dockerCli.Out(), "\033[2J")
 			fmt.Fprint(dockerCli.Out(), "\033[H")
 		}
-		io.WriteString(w, "CONTAINER\tCPU %\tMEM USAGE / LIMIT\tMEM %\tNET I/O\tBLOCK I/O\tPIDS\n")
+		switch daemonOSType {
+		case "":
+			// Before we have any stats from the daemon, we don't know the platform...
+			io.WriteString(w, "Waiting for statistics...\n")
+		case "windows":
+			io.WriteString(w, "CONTAINER\tCPU %\tPRIV WORKING SET\tNET I/O\tBLOCK I/O\n")
+		default:
+			io.WriteString(w, "CONTAINER\tCPU %\tMEM USAGE / LIMIT\tMEM %\tNET I/O\tBLOCK I/O\tPIDS\n")
+		}
 	}
 
 	for range time.Tick(500 * time.Millisecond) {

+ 101 - 43
cli/command/container/stats_helpers.go

@@ -19,23 +19,29 @@ import (
 type containerStats struct {
 	Name             string
 	CPUPercentage    float64
-	Memory           float64
-	MemoryLimit      float64
-	MemoryPercentage float64
+	Memory           float64 // On Windows this is the private working set
+	MemoryLimit      float64 // Not used on Windows
+	MemoryPercentage float64 // Not used on Windows
 	NetworkRx        float64
 	NetworkTx        float64
 	BlockRead        float64
 	BlockWrite       float64
-	PidsCurrent      uint64
+	PidsCurrent      uint64 // Not used on Windows
 	mu               sync.Mutex
 	err              error
 }
 
 type stats struct {
-	mu sync.Mutex
-	cs []*containerStats
+	mu     sync.Mutex
+	ostype string
+	cs     []*containerStats
 }
 
+// daemonOSType is set once we have at least one stat for a container
+// from the daemon. It is used to ensure we print the right header based
+// on the daemon platform.
+var daemonOSType string
+
 func (s *stats) add(cs *containerStats) bool {
 	s.mu.Lock()
 	defer s.mu.Unlock()
@@ -80,22 +86,28 @@ func (s *containerStats) Collect(ctx context.Context, cli client.APIClient, stre
 		}
 	}()
 
-	responseBody, err := cli.ContainerStats(ctx, s.Name, streamStats)
+	response, err := cli.ContainerStats(ctx, s.Name, streamStats)
 	if err != nil {
 		s.mu.Lock()
 		s.err = err
 		s.mu.Unlock()
 		return
 	}
-	defer responseBody.Close()
+	defer response.Body.Close()
 
-	dec := json.NewDecoder(responseBody)
+	dec := json.NewDecoder(response.Body)
 	go func() {
 		for {
-			var v *types.StatsJSON
+			var (
+				v                 *types.StatsJSON
+				memPercent        = 0.0
+				cpuPercent        = 0.0
+				blkRead, blkWrite uint64 // Only used on Linux
+				mem               = 0.0
+			)
 
 			if err := dec.Decode(&v); err != nil {
-				dec = json.NewDecoder(io.MultiReader(dec.Buffered(), responseBody))
+				dec = json.NewDecoder(io.MultiReader(dec.Buffered(), response.Body))
 				u <- err
 				if err == io.EOF {
 					break
@@ -104,28 +116,38 @@ func (s *containerStats) Collect(ctx context.Context, cli client.APIClient, stre
 				continue
 			}
 
-			var memPercent = 0.0
-			var cpuPercent = 0.0
+			daemonOSType = response.OSType
+
+			if daemonOSType != "windows" {
+				// MemoryStats.Limit will never be 0 unless the container is not running and we haven't
+				// got any data from cgroup
+				if v.MemoryStats.Limit != 0 {
+					memPercent = float64(v.MemoryStats.Usage) / float64(v.MemoryStats.Limit) * 100.0
+				}
+				previousCPU = v.PreCPUStats.CPUUsage.TotalUsage
+				previousSystem = v.PreCPUStats.SystemUsage
+				cpuPercent = calculateCPUPercentUnix(previousCPU, previousSystem, v)
+				blkRead, blkWrite = calculateBlockIO(v.BlkioStats)
+				mem = float64(v.MemoryStats.Usage)
 
-			// MemoryStats.Limit will never be 0 unless the container is not running and we haven't
-			// got any data from cgroup
-			if v.MemoryStats.Limit != 0 {
-				memPercent = float64(v.MemoryStats.Usage) / float64(v.MemoryStats.Limit) * 100.0
+			} else {
+				cpuPercent = calculateCPUPercentWindows(v)
+				blkRead = v.StorageStats.ReadSizeBytes
+				blkWrite = v.StorageStats.WriteSizeBytes
+				mem = float64(v.MemoryStats.PrivateWorkingSet)
 			}
 
-			previousCPU = v.PreCPUStats.CPUUsage.TotalUsage
-			previousSystem = v.PreCPUStats.SystemUsage
-			cpuPercent = calculateCPUPercent(previousCPU, previousSystem, v)
-			blkRead, blkWrite := calculateBlockIO(v.BlkioStats)
 			s.mu.Lock()
 			s.CPUPercentage = cpuPercent
-			s.Memory = float64(v.MemoryStats.Usage)
-			s.MemoryLimit = float64(v.MemoryStats.Limit)
-			s.MemoryPercentage = memPercent
+			s.Memory = mem
 			s.NetworkRx, s.NetworkTx = calculateNetwork(v.Networks)
 			s.BlockRead = float64(blkRead)
 			s.BlockWrite = float64(blkWrite)
-			s.PidsCurrent = v.PidsStats.Current
+			if daemonOSType != "windows" {
+				s.MemoryLimit = float64(v.MemoryStats.Limit)
+				s.MemoryPercentage = memPercent
+				s.PidsCurrent = v.PidsStats.Current
+			}
 			s.mu.Unlock()
 			u <- nil
 			if !streamStats {
@@ -178,29 +200,49 @@ func (s *containerStats) Collect(ctx context.Context, cli client.APIClient, stre
 func (s *containerStats) Display(w io.Writer) error {
 	s.mu.Lock()
 	defer s.mu.Unlock()
-	// NOTE: if you change this format, you must also change the err format below!
-	format := "%s\t%.2f%%\t%s / %s\t%.2f%%\t%s / %s\t%s / %s\t%d\n"
-	if s.err != nil {
-		format = "%s\t%s\t%s / %s\t%s\t%s / %s\t%s / %s\t%s\n"
-		errStr := "--"
+	if daemonOSType == "windows" {
+		// NOTE: if you change this format, you must also change the err format below!
+		format := "%s\t%.2f%%\t%s\t%s / %s\t%s / %s\n"
+		if s.err != nil {
+			format = "%s\t%s\t%s\t%s / %s\t%s / %s\n"
+			errStr := "--"
+			fmt.Fprintf(w, format,
+				s.Name, errStr, errStr, errStr, errStr, errStr, errStr,
+			)
+			err := s.err
+			return err
+		}
+		fmt.Fprintf(w, format,
+			s.Name,
+			s.CPUPercentage,
+			units.BytesSize(s.Memory),
+			units.HumanSizeWithPrecision(s.NetworkRx, 3), units.HumanSizeWithPrecision(s.NetworkTx, 3),
+			units.HumanSizeWithPrecision(s.BlockRead, 3), units.HumanSizeWithPrecision(s.BlockWrite, 3))
+	} else {
+		// NOTE: if you change this format, you must also change the err format below!
+		format := "%s\t%.2f%%\t%s / %s\t%.2f%%\t%s / %s\t%s / %s\t%d\n"
+		if s.err != nil {
+			format = "%s\t%s\t%s / %s\t%s\t%s / %s\t%s / %s\t%s\n"
+			errStr := "--"
+			fmt.Fprintf(w, format,
+				s.Name, errStr, errStr, errStr, errStr, errStr, errStr, errStr, errStr, errStr,
+			)
+			err := s.err
+			return err
+		}
 		fmt.Fprintf(w, format,
-			s.Name, errStr, errStr, errStr, errStr, errStr, errStr, errStr, errStr, errStr,
-		)
-		err := s.err
-		return err
+			s.Name,
+			s.CPUPercentage,
+			units.BytesSize(s.Memory), units.BytesSize(s.MemoryLimit),
+			s.MemoryPercentage,
+			units.HumanSizeWithPrecision(s.NetworkRx, 3), units.HumanSizeWithPrecision(s.NetworkTx, 3),
+			units.HumanSizeWithPrecision(s.BlockRead, 3), units.HumanSizeWithPrecision(s.BlockWrite, 3),
+			s.PidsCurrent)
 	}
-	fmt.Fprintf(w, format,
-		s.Name,
-		s.CPUPercentage,
-		units.BytesSize(s.Memory), units.BytesSize(s.MemoryLimit),
-		s.MemoryPercentage,
-		units.HumanSizeWithPrecision(s.NetworkRx, 3), units.HumanSizeWithPrecision(s.NetworkTx, 3),
-		units.HumanSizeWithPrecision(s.BlockRead, 3), units.HumanSizeWithPrecision(s.BlockWrite, 3),
-		s.PidsCurrent)
 	return nil
 }
 
-func calculateCPUPercent(previousCPU, previousSystem uint64, v *types.StatsJSON) float64 {
+func calculateCPUPercentUnix(previousCPU, previousSystem uint64, v *types.StatsJSON) float64 {
 	var (
 		cpuPercent = 0.0
 		// calculate the change for the cpu usage of the container in between readings
@@ -215,6 +257,22 @@ func calculateCPUPercent(previousCPU, previousSystem uint64, v *types.StatsJSON)
 	return cpuPercent
 }
 
+func calculateCPUPercentWindows(v *types.StatsJSON) float64 {
+	// Max number of 100ns intervals between the previous time read and now
+	possIntervals := uint64(v.Read.Sub(v.PreRead).Nanoseconds()) // Start with number of ns intervals
+	possIntervals /= 100                                         // Convert to number of 100ns intervals
+	possIntervals *= uint64(v.NumProcs)                          // Multiple by the number of processors
+
+	// Intervals used
+	intervalsUsed := v.CPUStats.CPUUsage.TotalUsage - v.PreCPUStats.CPUUsage.TotalUsage
+
+	// Percentage avoiding divide-by-zero
+	if possIntervals > 0 {
+		return float64(intervalsUsed) / float64(possIntervals) * 100.0
+	}
+	return 0.00
+}
+
 func calculateBlockIO(blkio types.BlkioStats) (blkRead uint64, blkWrite uint64) {
 	for _, bioEntry := range blkio.IoServiceBytesRecursive {
 		switch strings.ToLower(bioEntry.Op) {

+ 6 - 4
client/container_stats.go

@@ -1,15 +1,15 @@
 package client
 
 import (
-	"io"
 	"net/url"
 
+	"github.com/docker/docker/api/types"
 	"golang.org/x/net/context"
 )
 
 // ContainerStats returns near realtime stats for a given container.
 // It's up to the caller to close the io.ReadCloser returned.
-func (cli *Client) ContainerStats(ctx context.Context, containerID string, stream bool) (io.ReadCloser, error) {
+func (cli *Client) ContainerStats(ctx context.Context, containerID string, stream bool) (types.ContainerStats, error) {
 	query := url.Values{}
 	query.Set("stream", "0")
 	if stream {
@@ -18,7 +18,9 @@ func (cli *Client) ContainerStats(ctx context.Context, containerID string, strea
 
 	resp, err := cli.get(ctx, "/containers/"+containerID+"/stats", query, nil)
 	if err != nil {
-		return nil, err
+		return types.ContainerStats{}, err
 	}
-	return resp.body, err
+
+	osType := GetDockerOS(resp.header.Get("Server"))
+	return types.ContainerStats{Body: resp.body, OSType: osType}, err
 }

+ 3 - 3
client/container_stats_test.go

@@ -54,12 +54,12 @@ func TestContainerStats(t *testing.T) {
 				}, nil
 			}),
 		}
-		body, err := client.ContainerStats(context.Background(), "container_id", c.stream)
+		resp, err := client.ContainerStats(context.Background(), "container_id", c.stream)
 		if err != nil {
 			t.Fatal(err)
 		}
-		defer body.Close()
-		content, err := ioutil.ReadAll(body)
+		defer resp.Body.Close()
+		content, err := ioutil.ReadAll(resp.Body)
 		if err != nil {
 			t.Fatal(err)
 		}

+ 3 - 2
client/image_build.go

@@ -39,7 +39,7 @@ func (cli *Client) ImageBuild(ctx context.Context, buildContext io.Reader, optio
 		return types.ImageBuildResponse{}, err
 	}
 
-	osType := getDockerOS(serverResp.header.Get("Server"))
+	osType := GetDockerOS(serverResp.header.Get("Server"))
 
 	return types.ImageBuildResponse{
 		Body:   serverResp.body,
@@ -113,7 +113,8 @@ func imageBuildOptionsToQuery(options types.ImageBuildOptions) (url.Values, erro
 	return query, nil
 }
 
-func getDockerOS(serverHeader string) string {
+// GetDockerOS returns the operating system based on the server header from the daemon.
+func GetDockerOS(serverHeader string) string {
 	var osType string
 	matches := headerRegexp.FindStringSubmatch(serverHeader)
 	if len(matches) > 0 {

+ 1 - 1
client/image_build_test.go

@@ -222,7 +222,7 @@ func TestGetDockerOS(t *testing.T) {
 		"Foo/v1.22 (bar)":        "",
 	}
 	for header, os := range cases {
-		g := getDockerOS(header)
+		g := GetDockerOS(header)
 		if g != os {
 			t.Fatalf("Expected %s, got %s", os, g)
 		}

+ 1 - 1
client/interface.go

@@ -51,7 +51,7 @@ type ContainerAPIClient interface {
 	ContainerResize(ctx context.Context, container string, options types.ResizeOptions) error
 	ContainerRestart(ctx context.Context, container string, timeout *time.Duration) error
 	ContainerStatPath(ctx context.Context, container, path string) (types.ContainerPathStat, error)
-	ContainerStats(ctx context.Context, container string, stream bool) (io.ReadCloser, error)
+	ContainerStats(ctx context.Context, container string, stream bool) (types.ContainerStats, error)
 	ContainerStart(ctx context.Context, container string, options types.ContainerStartOptions) error
 	ContainerStop(ctx context.Context, container string, timeout *time.Duration) error
 	ContainerTop(ctx context.Context, container string, arguments []string) (types.ContainerProcessList, error)

+ 57 - 1
daemon/daemon_windows.go

@@ -14,6 +14,7 @@ import (
 	"github.com/docker/docker/image"
 	"github.com/docker/docker/pkg/idtools"
 	"github.com/docker/docker/pkg/parsers"
+	"github.com/docker/docker/pkg/platform"
 	"github.com/docker/docker/pkg/sysinfo"
 	"github.com/docker/docker/pkg/system"
 	"github.com/docker/docker/runconfig"
@@ -379,7 +380,62 @@ func driverOptions(config *Config) []nwconfig.Option {
 }
 
 func (daemon *Daemon) stats(c *container.Container) (*types.StatsJSON, error) {
-	return nil, nil
+	if !c.IsRunning() {
+		return nil, errNotRunning{c.ID}
+	}
+
+	// Obtain the stats from HCS via libcontainerd
+	stats, err := daemon.containerd.Stats(c.ID)
+	if err != nil {
+		return nil, err
+	}
+
+	// Start with an empty structure
+	s := &types.StatsJSON{}
+
+	// Populate the CPU/processor statistics
+	s.CPUStats = types.CPUStats{
+		CPUUsage: types.CPUUsage{
+			TotalUsage:        stats.Processor.TotalRuntime100ns,
+			UsageInKernelmode: stats.Processor.RuntimeKernel100ns,
+			UsageInUsermode:   stats.Processor.RuntimeKernel100ns,
+		},
+	}
+
+	// Populate the memory statistics
+	s.MemoryStats = types.MemoryStats{
+		Commit:            stats.Memory.UsageCommitBytes,
+		CommitPeak:        stats.Memory.UsageCommitPeakBytes,
+		PrivateWorkingSet: stats.Memory.UsagePrivateWorkingSetBytes,
+	}
+
+	// Populate the storage statistics
+	s.StorageStats = types.StorageStats{
+		ReadCountNormalized:  stats.Storage.ReadCountNormalized,
+		ReadSizeBytes:        stats.Storage.ReadSizeBytes,
+		WriteCountNormalized: stats.Storage.WriteCountNormalized,
+		WriteSizeBytes:       stats.Storage.WriteSizeBytes,
+	}
+
+	// Populate the network statistics
+	s.Networks = make(map[string]types.NetworkStats)
+
+	for _, nstats := range stats.Network {
+		s.Networks[nstats.EndpointId] = types.NetworkStats{
+			RxBytes:   nstats.BytesReceived,
+			RxPackets: nstats.PacketsReceived,
+			RxDropped: nstats.DroppedPacketsIncoming,
+			TxBytes:   nstats.BytesSent,
+			TxPackets: nstats.PacketsSent,
+			TxDropped: nstats.DroppedPacketsOutgoing,
+		}
+	}
+
+	// Set the timestamp
+	s.Stats.Read = stats.Timestamp
+	s.Stats.NumProcs = platform.NumProcs()
+
+	return s, nil
 }
 
 // setDefaultIsolation determine the default isolation mode for the

+ 9 - 53
daemon/stats.go

@@ -3,8 +3,8 @@ package daemon
 import (
 	"encoding/json"
 	"errors"
-	"fmt"
 	"runtime"
+	"time"
 
 	"golang.org/x/net/context"
 
@@ -19,9 +19,6 @@ import (
 // ContainerStats writes information about the container to the stream
 // given in the config object.
 func (daemon *Daemon) ContainerStats(ctx context.Context, prefixOrName string, config *backend.ContainerStatsConfig) error {
-	if runtime.GOOS == "windows" {
-		return errors.New("Windows does not support stats")
-	}
 	// Remote API version (used for backwards compatibility)
 	apiVersion := config.Version
 
@@ -44,10 +41,13 @@ func (daemon *Daemon) ContainerStats(ctx context.Context, prefixOrName string, c
 	}
 
 	var preCPUStats types.CPUStats
+	var preRead time.Time
 	getStatJSON := func(v interface{}) *types.StatsJSON {
 		ss := v.(types.StatsJSON)
 		ss.PreCPUStats = preCPUStats
+		ss.PreRead = preRead
 		preCPUStats = ss.CPUStats
+		preRead = ss.Read
 		return &ss
 	}
 
@@ -67,6 +67,9 @@ func (daemon *Daemon) ContainerStats(ctx context.Context, prefixOrName string, c
 			var statsJSON interface{}
 			statsJSONPost120 := getStatJSON(v)
 			if versions.LessThan(apiVersion, "1.21") {
+				if runtime.GOOS == "windows" {
+					return errors.New("API versions pre v1.21 do not support stats on Windows")
+				}
 				var (
 					rxBytes   uint64
 					rxPackets uint64
@@ -138,7 +141,8 @@ func (daemon *Daemon) GetContainerStats(container *container.Container) (*types.
 		return nil, err
 	}
 
-	if !container.Config.NetworkDisabled {
+	// We already have the network stats on Windows directly from HCS.
+	if !container.Config.NetworkDisabled && runtime.GOOS != "windows" {
 		if stats.Networks, err = daemon.getNetworkStats(container); err != nil {
 			return nil, err
 		}
@@ -146,51 +150,3 @@ func (daemon *Daemon) GetContainerStats(container *container.Container) (*types.
 
 	return stats, nil
 }
-
-// Resolve Network SandboxID in case the container reuse another container's network stack
-func (daemon *Daemon) getNetworkSandboxID(c *container.Container) (string, error) {
-	curr := c
-	for curr.HostConfig.NetworkMode.IsContainer() {
-		containerID := curr.HostConfig.NetworkMode.ConnectedContainer()
-		connected, err := daemon.GetContainer(containerID)
-		if err != nil {
-			return "", fmt.Errorf("Could not get container for %s", containerID)
-		}
-		curr = connected
-	}
-	return curr.NetworkSettings.SandboxID, nil
-}
-
-func (daemon *Daemon) getNetworkStats(c *container.Container) (map[string]types.NetworkStats, error) {
-	sandboxID, err := daemon.getNetworkSandboxID(c)
-	if err != nil {
-		return nil, err
-	}
-
-	sb, err := daemon.netController.SandboxByID(sandboxID)
-	if err != nil {
-		return nil, err
-	}
-
-	lnstats, err := sb.Statistics()
-	if err != nil {
-		return nil, err
-	}
-
-	stats := make(map[string]types.NetworkStats)
-	// Convert libnetwork nw stats into engine-api stats
-	for ifName, ifStats := range lnstats {
-		stats[ifName] = types.NetworkStats{
-			RxBytes:   ifStats.RxBytes,
-			RxPackets: ifStats.RxPackets,
-			RxErrors:  ifStats.RxErrors,
-			RxDropped: ifStats.RxDropped,
-			TxBytes:   ifStats.TxBytes,
-			TxPackets: ifStats.TxPackets,
-			TxErrors:  ifStats.TxErrors,
-			TxDropped: ifStats.TxDropped,
-		}
-	}
-
-	return stats, nil
-}

+ 132 - 0
daemon/stats_collector.go

@@ -0,0 +1,132 @@
+// +build !solaris
+
+package daemon
+
+import (
+	"bufio"
+	"sync"
+	"time"
+
+	"github.com/Sirupsen/logrus"
+	"github.com/docker/docker/api/types"
+	"github.com/docker/docker/container"
+	"github.com/docker/docker/pkg/pubsub"
+)
+
+type statsSupervisor interface {
+	// GetContainerStats collects all the stats related to a container
+	GetContainerStats(container *container.Container) (*types.StatsJSON, error)
+}
+
+// newStatsCollector returns a new statsCollector that collections
+// stats for a registered container at the specified interval.
+// The collector allows non-running containers to be added
+// and will start processing stats when they are started.
+func (daemon *Daemon) newStatsCollector(interval time.Duration) *statsCollector {
+	s := &statsCollector{
+		interval:   interval,
+		supervisor: daemon,
+		publishers: make(map[*container.Container]*pubsub.Publisher),
+		bufReader:  bufio.NewReaderSize(nil, 128),
+	}
+	platformNewStatsCollector(s)
+	go s.run()
+	return s
+}
+
+// statsCollector manages and provides container resource stats
+type statsCollector struct {
+	m          sync.Mutex
+	supervisor statsSupervisor
+	interval   time.Duration
+	publishers map[*container.Container]*pubsub.Publisher
+	bufReader  *bufio.Reader
+
+	// The following fields are not set on Windows currently.
+	clockTicksPerSecond uint64
+	machineMemory       uint64
+}
+
+// collect registers the container with the collector and adds it to
+// the event loop for collection on the specified interval returning
+// a channel for the subscriber to receive on.
+func (s *statsCollector) collect(c *container.Container) chan interface{} {
+	s.m.Lock()
+	defer s.m.Unlock()
+	publisher, exists := s.publishers[c]
+	if !exists {
+		publisher = pubsub.NewPublisher(100*time.Millisecond, 1024)
+		s.publishers[c] = publisher
+	}
+	return publisher.Subscribe()
+}
+
+// stopCollection closes the channels for all subscribers and removes
+// the container from metrics collection.
+func (s *statsCollector) stopCollection(c *container.Container) {
+	s.m.Lock()
+	if publisher, exists := s.publishers[c]; exists {
+		publisher.Close()
+		delete(s.publishers, c)
+	}
+	s.m.Unlock()
+}
+
+// unsubscribe removes a specific subscriber from receiving updates for a container's stats.
+func (s *statsCollector) unsubscribe(c *container.Container, ch chan interface{}) {
+	s.m.Lock()
+	publisher := s.publishers[c]
+	if publisher != nil {
+		publisher.Evict(ch)
+		if publisher.Len() == 0 {
+			delete(s.publishers, c)
+		}
+	}
+	s.m.Unlock()
+}
+
+func (s *statsCollector) run() {
+	type publishersPair struct {
+		container *container.Container
+		publisher *pubsub.Publisher
+	}
+	// we cannot determine the capacity here.
+	// it will grow enough in first iteration
+	var pairs []publishersPair
+
+	for range time.Tick(s.interval) {
+		// it does not make sense in the first iteration,
+		// but saves allocations in further iterations
+		pairs = pairs[:0]
+
+		s.m.Lock()
+		for container, publisher := range s.publishers {
+			// copy pointers here to release the lock ASAP
+			pairs = append(pairs, publishersPair{container, publisher})
+		}
+		s.m.Unlock()
+		if len(pairs) == 0 {
+			continue
+		}
+
+		systemUsage, err := s.getSystemCPUUsage()
+		if err != nil {
+			logrus.Errorf("collecting system cpu usage: %v", err)
+			continue
+		}
+
+		for _, pair := range pairs {
+			stats, err := s.supervisor.GetContainerStats(pair.container)
+			if err != nil {
+				if _, ok := err.(errNotRunning); !ok {
+					logrus.Errorf("collecting stats for %s: %v", pair.container.ID, err)
+				}
+				continue
+			}
+			// FIXME: move to containerd on Linux (not Windows)
+			stats.CPUStats.SystemUsage = systemUsage
+
+			pair.publisher.Publish(*stats)
+		}
+	}
+}

+ 4 - 122
daemon/stats_collector_unix.go

@@ -3,141 +3,23 @@
 package daemon
 
 import (
-	"bufio"
 	"fmt"
 	"os"
 	"strconv"
 	"strings"
-	"sync"
-	"time"
 
-	"github.com/Sirupsen/logrus"
-	"github.com/docker/docker/api/types"
-	"github.com/docker/docker/container"
-	"github.com/docker/docker/pkg/pubsub"
 	sysinfo "github.com/docker/docker/pkg/system"
 	"github.com/opencontainers/runc/libcontainer/system"
 )
 
-type statsSupervisor interface {
-	// GetContainerStats collects all the stats related to a container
-	GetContainerStats(container *container.Container) (*types.StatsJSON, error)
-}
-
-// newStatsCollector returns a new statsCollector that collections
-// network and cgroup stats for a registered container at the specified
-// interval.  The collector allows non-running containers to be added
-// and will start processing stats when they are started.
-func (daemon *Daemon) newStatsCollector(interval time.Duration) *statsCollector {
-	s := &statsCollector{
-		interval:            interval,
-		supervisor:          daemon,
-		publishers:          make(map[*container.Container]*pubsub.Publisher),
-		clockTicksPerSecond: uint64(system.GetClockTicks()),
-		bufReader:           bufio.NewReaderSize(nil, 128),
-	}
+// platformNewStatsCollector performs platform specific initialisation of the
+// statsCollector structure.
+func platformNewStatsCollector(s *statsCollector) {
+	s.clockTicksPerSecond = uint64(system.GetClockTicks())
 	meminfo, err := sysinfo.ReadMemInfo()
 	if err == nil && meminfo.MemTotal > 0 {
 		s.machineMemory = uint64(meminfo.MemTotal)
 	}
-
-	go s.run()
-	return s
-}
-
-// statsCollector manages and provides container resource stats
-type statsCollector struct {
-	m                   sync.Mutex
-	supervisor          statsSupervisor
-	interval            time.Duration
-	clockTicksPerSecond uint64
-	publishers          map[*container.Container]*pubsub.Publisher
-	bufReader           *bufio.Reader
-	machineMemory       uint64
-}
-
-// collect registers the container with the collector and adds it to
-// the event loop for collection on the specified interval returning
-// a channel for the subscriber to receive on.
-func (s *statsCollector) collect(c *container.Container) chan interface{} {
-	s.m.Lock()
-	defer s.m.Unlock()
-	publisher, exists := s.publishers[c]
-	if !exists {
-		publisher = pubsub.NewPublisher(100*time.Millisecond, 1024)
-		s.publishers[c] = publisher
-	}
-	return publisher.Subscribe()
-}
-
-// stopCollection closes the channels for all subscribers and removes
-// the container from metrics collection.
-func (s *statsCollector) stopCollection(c *container.Container) {
-	s.m.Lock()
-	if publisher, exists := s.publishers[c]; exists {
-		publisher.Close()
-		delete(s.publishers, c)
-	}
-	s.m.Unlock()
-}
-
-// unsubscribe removes a specific subscriber from receiving updates for a container's stats.
-func (s *statsCollector) unsubscribe(c *container.Container, ch chan interface{}) {
-	s.m.Lock()
-	publisher := s.publishers[c]
-	if publisher != nil {
-		publisher.Evict(ch)
-		if publisher.Len() == 0 {
-			delete(s.publishers, c)
-		}
-	}
-	s.m.Unlock()
-}
-
-func (s *statsCollector) run() {
-	type publishersPair struct {
-		container *container.Container
-		publisher *pubsub.Publisher
-	}
-	// we cannot determine the capacity here.
-	// it will grow enough in first iteration
-	var pairs []publishersPair
-
-	for range time.Tick(s.interval) {
-		// it does not make sense in the first iteration,
-		// but saves allocations in further iterations
-		pairs = pairs[:0]
-
-		s.m.Lock()
-		for container, publisher := range s.publishers {
-			// copy pointers here to release the lock ASAP
-			pairs = append(pairs, publishersPair{container, publisher})
-		}
-		s.m.Unlock()
-		if len(pairs) == 0 {
-			continue
-		}
-
-		systemUsage, err := s.getSystemCPUUsage()
-		if err != nil {
-			logrus.Errorf("collecting system cpu usage: %v", err)
-			continue
-		}
-
-		for _, pair := range pairs {
-			stats, err := s.supervisor.GetContainerStats(pair.container)
-			if err != nil {
-				if _, ok := err.(errNotRunning); !ok {
-					logrus.Errorf("collecting stats for %s: %v", pair.container.ID, err)
-				}
-				continue
-			}
-			// FIXME: move to containerd
-			stats.CPUStats.SystemUsage = systemUsage
-
-			pair.publisher.Publish(*stats)
-		}
-	}
 }
 
 const nanoSecondsPerSecond = 1e9

+ 10 - 30
daemon/stats_collector_windows.go

@@ -1,35 +1,15 @@
-package daemon
-
-import (
-	"time"
-
-	"github.com/docker/docker/container"
-)
-
-// newStatsCollector returns a new statsCollector for collection stats
-// for a registered container at the specified interval. The collector allows
-// non-running containers to be added and will start processing stats when
-// they are started.
-func (daemon *Daemon) newStatsCollector(interval time.Duration) *statsCollector {
-	return &statsCollector{}
-}
+// +build windows
 
-// statsCollector manages and provides container resource stats
-type statsCollector struct {
-}
-
-// collect registers the container with the collector and adds it to
-// the event loop for collection on the specified interval returning
-// a channel for the subscriber to receive on.
-func (s *statsCollector) collect(c *container.Container) chan interface{} {
-	return nil
-}
+package daemon
 
-// stopCollection closes the channels for all subscribers and removes
-// the container from metrics collection.
-func (s *statsCollector) stopCollection(c *container.Container) {
+// platformNewStatsCollector performs platform specific initialisation of the
+// statsCollector structure. This is a no-op on Windows.
+func platformNewStatsCollector(s *statsCollector) {
 }
 
-// unsubscribe removes a specific subscriber from receiving updates for a container's stats.
-func (s *statsCollector) unsubscribe(c *container.Container, ch chan interface{}) {
+// getSystemCPUUsage returns the host system's cpu usage in
+// nanoseconds. An error is returned if the format of the underlying
+// file does not match. This is a no-op on Windows.
+func (s *statsCollector) getSystemCPUUsage() (uint64, error) {
+	return 0, nil
 }

+ 58 - 0
daemon/stats_unix.go

@@ -0,0 +1,58 @@
+// +build !windows
+
+package daemon
+
+import (
+	"fmt"
+
+	"github.com/docker/docker/api/types"
+	"github.com/docker/docker/container"
+)
+
+// Resolve Network SandboxID in case the container reuse another container's network stack
+func (daemon *Daemon) getNetworkSandboxID(c *container.Container) (string, error) {
+	curr := c
+	for curr.HostConfig.NetworkMode.IsContainer() {
+		containerID := curr.HostConfig.NetworkMode.ConnectedContainer()
+		connected, err := daemon.GetContainer(containerID)
+		if err != nil {
+			return "", fmt.Errorf("Could not get container for %s", containerID)
+		}
+		curr = connected
+	}
+	return curr.NetworkSettings.SandboxID, nil
+}
+
+func (daemon *Daemon) getNetworkStats(c *container.Container) (map[string]types.NetworkStats, error) {
+	sandboxID, err := daemon.getNetworkSandboxID(c)
+	if err != nil {
+		return nil, err
+	}
+
+	sb, err := daemon.netController.SandboxByID(sandboxID)
+	if err != nil {
+		return nil, err
+	}
+
+	lnstats, err := sb.Statistics()
+	if err != nil {
+		return nil, err
+	}
+
+	stats := make(map[string]types.NetworkStats)
+	// Convert libnetwork nw stats into engine-api stats
+	for ifName, ifStats := range lnstats {
+		stats[ifName] = types.NetworkStats{
+			RxBytes:   ifStats.RxBytes,
+			RxPackets: ifStats.RxPackets,
+			RxErrors:  ifStats.RxErrors,
+			RxDropped: ifStats.RxDropped,
+			TxBytes:   ifStats.TxBytes,
+			TxPackets: ifStats.TxPackets,
+			TxErrors:  ifStats.TxErrors,
+			TxDropped: ifStats.TxDropped,
+		}
+	}
+
+	return stats, nil
+}

+ 11 - 0
daemon/stats_windows.go

@@ -0,0 +1,11 @@
+package daemon
+
+import (
+	"github.com/docker/docker/api/types"
+	"github.com/docker/docker/container"
+)
+
+// Windows network stats are obtained directly through HCS, hence this is a no-op.
+func (daemon *Daemon) getNetworkStats(c *container.Container) (map[string]types.NetworkStats, error) {
+	return make(map[string]types.NetworkStats), nil
+}

+ 23 - 2
docs/reference/commandline/stats.md

@@ -27,7 +27,7 @@ If you want more detailed information about a container's resource usage, use th
 
 ## Examples
 
-Running `docker stats` on all running containers
+Running `docker stats` on all running containers against a Linux daemon.
 
     $ docker stats
     CONTAINER           CPU %               MEM USAGE / LIMIT     MEM %               NET I/O             BLOCK I/O
@@ -35,9 +35,30 @@ Running `docker stats` on all running containers
     9c76f7834ae2        0.07%               2.746 MiB / 64 MiB      4.29%               1.266 KB / 648 B    12.4 MB / 0 B
     d1ea048f04e4        0.03%               4.583 MiB / 64 MiB      6.30%               2.854 KB / 648 B    27.7 MB / 0 B
 
-Running `docker stats` on multiple containers by name and id.
+Running `docker stats` on multiple containers by name and id against a Linux daemon.
 
     $ docker stats fervent_panini 5acfcb1b4fd1
     CONTAINER           CPU %               MEM USAGE/LIMIT     MEM %               NET I/O
     5acfcb1b4fd1        0.00%               115.2 MiB/1.045 GiB   11.03%              1.422 kB/648 B
     fervent_panini      0.02%               11.08 MiB/1.045 GiB   1.06%               648 B/648 B
+
+Running `docker stats` on all running containers against a Windows daemon.
+
+    PS E:\> docker stats
+    CONTAINER           CPU %               PRIV WORKING SET    NET I/O             BLOCK I/O
+    09d3bb5b1604        6.61%               38.21 MiB           17.1 kB / 7.73 kB   10.7 MB / 3.57 MB
+    9db7aa4d986d        9.19%               38.26 MiB           15.2 kB / 7.65 kB   10.6 MB / 3.3 MB
+    3f214c61ad1d        0.00%               28.64 MiB           64 kB / 6.84 kB     4.42 MB / 6.93 MB
+
+Running `docker stats` on multiple containers by name and id against a Windows daemon.
+
+    PS E:\> docker ps -a
+    CONTAINER ID        IMAGE               COMMAND             CREATED             STATUS              PORTS               NAMES
+    3f214c61ad1d        nanoserver          "cmd"               2 minutes ago       Up 2 minutes                            big_minsky
+    9db7aa4d986d        windowsservercore   "cmd"               2 minutes ago       Up 2 minutes                            mad_wilson
+    09d3bb5b1604        windowsservercore   "cmd"               2 minutes ago       Up 2 minutes                            affectionate_easley
+   
+    PS E:\> docker stats 3f214c61ad1d mad_wilson
+    CONTAINER           CPU %               PRIV WORKING SET    NET I/O             BLOCK I/O
+    3f214c61ad1d        0.00%               46.25 MiB           76.3 kB / 7.92 kB   10.3 MB / 14.7 MB
+    mad_wilson          9.59%               40.09 MiB           27.6 kB / 8.81 kB   17 MB / 20.1 MB	

+ 7 - 17
integration-cli/docker_api_containers_test.go

@@ -175,12 +175,10 @@ func (s *DockerSuite) TestContainerApiGetChanges(c *check.C) {
 }
 
 func (s *DockerSuite) TestGetContainerStats(c *check.C) {
-	// Problematic on Windows as Windows does not support stats
-	testRequires(c, DaemonIsLinux)
 	var (
 		name = "statscontainer"
 	)
-	dockerCmd(c, "run", "-d", "--name", name, "busybox", "top")
+	runSleepingContainer(c, "--name", name)
 
 	type b struct {
 		status int
@@ -214,9 +212,7 @@ func (s *DockerSuite) TestGetContainerStats(c *check.C) {
 }
 
 func (s *DockerSuite) TestGetContainerStatsRmRunning(c *check.C) {
-	// Problematic on Windows as Windows does not support stats
-	testRequires(c, DaemonIsLinux)
-	out, _ := dockerCmd(c, "run", "-d", "busybox", "top")
+	out, _ := runSleepingContainer(c)
 	id := strings.TrimSpace(out)
 
 	buf := &integration.ChannelBuffer{make(chan []byte, 1)}
@@ -251,10 +247,8 @@ func (s *DockerSuite) TestGetContainerStatsRmRunning(c *check.C) {
 // previous test was just checking one stat entry so it didn't fail (stats with
 // stream false always return one stat)
 func (s *DockerSuite) TestGetContainerStatsStream(c *check.C) {
-	// Problematic on Windows as Windows does not support stats
-	testRequires(c, DaemonIsLinux)
 	name := "statscontainer"
-	dockerCmd(c, "run", "-d", "--name", name, "busybox", "top")
+	runSleepingContainer(c, "--name", name)
 
 	type b struct {
 		status int
@@ -289,10 +283,8 @@ func (s *DockerSuite) TestGetContainerStatsStream(c *check.C) {
 }
 
 func (s *DockerSuite) TestGetContainerStatsNoStream(c *check.C) {
-	// Problematic on Windows as Windows does not support stats
-	testRequires(c, DaemonIsLinux)
 	name := "statscontainer"
-	dockerCmd(c, "run", "-d", "--name", name, "busybox", "top")
+	runSleepingContainer(c, "--name", name)
 
 	type b struct {
 		status int
@@ -319,16 +311,14 @@ func (s *DockerSuite) TestGetContainerStatsNoStream(c *check.C) {
 		c.Assert(sr.status, checker.Equals, http.StatusOK)
 
 		s := string(sr.body)
-		// count occurrences of "read" of types.Stats
-		c.Assert(strings.Count(s, "read"), checker.Equals, 1, check.Commentf("Expected only one stat streamed, got %d", strings.Count(s, "read")))
+		// count occurrences of `"read"` of types.Stats
+		c.Assert(strings.Count(s, `"read"`), checker.Equals, 1, check.Commentf("Expected only one stat streamed, got %d", strings.Count(s, `"read"`)))
 	}
 }
 
 func (s *DockerSuite) TestGetStoppedContainerStats(c *check.C) {
-	// Problematic on Windows as Windows does not support stats
-	testRequires(c, DaemonIsLinux)
 	name := "statscontainer"
-	dockerCmd(c, "create", "--name", name, "busybox", "top")
+	dockerCmd(c, "create", "--name", name, "busybox", "ps")
 
 	type stats struct {
 		status int

+ 32 - 11
integration-cli/docker_api_stats_test.go

@@ -20,7 +20,6 @@ import (
 var expectedNetworkInterfaceStats = strings.Split("rx_bytes rx_dropped rx_errors rx_packets tx_bytes tx_dropped tx_errors tx_packets", " ")
 
 func (s *DockerSuite) TestApiStatsNoStreamGetCpu(c *check.C) {
-	testRequires(c, DaemonIsLinux)
 	out, _ := dockerCmd(c, "run", "-d", "busybox", "/bin/sh", "-c", "while true;do echo 'Hello'; usleep 100000; done")
 
 	id := strings.TrimSpace(out)
@@ -37,15 +36,30 @@ func (s *DockerSuite) TestApiStatsNoStreamGetCpu(c *check.C) {
 	body.Close()
 
 	var cpuPercent = 0.0
-	cpuDelta := float64(v.CPUStats.CPUUsage.TotalUsage - v.PreCPUStats.CPUUsage.TotalUsage)
-	systemDelta := float64(v.CPUStats.SystemUsage - v.PreCPUStats.SystemUsage)
-	cpuPercent = (cpuDelta / systemDelta) * float64(len(v.CPUStats.CPUUsage.PercpuUsage)) * 100.0
+
+	if daemonPlatform != "windows" {
+		cpuDelta := float64(v.CPUStats.CPUUsage.TotalUsage - v.PreCPUStats.CPUUsage.TotalUsage)
+		systemDelta := float64(v.CPUStats.SystemUsage - v.PreCPUStats.SystemUsage)
+		cpuPercent = (cpuDelta / systemDelta) * float64(len(v.CPUStats.CPUUsage.PercpuUsage)) * 100.0
+	} else {
+		// Max number of 100ns intervals between the previous time read and now
+		possIntervals := uint64(v.Read.Sub(v.PreRead).Nanoseconds()) // Start with number of ns intervals
+		possIntervals /= 100                                         // Convert to number of 100ns intervals
+		possIntervals *= uint64(v.NumProcs)                          // Multiple by the number of processors
+
+		// Intervals used
+		intervalsUsed := v.CPUStats.CPUUsage.TotalUsage - v.PreCPUStats.CPUUsage.TotalUsage
+
+		// Percentage avoiding divide-by-zero
+		if possIntervals > 0 {
+			cpuPercent = float64(intervalsUsed) / float64(possIntervals) * 100.0
+		}
+	}
 
 	c.Assert(cpuPercent, check.Not(checker.Equals), 0.0, check.Commentf("docker stats with no-stream get cpu usage failed: was %v", cpuPercent))
 }
 
 func (s *DockerSuite) TestApiStatsStoppedContainerInGoroutines(c *check.C) {
-	testRequires(c, DaemonIsLinux)
 	out, _ := dockerCmd(c, "run", "-d", "busybox", "/bin/sh", "-c", "echo 1")
 	id := strings.TrimSpace(out)
 
@@ -82,14 +96,17 @@ func (s *DockerSuite) TestApiStatsStoppedContainerInGoroutines(c *check.C) {
 
 func (s *DockerSuite) TestApiStatsNetworkStats(c *check.C) {
 	testRequires(c, SameHostDaemon)
-	testRequires(c, DaemonIsLinux)
 
 	out, _ := runSleepingContainer(c)
 	id := strings.TrimSpace(out)
 	c.Assert(waitRun(id), checker.IsNil)
 
 	// Retrieve the container address
-	contIP := findContainerIP(c, id, "bridge")
+	net := "bridge"
+	if daemonPlatform == "windows" {
+		net = "nat"
+	}
+	contIP := findContainerIP(c, id, net)
 	numPings := 1
 
 	var preRxPackets uint64
@@ -130,9 +147,14 @@ func (s *DockerSuite) TestApiStatsNetworkStats(c *check.C) {
 		postTxPackets += v.TxPackets
 	}
 
-	// Verify the stats contain at least the expected number of packets (account for ARP)
-	expRxPkts := 1 + preRxPackets + uint64(numPings)
-	expTxPkts := 1 + preTxPackets + uint64(numPings)
+	// Verify the stats contain at least the expected number of packets
+	// On Linux, account for ARP.
+	expRxPkts := preRxPackets + uint64(numPings)
+	expTxPkts := preTxPackets + uint64(numPings)
+	if daemonPlatform != "windows" {
+		expRxPkts++
+		expTxPkts++
+	}
 	c.Assert(postTxPackets, checker.GreaterOrEqualThan, expTxPkts,
 		check.Commentf("Reported less TxPackets than expected. Expected >= %d. Found %d. %s", expTxPkts, postTxPackets, pingouts))
 	c.Assert(postRxPackets, checker.GreaterOrEqualThan, expRxPkts,
@@ -141,7 +163,6 @@ func (s *DockerSuite) TestApiStatsNetworkStats(c *check.C) {
 
 func (s *DockerSuite) TestApiStatsNetworkStatsVersioning(c *check.C) {
 	testRequires(c, SameHostDaemon)
-	testRequires(c, DaemonIsLinux)
 
 	out, _ := runSleepingContainer(c)
 	id := strings.TrimSpace(out)

+ 13 - 1
libcontainerd/client_windows.go

@@ -369,7 +369,19 @@ func (clnt *client) Resume(containerID string) error {
 
 // Stats handles stats requests for containers
 func (clnt *client) Stats(containerID string) (*Stats, error) {
-	return nil, errors.New("Windows: Stats not implemented")
+	// Get the libcontainerd container object
+	clnt.lock(containerID)
+	defer clnt.unlock(containerID)
+	container, err := clnt.getContainer(containerID)
+	if err != nil {
+		return nil, err
+	}
+	s, err := container.hcsContainer.Statistics()
+	if err != nil {
+		return nil, err
+	}
+	st := Stats(s)
+	return &st, nil
 }
 
 // Restore is the handler for restoring a container

+ 2 - 2
libcontainerd/types_windows.go

@@ -26,8 +26,8 @@ type StateInfo struct {
 	UpdatePending bool // Indicates that there are some update operations pending that should be completed by a servicing container.
 }
 
-// Stats contains a stats properties from containerd.
-type Stats struct{}
+// Stats contains statics from HCS
+type Stats hcsshim.Statistics
 
 // Resources defines updatable container resource values.
 type Resources struct{}

+ 6 - 0
pkg/platform/architecture_windows.go

@@ -50,3 +50,9 @@ func runtimeArchitecture() (string, error) {
 		return "", fmt.Errorf("Unknown processor architecture")
 	}
 }
+
+// NumProcs returns the number of processors on the system
+func NumProcs() uint32 {
+	syscall.Syscall(procGetSystemInfo.Addr(), 1, uintptr(unsafe.Pointer(&sysinfo)), 0, 0)
+	return sysinfo.dwNumberOfProcessors
+}