mirror of
https://github.com/moby/moby.git
synced 2022-11-09 12:21:53 -05:00
daemon/logger: fix refcounting decompressed files
The refCounter used for sharing temporary decompressed log files and tracking when the files can be deleted is keyed off the source file's path. But the path of a log file is not stable: it is renamed on each rotation. Consequently, when logging is configured with both rotation and compression, multiple concurrent readers of a container's logs could read logs out of order, see duplicates or decompress a log file which has already been decompressed. Replace refCounter with a new implementation, sharedTempFileConverter, which is agnostic to the file path, keying off the source file's identity instead. Additionally, sharedTempFileConverter handles the full lifecycle of the temporary file, from creation to deletion. This is all abstracted from the consumer: all the bookkeeping and cleanup is handled behind the scenes when Close() is called on the returned reader value. Only one file descriptor is used per temporary file, which is shared by all readers. A channel is used for concurrency control so that the lock can be acquired inside a select statement. While not currently utilized, this makes it possible to add support for cancellation to sharedTempFileConverter in the future. Signed-off-by: Cory Snider <csnider@mirantis.com>
This commit is contained in:
parent
49aa66b597
commit
6d5bc07189
6 changed files with 642 additions and 154 deletions
|
@ -1,11 +1,16 @@
|
|||
package jsonfilelog // import "github.com/docker/docker/daemon/logger/jsonfilelog"
|
||||
|
||||
import (
|
||||
"bufio"
|
||||
"bytes"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"io"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"strconv"
|
||||
"testing"
|
||||
"text/tabwriter"
|
||||
"time"
|
||||
|
||||
"github.com/docker/docker/daemon/logger"
|
||||
|
@ -142,7 +147,8 @@ func TestUnexpectedEOF(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestReadLogs(t *testing.T) {
|
||||
loggertest.Reader{
|
||||
t.Parallel()
|
||||
r := loggertest.Reader{
|
||||
Factory: func(t *testing.T, info logger.Info) func(*testing.T) logger.Logger {
|
||||
dir := t.TempDir()
|
||||
info.LogPath = filepath.Join(dir, info.ContainerID+".log")
|
||||
|
@ -152,7 +158,67 @@ func TestReadLogs(t *testing.T) {
|
|||
return l
|
||||
}
|
||||
},
|
||||
}.Do(t)
|
||||
}
|
||||
t.Run("Tail", r.TestTail)
|
||||
t.Run("Follow", r.TestFollow)
|
||||
}
|
||||
|
||||
func TestTailLogsWithRotation(t *testing.T) {
|
||||
t.Parallel()
|
||||
compress := func(cmprs bool) {
|
||||
t.Run(fmt.Sprintf("compress=%v", cmprs), func(t *testing.T) {
|
||||
t.Parallel()
|
||||
(&loggertest.Reader{
|
||||
Factory: func(t *testing.T, info logger.Info) func(*testing.T) logger.Logger {
|
||||
info.Config = map[string]string{
|
||||
"compress": strconv.FormatBool(cmprs),
|
||||
"max-size": "1b",
|
||||
"max-file": "10",
|
||||
}
|
||||
dir := t.TempDir()
|
||||
t.Cleanup(func() {
|
||||
t.Logf("%s:\n%s", t.Name(), dirStringer{dir})
|
||||
})
|
||||
info.LogPath = filepath.Join(dir, info.ContainerID+".log")
|
||||
return func(t *testing.T) logger.Logger {
|
||||
l, err := New(info)
|
||||
assert.NilError(t, err)
|
||||
return l
|
||||
}
|
||||
},
|
||||
}).TestTail(t)
|
||||
})
|
||||
}
|
||||
compress(true)
|
||||
compress(false)
|
||||
}
|
||||
|
||||
type dirStringer struct {
|
||||
d string
|
||||
}
|
||||
|
||||
func (d dirStringer) String() string {
|
||||
ls, err := os.ReadDir(d.d)
|
||||
if err != nil {
|
||||
return ""
|
||||
}
|
||||
buf := bytes.NewBuffer(nil)
|
||||
tw := tabwriter.NewWriter(buf, 1, 8, 1, '\t', 0)
|
||||
buf.WriteString("\n")
|
||||
|
||||
btw := bufio.NewWriter(tw)
|
||||
|
||||
for _, entry := range ls {
|
||||
fi, err := entry.Info()
|
||||
if err != nil {
|
||||
return ""
|
||||
}
|
||||
|
||||
btw.WriteString(fmt.Sprintf("%s\t%s\t%dB\t%s\n", fi.Name(), fi.Mode(), fi.Size(), fi.ModTime()))
|
||||
}
|
||||
btw.Flush()
|
||||
tw.Flush()
|
||||
return buf.String()
|
||||
}
|
||||
|
||||
type readerWithErr struct {
|
||||
|
|
|
@ -79,7 +79,7 @@ func TestWriteLog(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestReadLog(t *testing.T) {
|
||||
loggertest.Reader{
|
||||
r := loggertest.Reader{
|
||||
Factory: func(t *testing.T, info logger.Info) func(*testing.T) logger.Logger {
|
||||
dir := t.TempDir()
|
||||
info.LogPath = filepath.Join(dir, info.ContainerID+".log")
|
||||
|
@ -89,7 +89,9 @@ func TestReadLog(t *testing.T) {
|
|||
return l
|
||||
}
|
||||
},
|
||||
}.Do(t)
|
||||
}
|
||||
t.Run("Tail", r.TestTail)
|
||||
t.Run("Follow", r.TestFollow)
|
||||
}
|
||||
|
||||
func BenchmarkLogWrite(b *testing.B) {
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
package loggertest // import "github.com/docker/docker/daemon/logger/loggertest"
|
||||
|
||||
import (
|
||||
"runtime"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
@ -29,13 +30,12 @@ var compareLog cmp.Options = []cmp.Option{
|
|||
cmp.Transformer("string", func(b []byte) string { return string(b) }),
|
||||
}
|
||||
|
||||
// Do tests the behavior of the LogReader implementation.
|
||||
func (tr Reader) Do(t *testing.T) {
|
||||
t.Run("Live/Tail", func(t *testing.T) { tr.testTail(t, true) })
|
||||
t.Run("Live/TailEmpty", func(t *testing.T) { tr.testTailEmptyLogs(t, true) })
|
||||
t.Run("Live/Follow", tr.testFollow)
|
||||
t.Run("Stopped/Tail", func(t *testing.T) { tr.testTail(t, false) })
|
||||
t.Run("Stopped/TailEmpty", func(t *testing.T) { tr.testTailEmptyLogs(t, false) })
|
||||
// TestTail tests the behavior of the LogReader's tail implementation.
|
||||
func (tr Reader) TestTail(t *testing.T) {
|
||||
t.Run("Live", func(t *testing.T) { tr.testTail(t, true) })
|
||||
t.Run("LiveEmpty", func(t *testing.T) { tr.testTailEmptyLogs(t, true) })
|
||||
t.Run("Stopped", func(t *testing.T) { tr.testTail(t, false) })
|
||||
t.Run("StoppedEmpty", func(t *testing.T) { tr.testTailEmptyLogs(t, false) })
|
||||
}
|
||||
|
||||
func makeTestMessages() []*logger.Message {
|
||||
|
@ -170,8 +170,11 @@ func (tr Reader) testTailEmptyLogs(t *testing.T, live bool) {
|
|||
}
|
||||
}
|
||||
|
||||
func (tr Reader) testFollow(t *testing.T) {
|
||||
t.Parallel()
|
||||
// TestFollow tests the LogReader's follow implementation.
|
||||
//
|
||||
// The LogReader is expected to be able to follow an arbitrary number of
|
||||
// messages at a high rate with no dropped messages.
|
||||
func (tr Reader) TestFollow(t *testing.T) {
|
||||
// Reader sends all logs and closes after logger is closed
|
||||
// - Starting from empty log (like run)
|
||||
t.Run("FromEmptyLog", func(t *testing.T) {
|
||||
|
@ -390,6 +393,7 @@ func logMessages(t *testing.T, l logger.Logger, messages []*logger.Message) []*l
|
|||
// Copy the log message because the underlying log writer resets
|
||||
// the log message and returns it to a buffer pool.
|
||||
assert.NilError(t, l.Log(copyLogMessage(m)))
|
||||
runtime.Gosched()
|
||||
|
||||
// Copy the log message again so as not to mutate the input.
|
||||
expect := copyLogMessage(m)
|
||||
|
@ -437,6 +441,9 @@ func readMessage(t *testing.T, lw *logger.LogWatcher) *logger.Message {
|
|||
default:
|
||||
}
|
||||
}
|
||||
if msg != nil {
|
||||
t.Logf("loggertest: ReadMessage [%v %v] %s", msg.Source, msg.Timestamp, msg.Line)
|
||||
}
|
||||
return msg
|
||||
}
|
||||
}
|
||||
|
|
|
@ -10,7 +10,6 @@ import (
|
|||
"os"
|
||||
"runtime"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
|
@ -22,77 +21,29 @@ import (
|
|||
"github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
const tmpLogfileSuffix = ".tmp"
|
||||
|
||||
// rotateFileMetadata is a metadata of the gzip header of the compressed log file
|
||||
type rotateFileMetadata struct {
|
||||
LastTime time.Time `json:"lastTime,omitempty"`
|
||||
}
|
||||
|
||||
// refCounter is a counter of logfile being referenced
|
||||
type refCounter struct {
|
||||
mu sync.Mutex
|
||||
counter map[string]int
|
||||
}
|
||||
|
||||
// Reference increase the reference counter for specified logfile
|
||||
func (rc *refCounter) GetReference(fileName string, openRefFile func(fileName string, exists bool) (*os.File, error)) (*os.File, error) {
|
||||
rc.mu.Lock()
|
||||
defer rc.mu.Unlock()
|
||||
|
||||
var (
|
||||
file *os.File
|
||||
err error
|
||||
)
|
||||
_, ok := rc.counter[fileName]
|
||||
file, err = openRefFile(fileName, ok)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if ok {
|
||||
rc.counter[fileName]++
|
||||
} else if file != nil {
|
||||
rc.counter[file.Name()] = 1
|
||||
}
|
||||
|
||||
return file, nil
|
||||
}
|
||||
|
||||
// Dereference reduce the reference counter for specified logfile
|
||||
func (rc *refCounter) Dereference(fileName string) error {
|
||||
rc.mu.Lock()
|
||||
defer rc.mu.Unlock()
|
||||
|
||||
rc.counter[fileName]--
|
||||
if rc.counter[fileName] <= 0 {
|
||||
delete(rc.counter, fileName)
|
||||
err := unlink(fileName)
|
||||
if err != nil && !errors.Is(err, fs.ErrNotExist) {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// LogFile is Logger implementation for default Docker logging.
|
||||
type LogFile struct {
|
||||
mu sync.RWMutex // protects the logfile access
|
||||
f *os.File // store for closing
|
||||
closed bool
|
||||
closedCh chan struct{}
|
||||
rotateMu sync.Mutex // blocks the next rotation until the current rotation is completed
|
||||
capacity int64 // maximum size of each file
|
||||
currentSize int64 // current size of the latest file
|
||||
maxFiles int // maximum number of files
|
||||
compress bool // whether old versions of log files are compressed
|
||||
lastTimestamp time.Time // timestamp of the last log
|
||||
filesRefCounter refCounter // keep reference-counted of decompressed files
|
||||
notifyReaders *pubsub.Publisher
|
||||
marshal logger.MarshalFunc
|
||||
createDecoder MakeDecoderFn
|
||||
getTailReader GetTailReaderFunc
|
||||
perms os.FileMode
|
||||
mu sync.RWMutex // protects the logfile access
|
||||
f *os.File // store for closing
|
||||
closed bool
|
||||
closedCh chan struct{}
|
||||
rotateMu sync.Mutex // blocks the next rotation until the current rotation is completed
|
||||
capacity int64 // maximum size of each file
|
||||
currentSize int64 // current size of the latest file
|
||||
maxFiles int // maximum number of files
|
||||
compress bool // whether old versions of log files are compressed
|
||||
lastTimestamp time.Time // timestamp of the last log
|
||||
decompress *sharedTempFileConverter // keep reference-counted decompressed files
|
||||
notifyReaders *pubsub.Publisher
|
||||
marshal logger.MarshalFunc
|
||||
createDecoder MakeDecoderFn
|
||||
getTailReader GetTailReaderFunc
|
||||
perms os.FileMode
|
||||
}
|
||||
|
||||
// MakeDecoderFn creates a decoder
|
||||
|
@ -113,10 +64,16 @@ type Decoder interface {
|
|||
// SizeReaderAt defines a ReaderAt that also reports its size.
|
||||
// This is used for tailing log files.
|
||||
type SizeReaderAt interface {
|
||||
io.Reader
|
||||
io.ReaderAt
|
||||
Size() int64
|
||||
}
|
||||
|
||||
type readAtCloser interface {
|
||||
io.ReaderAt
|
||||
io.Closer
|
||||
}
|
||||
|
||||
// GetTailReaderFunc is used to truncate a reader to only read as much as is required
|
||||
// in order to get the passed in number of log lines.
|
||||
// It returns the sectioned reader, the number of lines that the section reader
|
||||
|
@ -136,18 +93,18 @@ func NewLogFile(logPath string, capacity int64, maxFiles int, compress bool, mar
|
|||
}
|
||||
|
||||
return &LogFile{
|
||||
f: log,
|
||||
closedCh: make(chan struct{}),
|
||||
capacity: capacity,
|
||||
currentSize: size,
|
||||
maxFiles: maxFiles,
|
||||
compress: compress,
|
||||
filesRefCounter: refCounter{counter: make(map[string]int)},
|
||||
notifyReaders: pubsub.NewPublisher(0, 1),
|
||||
marshal: marshaller,
|
||||
createDecoder: decodeFunc,
|
||||
perms: perms,
|
||||
getTailReader: getTailReader,
|
||||
f: log,
|
||||
closedCh: make(chan struct{}),
|
||||
capacity: capacity,
|
||||
currentSize: size,
|
||||
maxFiles: maxFiles,
|
||||
compress: compress,
|
||||
decompress: newSharedTempFileConverter(decompress),
|
||||
notifyReaders: pubsub.NewPublisher(0, 1),
|
||||
marshal: marshaller,
|
||||
createDecoder: decodeFunc,
|
||||
perms: perms,
|
||||
getTailReader: getTailReader,
|
||||
}, nil
|
||||
}
|
||||
|
||||
|
@ -411,25 +368,25 @@ func (w *LogFile) readLogsLocked(config logger.ReadConfig, watcher *logger.LogWa
|
|||
closeFiles := func() {
|
||||
for _, f := range files {
|
||||
f.Close()
|
||||
fileName := f.Name()
|
||||
if strings.HasSuffix(fileName, tmpLogfileSuffix) {
|
||||
err := w.filesRefCounter.Dereference(fileName)
|
||||
if err != nil {
|
||||
logrus.WithError(err).WithField("file", fileName).Error("Failed to dereference the log file")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
readers := make([]SizeReaderAt, 0, len(files)+1)
|
||||
for _, f := range files {
|
||||
stat, err := f.Stat()
|
||||
if err != nil {
|
||||
watcher.Err <- errors.Wrap(err, "error reading size of rotated file")
|
||||
closeFiles()
|
||||
return
|
||||
switch ff := f.(type) {
|
||||
case SizeReaderAt:
|
||||
readers = append(readers, ff)
|
||||
case interface{ Stat() (fs.FileInfo, error) }:
|
||||
stat, err := ff.Stat()
|
||||
if err != nil {
|
||||
watcher.Err <- errors.Wrap(err, "error reading size of rotated file")
|
||||
closeFiles()
|
||||
return
|
||||
}
|
||||
readers = append(readers, io.NewSectionReader(f, 0, stat.Size()))
|
||||
default:
|
||||
panic(fmt.Errorf("rotated file value %#v (%[1]T) has neither Size() nor Stat() methods", f))
|
||||
}
|
||||
readers = append(readers, io.NewSectionReader(f, 0, stat.Size()))
|
||||
}
|
||||
if currentChunk.Size() > 0 {
|
||||
readers = append(readers, currentChunk)
|
||||
|
@ -457,7 +414,8 @@ func (w *LogFile) readLogsLocked(config logger.ReadConfig, watcher *logger.LogWa
|
|||
followLogs(currentFile, watcher, w.closedCh, notifyRotate, notifyEvict, dec, config.Since, config.Until)
|
||||
}
|
||||
|
||||
func (w *LogFile) openRotatedFiles(config logger.ReadConfig) (files []*os.File, err error) {
|
||||
// openRotatedFiles returns a slice of files open for reading, in order from oldest to newest.
|
||||
func (w *LogFile) openRotatedFiles(config logger.ReadConfig) (files []readAtCloser, err error) {
|
||||
w.rotateMu.Lock()
|
||||
defer w.rotateMu.Unlock()
|
||||
|
||||
|
@ -467,44 +425,27 @@ func (w *LogFile) openRotatedFiles(config logger.ReadConfig) (files []*os.File,
|
|||
}
|
||||
for _, f := range files {
|
||||
f.Close()
|
||||
if strings.HasSuffix(f.Name(), tmpLogfileSuffix) {
|
||||
err := unlink(f.Name())
|
||||
if err != nil && !errors.Is(err, fs.ErrNotExist) {
|
||||
logrus.Warnf("Failed to remove logfile: %v", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
for i := w.maxFiles; i > 1; i-- {
|
||||
f, err := open(fmt.Sprintf("%s.%d", w.f.Name(), i-1))
|
||||
var f readAtCloser
|
||||
f, err = open(fmt.Sprintf("%s.%d", w.f.Name(), i-1))
|
||||
if err != nil {
|
||||
if !errors.Is(err, fs.ErrNotExist) {
|
||||
return nil, errors.Wrap(err, "error opening rotated log file")
|
||||
}
|
||||
|
||||
fileName := fmt.Sprintf("%s.%d.gz", w.f.Name(), i-1)
|
||||
decompressedFileName := fileName + tmpLogfileSuffix
|
||||
tmpFile, err := w.filesRefCounter.GetReference(decompressedFileName, func(refFileName string, exists bool) (*os.File, error) {
|
||||
if exists {
|
||||
return open(refFileName)
|
||||
}
|
||||
return decompressfile(fileName, refFileName, config.Since)
|
||||
})
|
||||
|
||||
f, err = w.maybeDecompressFile(fmt.Sprintf("%s.%d.gz", w.f.Name(), i-1), config)
|
||||
if err != nil {
|
||||
if !errors.Is(err, fs.ErrNotExist) {
|
||||
return nil, errors.Wrap(err, "error getting reference to decompressed log file")
|
||||
return nil, err
|
||||
}
|
||||
continue
|
||||
}
|
||||
if tmpFile == nil {
|
||||
} else if f == nil {
|
||||
// The log before `config.Since` does not need to read
|
||||
break
|
||||
continue
|
||||
}
|
||||
|
||||
files = append(files, tmpFile)
|
||||
continue
|
||||
}
|
||||
files = append(files, f)
|
||||
}
|
||||
|
@ -512,7 +453,7 @@ func (w *LogFile) openRotatedFiles(config logger.ReadConfig) (files []*os.File,
|
|||
return files, nil
|
||||
}
|
||||
|
||||
func decompressfile(fileName, destFileName string, since time.Time) (*os.File, error) {
|
||||
func (w *LogFile) maybeDecompressFile(fileName string, config logger.ReadConfig) (readAtCloser, error) {
|
||||
cf, err := open(fileName)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "error opening file for decompression")
|
||||
|
@ -528,26 +469,26 @@ func decompressfile(fileName, destFileName string, since time.Time) (*os.File, e
|
|||
// Extract the last log entry timestramp from the gzip header
|
||||
extra := &rotateFileMetadata{}
|
||||
err = json.Unmarshal(rc.Header.Extra, extra)
|
||||
if err == nil && extra.LastTime.Before(since) {
|
||||
if err == nil && !extra.LastTime.IsZero() && extra.LastTime.Before(config.Since) {
|
||||
return nil, nil
|
||||
}
|
||||
tmpf, err := w.decompress.Do(cf)
|
||||
return tmpf, errors.Wrap(err, "error decompressing log file")
|
||||
}
|
||||
|
||||
rs, err := openFile(destFileName, os.O_CREATE|os.O_RDWR, 0640)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "error creating file for copying decompressed log stream")
|
||||
func decompress(dst io.WriteSeeker, src io.ReadSeeker) error {
|
||||
if _, err := src.Seek(0, io.SeekStart); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
_, err = pools.Copy(rs, rc)
|
||||
rc, err := gzip.NewReader(src)
|
||||
if err != nil {
|
||||
rs.Close()
|
||||
rErr := unlink(rs.Name())
|
||||
if rErr != nil && !errors.Is(rErr, fs.ErrNotExist) {
|
||||
logrus.Errorf("Failed to remove logfile: %v", rErr)
|
||||
}
|
||||
return nil, errors.Wrap(err, "error while copying decompressed log stream to file")
|
||||
return err
|
||||
}
|
||||
|
||||
return rs, nil
|
||||
_, err = pools.Copy(dst, rc)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return rc.Close()
|
||||
}
|
||||
|
||||
func newSectionReader(f *os.File) (*io.SectionReader, error) {
|
||||
|
@ -597,7 +538,7 @@ func tailFiles(files []SizeReaderAt, watcher *logger.LogWatcher, dec Decoder, ge
|
|||
}
|
||||
} else {
|
||||
for _, r := range files {
|
||||
readers = append(readers, &wrappedReaderAt{ReaderAt: r})
|
||||
readers = append(readers, r)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -663,14 +604,3 @@ func watchFile(name string) (filenotify.FileWatcher, error) {
|
|||
|
||||
return fileWatcher, nil
|
||||
}
|
||||
|
||||
type wrappedReaderAt struct {
|
||||
io.ReaderAt
|
||||
pos int64
|
||||
}
|
||||
|
||||
func (r *wrappedReaderAt) Read(p []byte) (int, error) {
|
||||
n, err := r.ReaderAt.ReadAt(p, r.pos)
|
||||
r.pos += int64(n)
|
||||
return n, err
|
||||
}
|
||||
|
|
227
daemon/logger/loggerutils/sharedtemp.go
Normal file
227
daemon/logger/loggerutils/sharedtemp.go
Normal file
|
@ -0,0 +1,227 @@
|
|||
package loggerutils // import "github.com/docker/docker/daemon/logger/loggerutils"
|
||||
|
||||
import (
|
||||
"io"
|
||||
"io/fs"
|
||||
"os"
|
||||
"runtime"
|
||||
)
|
||||
|
||||
type fileConvertFn func(dst io.WriteSeeker, src io.ReadSeeker) error
|
||||
|
||||
type stfID uint64
|
||||
|
||||
// sharedTempFileConverter converts files using a user-supplied function and
|
||||
// writes the results to temporary files which are automatically cleaned up on
|
||||
// close. If another request is made to convert the same file, the conversion
|
||||
// result and temporary file are reused if they have not yet been cleaned up.
|
||||
//
|
||||
// A file is considered the same as another file using the os.SameFile function,
|
||||
// which compares file identity (e.g. device and inode numbers on Linux) and is
|
||||
// robust to file renames. Input files are assumed to be immutable; no attempt
|
||||
// is made to ascertain whether the file contents have changed between requests.
|
||||
//
|
||||
// One file descriptor is used per source file, irrespective of the number of
|
||||
// concurrent readers of the converted contents.
|
||||
type sharedTempFileConverter struct {
|
||||
// The directory where temporary converted files are to be written to.
|
||||
// If set to the empty string, the default directory for temporary files
|
||||
// is used.
|
||||
TempDir string
|
||||
|
||||
conv fileConvertFn
|
||||
st chan stfcState
|
||||
}
|
||||
|
||||
type stfcState struct {
|
||||
fl map[stfID]sharedTempFile
|
||||
nextID stfID
|
||||
}
|
||||
|
||||
type sharedTempFile struct {
|
||||
src os.FileInfo // Info about the source file for path-independent identification with os.SameFile.
|
||||
fd *os.File
|
||||
size int64
|
||||
ref int // Reference count of open readers on the temporary file.
|
||||
wait []chan<- stfConvertResult // Wait list for the conversion to complete.
|
||||
}
|
||||
|
||||
type stfConvertResult struct {
|
||||
fr *sharedFileReader
|
||||
err error
|
||||
}
|
||||
|
||||
func newSharedTempFileConverter(conv fileConvertFn) *sharedTempFileConverter {
|
||||
st := make(chan stfcState, 1)
|
||||
st <- stfcState{fl: make(map[stfID]sharedTempFile)}
|
||||
return &sharedTempFileConverter{conv: conv, st: st}
|
||||
}
|
||||
|
||||
// Do returns a reader for the contents of f as converted by the c.C function.
|
||||
// It is the caller's responsibility to close the returned reader.
|
||||
//
|
||||
// This function is safe for concurrent use by multiple goroutines.
|
||||
func (c *sharedTempFileConverter) Do(f *os.File) (*sharedFileReader, error) {
|
||||
stat, err := f.Stat()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
st := <-c.st
|
||||
for id, tf := range st.fl {
|
||||
// os.SameFile can have false positives if one of the files was
|
||||
// deleted before the other file was created -- such as during
|
||||
// log rotations... https://github.com/golang/go/issues/36895
|
||||
// Weed out those false positives by also comparing the files'
|
||||
// ModTime, which conveniently also handles the case of true
|
||||
// positives where the file has also been modified since it was
|
||||
// first converted.
|
||||
if os.SameFile(tf.src, stat) && tf.src.ModTime() == stat.ModTime() {
|
||||
return c.openExisting(st, id, tf)
|
||||
}
|
||||
}
|
||||
return c.openNew(st, f, stat)
|
||||
}
|
||||
|
||||
func (c *sharedTempFileConverter) openNew(st stfcState, f *os.File, stat os.FileInfo) (*sharedFileReader, error) {
|
||||
// Record that we are starting to convert this file so that any other
|
||||
// requests for the same source file while the conversion is in progress
|
||||
// can join.
|
||||
id := st.nextID
|
||||
st.nextID++
|
||||
st.fl[id] = sharedTempFile{src: stat}
|
||||
c.st <- st
|
||||
|
||||
dst, size, convErr := c.convert(f)
|
||||
|
||||
st = <-c.st
|
||||
flid := st.fl[id]
|
||||
|
||||
if convErr != nil {
|
||||
// Conversion failed. Delete it from the state so that future
|
||||
// requests to convert the same file can try again fresh.
|
||||
delete(st.fl, id)
|
||||
c.st <- st
|
||||
for _, w := range flid.wait {
|
||||
w <- stfConvertResult{err: convErr}
|
||||
}
|
||||
return nil, convErr
|
||||
}
|
||||
|
||||
flid.fd = dst
|
||||
flid.size = size
|
||||
flid.ref = len(flid.wait) + 1
|
||||
for _, w := range flid.wait {
|
||||
// Each waiter needs its own reader with an independent read pointer.
|
||||
w <- stfConvertResult{fr: flid.Reader(c, id)}
|
||||
}
|
||||
flid.wait = nil
|
||||
st.fl[id] = flid
|
||||
c.st <- st
|
||||
return flid.Reader(c, id), nil
|
||||
}
|
||||
|
||||
func (c *sharedTempFileConverter) openExisting(st stfcState, id stfID, v sharedTempFile) (*sharedFileReader, error) {
|
||||
if v.fd != nil {
|
||||
// Already converted.
|
||||
v.ref++
|
||||
st.fl[id] = v
|
||||
c.st <- st
|
||||
return v.Reader(c, id), nil
|
||||
}
|
||||
// The file has not finished being converted.
|
||||
// Add ourselves to the wait list. "Don't call us; we'll call you."
|
||||
wait := make(chan stfConvertResult, 1)
|
||||
v.wait = append(v.wait, wait)
|
||||
st.fl[id] = v
|
||||
c.st <- st
|
||||
|
||||
res := <-wait
|
||||
return res.fr, res.err
|
||||
|
||||
}
|
||||
|
||||
func (c *sharedTempFileConverter) convert(f *os.File) (converted *os.File, size int64, err error) {
|
||||
dst, err := os.CreateTemp(c.TempDir, "dockerdtemp.*")
|
||||
if err != nil {
|
||||
return nil, 0, err
|
||||
}
|
||||
defer func() {
|
||||
_ = dst.Close()
|
||||
// Delete the temporary file immediately so that final cleanup
|
||||
// of the file on disk is deferred to the OS once we close all
|
||||
// our file descriptors (or the process dies). Assuming no early
|
||||
// returns due to errors, the file will be open by this process
|
||||
// with a read-only descriptor at this point. As we don't care
|
||||
// about being able to reuse the file name -- it's randomly
|
||||
// generated and unique -- we can safely use os.Remove on
|
||||
// Windows.
|
||||
_ = os.Remove(dst.Name())
|
||||
}()
|
||||
err = c.conv(dst, f)
|
||||
if err != nil {
|
||||
return nil, 0, err
|
||||
}
|
||||
// Close the exclusive read-write file descriptor, catching any delayed
|
||||
// write errors (and on Windows, releasing the share-locks on the file)
|
||||
if err := dst.Close(); err != nil {
|
||||
_ = os.Remove(dst.Name())
|
||||
return nil, 0, err
|
||||
}
|
||||
// Open the file again read-only (without locking the file against
|
||||
// deletion on Windows).
|
||||
converted, err = open(dst.Name())
|
||||
if err != nil {
|
||||
return nil, 0, err
|
||||
}
|
||||
|
||||
// The position of the file's read pointer doesn't matter as all readers
|
||||
// will be accessing the file through its io.ReaderAt interface.
|
||||
size, err = converted.Seek(0, io.SeekEnd)
|
||||
if err != nil {
|
||||
_ = converted.Close()
|
||||
return nil, 0, err
|
||||
}
|
||||
return converted, size, nil
|
||||
}
|
||||
|
||||
type sharedFileReader struct {
|
||||
*io.SectionReader
|
||||
|
||||
c *sharedTempFileConverter
|
||||
id stfID
|
||||
closed bool
|
||||
}
|
||||
|
||||
func (stf sharedTempFile) Reader(c *sharedTempFileConverter, id stfID) *sharedFileReader {
|
||||
rdr := &sharedFileReader{SectionReader: io.NewSectionReader(stf.fd, 0, stf.size), c: c, id: id}
|
||||
runtime.SetFinalizer(rdr, (*sharedFileReader).Close)
|
||||
return rdr
|
||||
}
|
||||
|
||||
func (r *sharedFileReader) Close() error {
|
||||
if r.closed {
|
||||
return fs.ErrClosed
|
||||
}
|
||||
|
||||
st := <-r.c.st
|
||||
flid, ok := st.fl[r.id]
|
||||
if !ok {
|
||||
panic("invariant violation: temp file state missing from map")
|
||||
}
|
||||
flid.ref--
|
||||
lastRef := flid.ref <= 0
|
||||
if lastRef {
|
||||
delete(st.fl, r.id)
|
||||
} else {
|
||||
st.fl[r.id] = flid
|
||||
}
|
||||
r.closed = true
|
||||
r.c.st <- st
|
||||
|
||||
if lastRef {
|
||||
return flid.fd.Close()
|
||||
}
|
||||
runtime.SetFinalizer(r, nil)
|
||||
return nil
|
||||
}
|
256
daemon/logger/loggerutils/sharedtemp_test.go
Normal file
256
daemon/logger/loggerutils/sharedtemp_test.go
Normal file
|
@ -0,0 +1,256 @@
|
|||
package loggerutils // import "github.com/docker/docker/daemon/logger/loggerutils"
|
||||
|
||||
import (
|
||||
"io"
|
||||
"io/fs"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"runtime"
|
||||
"strings"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
"gotest.tools/v3/assert"
|
||||
"gotest.tools/v3/assert/cmp"
|
||||
)
|
||||
|
||||
func TestSharedTempFileConverter(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
t.Run("OneReaderAtATime", func(t *testing.T) {
|
||||
t.Parallel()
|
||||
dir := t.TempDir()
|
||||
name := filepath.Join(dir, "test.txt")
|
||||
createFile(t, name, "hello, world!")
|
||||
|
||||
uut := newSharedTempFileConverter(copyTransform(strings.ToUpper))
|
||||
uut.TempDir = dir
|
||||
|
||||
for i := 0; i < 3; i++ {
|
||||
t.Logf("Iteration %v", i)
|
||||
|
||||
rdr := convertPath(t, uut, name)
|
||||
assert.Check(t, cmp.Equal("HELLO, WORLD!", readAll(t, rdr)))
|
||||
assert.Check(t, rdr.Close())
|
||||
assert.Check(t, cmp.Equal(fs.ErrClosed, rdr.Close()), "closing an already-closed reader should return an error")
|
||||
}
|
||||
|
||||
assert.NilError(t, os.Remove(name))
|
||||
checkDirEmpty(t, dir)
|
||||
})
|
||||
|
||||
t.Run("RobustToRenames", func(t *testing.T) {
|
||||
t.Parallel()
|
||||
dir := t.TempDir()
|
||||
apath := filepath.Join(dir, "test.txt")
|
||||
createFile(t, apath, "file a")
|
||||
|
||||
var conversions int
|
||||
uut := newSharedTempFileConverter(
|
||||
func(dst io.WriteSeeker, src io.ReadSeeker) error {
|
||||
conversions++
|
||||
return copyTransform(strings.ToUpper)(dst, src)
|
||||
},
|
||||
)
|
||||
uut.TempDir = dir
|
||||
|
||||
ra1 := convertPath(t, uut, apath)
|
||||
|
||||
// Rotate the file to a new name and write a new file in its place.
|
||||
bpath := apath
|
||||
apath = filepath.Join(dir, "test2.txt")
|
||||
assert.NilError(t, os.Rename(bpath, apath))
|
||||
createFile(t, bpath, "file b")
|
||||
|
||||
rb1 := convertPath(t, uut, bpath) // Same path, different file.
|
||||
ra2 := convertPath(t, uut, apath) // New path, old file.
|
||||
assert.Check(t, cmp.Equal(2, conversions), "expected only one conversion per unique file")
|
||||
|
||||
// Interleave reading and closing to shake out ref-counting bugs:
|
||||
// closing one reader shouldn't affect any other open readers.
|
||||
assert.Check(t, cmp.Equal("FILE A", readAll(t, ra1)))
|
||||
assert.NilError(t, ra1.Close())
|
||||
assert.Check(t, cmp.Equal("FILE A", readAll(t, ra2)))
|
||||
assert.NilError(t, ra2.Close())
|
||||
assert.Check(t, cmp.Equal("FILE B", readAll(t, rb1)))
|
||||
assert.NilError(t, rb1.Close())
|
||||
|
||||
assert.NilError(t, os.Remove(apath))
|
||||
assert.NilError(t, os.Remove(bpath))
|
||||
checkDirEmpty(t, dir)
|
||||
})
|
||||
|
||||
t.Run("ConcurrentRequests", func(t *testing.T) {
|
||||
t.Parallel()
|
||||
dir := t.TempDir()
|
||||
name := filepath.Join(dir, "test.txt")
|
||||
createFile(t, name, "hi there")
|
||||
|
||||
var conversions int32
|
||||
notify := make(chan chan struct{}, 1)
|
||||
firstConversionStarted := make(chan struct{})
|
||||
notify <- firstConversionStarted
|
||||
unblock := make(chan struct{})
|
||||
uut := newSharedTempFileConverter(
|
||||
func(dst io.WriteSeeker, src io.ReadSeeker) error {
|
||||
t.Log("Convert: enter")
|
||||
defer t.Log("Convert: exit")
|
||||
select {
|
||||
case c := <-notify:
|
||||
close(c)
|
||||
default:
|
||||
}
|
||||
<-unblock
|
||||
atomic.AddInt32(&conversions, 1)
|
||||
return copyTransform(strings.ToUpper)(dst, src)
|
||||
},
|
||||
)
|
||||
uut.TempDir = dir
|
||||
|
||||
closers := make(chan io.Closer, 4)
|
||||
var wg sync.WaitGroup
|
||||
wg.Add(3)
|
||||
for i := 0; i < 3; i++ {
|
||||
i := i
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
t.Logf("goroutine %v: enter", i)
|
||||
defer t.Logf("goroutine %v: exit", i)
|
||||
f := convertPath(t, uut, name)
|
||||
assert.Check(t, cmp.Equal("HI THERE", readAll(t, f)), "in goroutine %v", i)
|
||||
closers <- f
|
||||
}()
|
||||
}
|
||||
|
||||
select {
|
||||
case <-firstConversionStarted:
|
||||
case <-time.After(2 * time.Second):
|
||||
t.Fatal("the first conversion should have started by now")
|
||||
}
|
||||
close(unblock)
|
||||
t.Log("starting wait")
|
||||
wg.Wait()
|
||||
t.Log("wait done")
|
||||
|
||||
f := convertPath(t, uut, name)
|
||||
closers <- f
|
||||
close(closers)
|
||||
assert.Check(t, cmp.Equal("HI THERE", readAll(t, f)), "after all goroutines returned")
|
||||
for c := range closers {
|
||||
assert.Check(t, c.Close())
|
||||
}
|
||||
|
||||
assert.Check(t, cmp.Equal(int32(1), conversions))
|
||||
|
||||
assert.NilError(t, os.Remove(name))
|
||||
checkDirEmpty(t, dir)
|
||||
})
|
||||
|
||||
t.Run("ConvertError", func(t *testing.T) {
|
||||
t.Parallel()
|
||||
dir := t.TempDir()
|
||||
name := filepath.Join(dir, "test.txt")
|
||||
createFile(t, name, "hi there")
|
||||
src, err := open(name)
|
||||
assert.NilError(t, err)
|
||||
defer src.Close()
|
||||
|
||||
fakeErr := errors.New("fake error")
|
||||
var start sync.WaitGroup
|
||||
start.Add(3)
|
||||
uut := newSharedTempFileConverter(
|
||||
func(dst io.WriteSeeker, src io.ReadSeeker) error {
|
||||
start.Wait()
|
||||
runtime.Gosched()
|
||||
if fakeErr != nil {
|
||||
return fakeErr
|
||||
}
|
||||
return copyTransform(strings.ToUpper)(dst, src)
|
||||
},
|
||||
)
|
||||
uut.TempDir = dir
|
||||
|
||||
var done sync.WaitGroup
|
||||
done.Add(3)
|
||||
for i := 0; i < 3; i++ {
|
||||
i := i
|
||||
go func() {
|
||||
defer done.Done()
|
||||
t.Logf("goroutine %v: enter", i)
|
||||
defer t.Logf("goroutine %v: exit", i)
|
||||
start.Done()
|
||||
_, err := uut.Do(src)
|
||||
assert.Check(t, errors.Is(err, fakeErr), "in goroutine %v", i)
|
||||
}()
|
||||
}
|
||||
done.Wait()
|
||||
|
||||
// Conversion errors should not be "sticky". A subsequent
|
||||
// request should retry from scratch.
|
||||
fakeErr = errors.New("another fake error")
|
||||
_, err = uut.Do(src)
|
||||
assert.Check(t, errors.Is(err, fakeErr))
|
||||
|
||||
fakeErr = nil
|
||||
f, err := uut.Do(src)
|
||||
assert.Check(t, err)
|
||||
assert.Check(t, cmp.Equal("HI THERE", readAll(t, f)))
|
||||
assert.Check(t, f.Close())
|
||||
|
||||
// Files pending delete continue to show up in directory
|
||||
// listings on Windows RS5. Close the remaining handle before
|
||||
// deleting the file to prevent spurious failures with
|
||||
// checkDirEmpty.
|
||||
assert.Check(t, src.Close())
|
||||
assert.NilError(t, os.Remove(name))
|
||||
checkDirEmpty(t, dir)
|
||||
|
||||
})
|
||||
}
|
||||
|
||||
func createFile(t *testing.T, path string, content string) {
|
||||
t.Helper()
|
||||
f, err := openFile(path, os.O_RDWR|os.O_CREATE|os.O_EXCL, 0644)
|
||||
assert.NilError(t, err)
|
||||
_, err = io.WriteString(f, content)
|
||||
assert.NilError(t, err)
|
||||
assert.NilError(t, f.Close())
|
||||
}
|
||||
|
||||
func convertPath(t *testing.T, uut *sharedTempFileConverter, path string) *sharedFileReader {
|
||||
t.Helper()
|
||||
f, err := open(path)
|
||||
assert.NilError(t, err)
|
||||
defer func() { assert.NilError(t, f.Close()) }()
|
||||
r, err := uut.Do(f)
|
||||
assert.NilError(t, err)
|
||||
return r
|
||||
}
|
||||
|
||||
func readAll(t *testing.T, r io.Reader) string {
|
||||
t.Helper()
|
||||
v, err := io.ReadAll(r)
|
||||
assert.NilError(t, err)
|
||||
return string(v)
|
||||
}
|
||||
|
||||
func checkDirEmpty(t *testing.T, path string) {
|
||||
t.Helper()
|
||||
ls, err := os.ReadDir(path)
|
||||
assert.NilError(t, err)
|
||||
assert.Check(t, cmp.Len(ls, 0), "directory should be free of temp files")
|
||||
}
|
||||
|
||||
func copyTransform(f func(string) string) func(dst io.WriteSeeker, src io.ReadSeeker) error {
|
||||
return func(dst io.WriteSeeker, src io.ReadSeeker) error {
|
||||
s, err := io.ReadAll(src)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
_, err = io.WriteString(dst, f(string(s)))
|
||||
return err
|
||||
}
|
||||
}
|
Loading…
Reference in a new issue