WIP2 logging done somewhat

This commit is contained in:
Christian Schwarz 2018-08-10 17:06:00 +02:00
parent 74445a0017
commit c1f3076eb3
14 changed files with 573 additions and 397 deletions

View File

@ -2,12 +2,15 @@ package cmd
import ( import (
"context" "context"
"fmt"
"io" "io"
"net" "net"
"strings"
"time" "time"
"github.com/problame/go-streamrpc" "github.com/problame/go-streamrpc"
"github.com/zrepl/zrepl/util" "github.com/zrepl/zrepl/util"
"github.com/zrepl/zrepl/logger"
) )
type logNetConnConnecter struct { type logNetConnConnecter struct {
@ -74,3 +77,29 @@ func (netsshConnToNetConnAdatper) SetDeadline(t time.Time) error { return nil }
func (netsshConnToNetConnAdatper) SetReadDeadline(t time.Time) error { return nil } func (netsshConnToNetConnAdatper) SetReadDeadline(t time.Time) error { return nil }
func (netsshConnToNetConnAdatper) SetWriteDeadline(t time.Time) error { return nil } func (netsshConnToNetConnAdatper) SetWriteDeadline(t time.Time) error { return nil }
type streamrpcLogAdaptor = twoClassLogAdaptor
type replicationLogAdaptor = twoClassLogAdaptor
type twoClassLogAdaptor struct {
logger.Logger
}
var _ streamrpc.Logger = twoClassLogAdaptor{}
func (a twoClassLogAdaptor) Errorf(fmtStr string, args... interface{}) {
const errorSuffix = ": %s"
if len(args) == 1 {
if err, ok := args[0].(error); ok && strings.HasSuffix(fmtStr, errorSuffix) {
msg := strings.TrimSuffix(fmtStr, errorSuffix)
a.WithError(err).Error(msg)
return
}
}
a.Logger.Error(fmt.Sprintf(fmtStr, args...))
}
func (a twoClassLogAdaptor) Infof(fmtStr string, args... interface{}) {
a.Logger.Info(fmt.Sprintf(fmtStr, args...))
}

View File

@ -8,6 +8,7 @@ import (
"io" "io"
"net" "net"
"net/http" "net/http"
"github.com/zrepl/zrepl/logger"
) )
type ControlJob struct { type ControlJob struct {
@ -126,7 +127,7 @@ func (j jsonResponder) ServeHTTP(w http.ResponseWriter, r *http.Request) {
} }
type requestLogger struct { type requestLogger struct {
log Logger log logger.Logger
handler http.Handler handler http.Handler
handlerFunc http.HandlerFunc handlerFunc http.HandlerFunc
} }

View File

@ -146,12 +146,7 @@ outer:
j.mainTask.Log().Debug("replicating from lhs to rhs") j.mainTask.Log().Debug("replicating from lhs to rhs")
j.mainTask.Enter("replicate") j.mainTask.Enter("replicate")
replication.Replicate( replication.Replicate(ctx, replication.NewEndpointPairPull(sender, receiver))
ctx,
replication.NewEndpointPairPull(sender, receiver),
replication.NewIncrementalPathReplicator(),
nil, //FIXME
)
j.mainTask.Finish() j.mainTask.Finish()

View File

@ -1,15 +1,18 @@
package cmd package cmd
import ( import (
"os"
"os/signal"
"syscall"
"time" "time"
"context" "context"
"fmt" "fmt"
"github.com/mitchellh/mapstructure" "github.com/mitchellh/mapstructure"
"github.com/pkg/errors" "github.com/pkg/errors"
"github.com/zrepl/zrepl/util"
"github.com/zrepl/zrepl/cmd/replication"
"github.com/problame/go-streamrpc" "github.com/problame/go-streamrpc"
"github.com/zrepl/zrepl/cmd/replication"
) )
type PullJob struct { type PullJob struct {
@ -91,7 +94,7 @@ func parsePullJob(c JobParsingContext, name string, i map[string]interface{}) (j
if j.Debug.Conn.ReadDump != "" || j.Debug.Conn.WriteDump != "" { if j.Debug.Conn.ReadDump != "" || j.Debug.Conn.WriteDump != "" {
logConnecter := logNetConnConnecter{ logConnecter := logNetConnConnecter{
Connecter: j.Connect, Connecter: j.Connect,
ReadDump: j.Debug.Conn.ReadDump, ReadDump: j.Debug.Conn.ReadDump,
WriteDump: j.Debug.Conn.WriteDump, WriteDump: j.Debug.Conn.WriteDump,
} }
j.Connect = logConnecter j.Connect = logConnecter
@ -113,6 +116,9 @@ func (j *PullJob) JobStart(ctx context.Context) {
j.task = NewTask("main", j, log) j.task = NewTask("main", j, log)
// j.task is idle here idle here // j.task is idle here idle here
usr1 := make(chan os.Signal)
signal.Notify(usr1, syscall.SIGUSR1)
defer signal.Stop(usr1)
ticker := time.NewTicker(j.Interval) ticker := time.NewTicker(j.Interval)
for { for {
@ -130,23 +136,25 @@ func (j *PullJob) JobStart(ctx context.Context) {
j.task.Log().WithError(ctx.Err()).Info("context") j.task.Log().WithError(ctx.Err()).Info("context")
return return
case <-ticker.C: case <-ticker.C:
case <-usr1:
} }
} }
} }
var STREAMRPC_CONFIG = &streamrpc.ConnConfig{ // FIXME oversight and configurability var STREAMRPC_CONFIG = &streamrpc.ConnConfig{ // FIXME oversight and configurability
RxHeaderMaxLen: 4096, RxHeaderMaxLen: 4096,
RxStructuredMaxLen: 4096 * 4096, RxStructuredMaxLen: 4096 * 4096,
RxStreamMaxChunkSize: 4096 * 4096, RxStreamMaxChunkSize: 4096 * 4096,
TxChunkSize: 4096 * 4096, TxChunkSize: 4096 * 4096,
RxTimeout: streamrpc.Timeout{ RxTimeout: streamrpc.Timeout{
Progress: 10*time.Second, Progress: 10 * time.Second,
}, },
TxTimeout: streamrpc.Timeout{ TxTimeout: streamrpc.Timeout{
Progress: 10*time.Second, Progress: 10 * time.Second,
}, },
} }
func (j *PullJob) doRun(ctx context.Context) { func (j *PullJob) doRun(ctx context.Context) {
j.task.Enter("run") j.task.Enter("run")
@ -174,26 +182,10 @@ func (j *PullJob) doRun(ctx context.Context) {
return return
} }
replicator := replication.NewIncrementalPathReplicator() ctx = replication.ContextWithLogger(ctx, replicationLogAdaptor{j.task.Log().WithField("subsystem", "replication")})
ctx = context.WithValue(ctx, replication.ContextKeyLog, j.task.Log()) ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{j.task.Log().WithField("subsystem", "rpc.protocol")})
ctx = context.WithValue(ctx, streamrpc.ContextKeyLogger, j.task.Log()) ctx = context.WithValue(ctx, contextKeyLog, j.task.Log().WithField("subsystem", "rpc.endpoint"))
ctx, enforceDeadline := util.ContextWithOptionalDeadline(ctx) replication.Replicate(ctx, replication.NewEndpointPairPull(sender, puller))
// Try replicating each file system regardless of j.Interval
// (this does not solve the underlying problem that j.Interval is too short,
// but it covers the case of initial replication taking longer than all
// incremental replications afterwards)
allTriedOnce := make(chan struct{})
replicationBegin := time.Now()
go func() {
select {
case <-allTriedOnce:
enforceDeadline(replicationBegin.Add(j.Interval))
case <-ctx.Done():
}
}()
replication.Replicate(ctx, replication.NewEndpointPairPull(sender, puller), replicator, allTriedOnce)
client.Close() client.Close()
j.task.Finish() j.task.Finish()

View File

@ -71,8 +71,8 @@ func parseSourceJob(c JobParsingContext, name string, i map[string]interface{})
if j.Debug.Conn.ReadDump != "" || j.Debug.Conn.WriteDump != "" { if j.Debug.Conn.ReadDump != "" || j.Debug.Conn.WriteDump != "" {
logServe := logListenerFactory{ logServe := logListenerFactory{
ListenerFactory: j.Serve, ListenerFactory: j.Serve,
ReadDump: j.Debug.Conn.ReadDump, ReadDump: j.Debug.Conn.ReadDump,
WriteDump: j.Debug.Conn.WriteDump, WriteDump: j.Debug.Conn.WriteDump,
} }
j.Serve = logServe j.Serve = logServe
} }
@ -209,17 +209,13 @@ func (j *SourceJob) handleConnection(conn net.Conn, task *Task) {
task.Log().Info("handling client connection") task.Log().Info("handling client connection")
senderEP := NewSenderEndpoint(j.Filesystems, NewPrefixFilter(j.SnapshotPrefix)) senderEP := NewSenderEndpoint(j.Filesystems, NewPrefixFilter(j.SnapshotPrefix))
handler := HandlerAdaptor{senderEP, task.Log()} ctx := context.Background()
// FIXME logging support or erase config ctx = context.WithValue(ctx, contextKeyLog, task.Log().WithField("subsystem", "rpc.endpoint"))
//if j.Debug.RPC.Log { ctx = streamrpc.ContextWithLogger(ctx, streamrpcLogAdaptor{task.Log().WithField("subsystem", "rpc.protocol")})
// rpclog := task.Log().WithField("subsystem", "rpc") handler := HandlerAdaptor{senderEP}
// rpcServer.SetLogger(rpclog, true) if err := streamrpc.ServeConn(ctx, conn, STREAMRPC_CONFIG, handler.Handle); err != nil {
//}
if err := streamrpc.ServeConn(context.TODO(), conn, STREAMRPC_CONFIG, handler.Handle); err != nil {
task.Log().WithError(err).Error("error serving connection") task.Log().WithError(err).Error("error serving connection")
} else { } else {
task.Log().Info("client closed connection") task.Log().Info("client closed connection")

View File

@ -9,7 +9,7 @@ import (
"io" "io"
"os" "os"
"os/signal" "os/signal"
"strings" //"strings"
"sync" "sync"
"syscall" "syscall"
"time" "time"
@ -88,6 +88,10 @@ const (
contextKeyDaemon contextKey = contextKey("daemon") contextKeyDaemon contextKey = contextKey("daemon")
) )
func getLogger(ctx context.Context) Logger {
return ctx.Value(contextKeyLog).(Logger)
}
type Daemon struct { type Daemon struct {
conf *Config conf *Config
startedAt time.Time startedAt time.Time
@ -276,7 +280,7 @@ func (p *taskProgress) Read() (out taskProgress) {
type taskActivity struct { type taskActivity struct {
name string name string
idle bool idle bool
logger *logger.Logger logger logger.Logger
// The progress of the task that is updated by UpdateIO() and UpdateLogEntry() // The progress of the task that is updated by UpdateIO() and UpdateLogEntry()
// //
// Progress happens on a task-level and is thus global to the task. // Progress happens on a task-level and is thus global to the task.
@ -285,7 +289,7 @@ type taskActivity struct {
progress *taskProgress progress *taskProgress
} }
func NewTask(name string, parent Job, lg *logger.Logger) *Task { func NewTask(name string, parent Job, lg logger.Logger) *Task {
t := &Task{ t := &Task{
name: name, name: name,
parent: parent, parent: parent,
@ -336,9 +340,10 @@ func (t *Task) Enter(activity string) {
} }
act := &taskActivity{activity, false, nil, prev.progress} act := &taskActivity{activity, false, nil, prev.progress}
t.activities.PushFront(act) t.activities.PushFront(act)
stack := t.buildActivityStack() //stack := t.buildActivityStack()
activityField := strings.Join(stack, ".") //activityField := strings.Join(stack, ".")
act.logger = prev.logger.ReplaceField(logTaskField, activityField) act.logger = prev.logger
// act.logger = prev.logger.ReplaceField(logTaskField, activityField)
t.activitiesLastUpdate = time.Now() t.activitiesLastUpdate = time.Now()
} }
@ -425,7 +430,7 @@ func (t *Task) Finish() {
// Returns a logger derived from the logger passed to the constructor function. // Returns a logger derived from the logger passed to the constructor function.
// The logger's task field contains the current activity stack joined by '.'. // The logger's task field contains the current activity stack joined by '.'.
func (t *Task) Log() *logger.Logger { func (t *Task) Log() logger.Logger {
t.rwl.RLock() t.rwl.RLock()
defer t.rwl.RUnlock() defer t.rwl.RUnlock()
// FIXME should influence TaskStatus's LastUpdate field // FIXME should influence TaskStatus's LastUpdate field

View File

@ -25,6 +25,7 @@ const (
logJobField string = "job" logJobField string = "job"
logTaskField string = "task" logTaskField string = "task"
logFSField string = "filesystem" logFSField string = "filesystem"
logSubsysField string = "subsystem"
logMapFromField string = "map_from" logMapFromField string = "map_from"
logMapToField string = "map_to" logMapToField string = "map_to"
logIncFromField string = "inc_from" logIncFromField string = "inc_from"
@ -77,7 +78,7 @@ func (f *HumanFormatter) Format(e *logger.Entry) (out []byte, err error) {
fmt.Fprintf(&line, "[%s]", e.Level.Short()) fmt.Fprintf(&line, "[%s]", e.Level.Short())
} }
prefixFields := []string{logJobField, logTaskField, logFSField} prefixFields := []string{logJobField, logTaskField, logSubsysField}
prefixed := make(map[string]bool, len(prefixFields)+2) prefixed := make(map[string]bool, len(prefixFields)+2)
for _, field := range prefixFields { for _, field := range prefixFields {
val, ok := e.Fields[field].(string) val, ok := e.Fields[field].(string)
@ -91,18 +92,18 @@ func (f *HumanFormatter) Format(e *logger.Entry) (out []byte, err error) {
} }
} }
// even more prefix fields // even more prefix fields
mapFrom, mapFromOk := e.Fields[logMapFromField].(string) //mapFrom, mapFromOk := e.Fields[logMapFromField].(string)
mapTo, mapToOk := e.Fields[logMapToField].(string) //mapTo, mapToOk := e.Fields[logMapToField].(string)
if mapFromOk && mapToOk && !f.ignored(logMapFromField) && !f.ignored(logMapToField) { //if mapFromOk && mapToOk && !f.ignored(logMapFromField) && !f.ignored(logMapToField) {
fmt.Fprintf(&line, "[%s => %s]", mapFrom, mapTo) // fmt.Fprintf(&line, "[%s => %s]", mapFrom, mapTo)
prefixed[logMapFromField], prefixed[logMapToField] = true, true // prefixed[logMapFromField], prefixed[logMapToField] = true, true
} //}
incFrom, incFromOk := e.Fields[logIncFromField].(string) //incFrom, incFromOk := e.Fields[logIncFromField].(string)
incTo, incToOk := e.Fields[logIncToField].(string) //incTo, incToOk := e.Fields[logIncToField].(string)
if incFromOk && incToOk && !f.ignored(logIncFromField) && !f.ignored(logMapToField) { //if incFromOk && incToOk && !f.ignored(logIncFromField) && !f.ignored(logMapToField) {
fmt.Fprintf(&line, "[%s => %s]", incFrom, incTo) // fmt.Fprintf(&line, "[%s => %s]", incFrom, incTo)
prefixed[logIncFromField], prefixed[logIncToField] = true, true // prefixed[logIncFromField], prefixed[logIncToField] = true, true
} //}
if line.Len() > 0 { if line.Len() > 0 {
fmt.Fprint(&line, ": ") fmt.Fprint(&line, ": ")
@ -179,7 +180,7 @@ func (f *LogfmtFormatter) Format(e *logger.Entry) ([]byte, error) {
// at least try and put job and task in front // at least try and put job and task in front
prefixed := make(map[string]bool, 2) prefixed := make(map[string]bool, 2)
prefix := []string{logJobField, logTaskField} prefix := []string{logJobField, logTaskField, logSubsysField}
for _, pf := range prefix { for _, pf := range prefix {
v, ok := e.Fields[pf] v, ok := e.Fields[pf]
if !ok { if !ok {

View File

@ -24,7 +24,7 @@ var (
zreplVersion string // set by build infrastructure zreplVersion string // set by build infrastructure
) )
type Logger = *logger.Logger type Logger = logger.Logger
var RootCmd = &cobra.Command{ var RootCmd = &cobra.Command{
Use: "zrepl", Use: "zrepl",

View File

@ -9,7 +9,6 @@ import (
"github.com/pkg/errors" "github.com/pkg/errors"
"github.com/golang/protobuf/proto" "github.com/golang/protobuf/proto"
"bytes" "bytes"
"os"
"context" "context"
) )
@ -71,7 +70,6 @@ func (p *SenderEndpoint) ListFilesystemVersions(ctx context.Context, fs string)
} }
func (p *SenderEndpoint) Send(ctx context.Context, r *replication.SendReq) (*replication.SendRes, io.ReadCloser, error) { func (p *SenderEndpoint) Send(ctx context.Context, r *replication.SendReq) (*replication.SendRes, io.ReadCloser, error) {
os.Stderr.WriteString("sending " + r.String() + "\n")
dp, err := zfs.NewDatasetPath(r.Filesystem) dp, err := zfs.NewDatasetPath(r.Filesystem)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
@ -175,23 +173,27 @@ func (e *ReceiverEndpoint) Receive(ctx context.Context, req *replication.Receive
var visitErr error var visitErr error
f := zfs.NewDatasetPathForest() f := zfs.NewDatasetPathForest()
f.Add(lp) f.Add(lp)
getLogger(ctx).Debug("begin tree-walk")
f.WalkTopDown(func(v zfs.DatasetPathVisit) (visitChildTree bool) { f.WalkTopDown(func(v zfs.DatasetPathVisit) (visitChildTree bool) {
if v.Path.Equal(lp) { if v.Path.Equal(lp) {
return false return false
} }
_, err := zfs.ZFSGet(v.Path, []string{zfs.ZREPL_PLACEHOLDER_PROPERTY_NAME}) _, err := zfs.ZFSGet(v.Path, []string{zfs.ZREPL_PLACEHOLDER_PROPERTY_NAME})
if err != nil { if err != nil {
os.Stderr.WriteString("error zfsget " + err.Error() + "\n")
// interpret this as an early exit of the zfs binary due to the fs not existing // interpret this as an early exit of the zfs binary due to the fs not existing
if err := zfs.ZFSCreatePlaceholderFilesystem(v.Path); err != nil { if err := zfs.ZFSCreatePlaceholderFilesystem(v.Path); err != nil {
os.Stderr.WriteString("error creating placeholder " + v.Path.ToString() + "\n") getLogger(ctx).
WithError(err).
WithField("placeholder_fs", v.Path).
Error("cannot create placeholder filesystem")
visitErr = err visitErr = err
return false return false
} }
} }
os.Stderr.WriteString(v.Path.ToString() + " exists\n") getLogger(ctx).WithField("filesystem", v.Path.ToString()).Debug("exists")
return true // leave this fs as is return true // leave this fs as is
}) })
getLogger(ctx).WithField("visitErr", visitErr).Debug("complete tree-walk")
if visitErr != nil { if visitErr != nil {
return visitErr return visitErr
@ -210,7 +212,7 @@ func (e *ReceiverEndpoint) Receive(ctx context.Context, req *replication.Receive
args = append(args, "-F") args = append(args, "-F")
} }
os.Stderr.WriteString("receiving...\n") getLogger(ctx).Debug("start receive command")
if err := zfs.ZFSRecv(lp.ToString(), sendStream, args...); err != nil { if err := zfs.ZFSRecv(lp.ToString(), sendStream, args...); err != nil {
return err return err
@ -322,16 +324,10 @@ func (s RemoteEndpoint) Receive(ctx context.Context, r *replication.ReceiveReq,
type HandlerAdaptor struct { type HandlerAdaptor struct {
ep replication.ReplicationEndpoint ep replication.ReplicationEndpoint
log Logger
} }
func (a *HandlerAdaptor) Handle(ctx context.Context, endpoint string, reqStructured *bytes.Buffer, reqStream io.ReadCloser) (resStructured *bytes.Buffer, resStream io.ReadCloser, err error) { func (a *HandlerAdaptor) Handle(ctx context.Context, endpoint string, reqStructured *bytes.Buffer, reqStream io.ReadCloser) (resStructured *bytes.Buffer, resStream io.ReadCloser, err error) {
if a.log != nil {
// FIXME validate type conversion here?
ctx = context.WithValue(ctx, streamrpc.ContextKeyLogger, a.log)
}
switch endpoint { switch endpoint {
case RPCListFilesystems: case RPCListFilesystems:
var req replication.ListFilesystemReq var req replication.ListFilesystemReq

View File

@ -41,6 +41,10 @@ func FilesystemVersionFromZFS(fsv zfs.FilesystemVersion) *FilesystemVersion {
} }
} }
func (v *FilesystemVersion) CreationAsTime() (time.Time, error) {
return time.Parse(time.RFC3339, v.Creation)
}
func (v *FilesystemVersion) ZFSFilesystemVersion() *zfs.FilesystemVersion { func (v *FilesystemVersion) ZFSFilesystemVersion() *zfs.FilesystemVersion {
ct := time.Time{} ct := time.Time{}
if v.Creation != "" { if v.Creation != "" {

View File

@ -2,18 +2,20 @@ package replication
import ( import (
"context" "context"
"io"
"container/list"
"fmt" "fmt"
"github.com/zrepl/zrepl/logger"
"io"
"net" "net"
"sort"
"time"
) )
type ReplicationEndpoint interface { type ReplicationEndpoint interface {
// Does not include placeholder filesystems // Does not include placeholder filesystems
ListFilesystems(ctx context.Context) ([]*Filesystem, error) ListFilesystems(ctx context.Context) ([]*Filesystem, error)
ListFilesystemVersions(ctx context.Context, fs string) ([]*FilesystemVersion, error) // fix depS ListFilesystemVersions(ctx context.Context, fs string) ([]*FilesystemVersion, error) // fix depS
Sender Send(ctx context.Context, r *SendReq) (*SendRes, io.ReadCloser, error)
Receiver Receive(ctx context.Context, r *ReceiveReq, sendStream io.ReadCloser) error
} }
type FilteredError struct{ fs string } type FilteredError struct{ fs string }
@ -73,62 +75,175 @@ func (p EndpointPair) Mode() ReplicationMode {
type contextKey int type contextKey int
const ( const (
ContextKeyLog contextKey = iota contextKeyLog contextKey = iota
) )
type Logger interface{ //type Logger interface {
Printf(fmt string, args ... interface{}) // Infof(fmt string, args ...interface{})
// Errorf(fmt string, args ...interface{})
//}
//var _ Logger = nullLogger{}
//type nullLogger struct{}
//
//func (nullLogger) Infof(fmt string, args ...interface{}) {}
//func (nullLogger) Errorf(fmt string, args ...interface{}) {}
type Logger = logger.Logger
func ContextWithLogger(ctx context.Context, l Logger) context.Context {
return context.WithValue(ctx, contextKeyLog, l)
} }
type replicationWork struct { func getLogger(ctx context.Context) Logger {
fs *Filesystem l, ok := ctx.Value(contextKeyLog).(Logger)
} if !ok {
l = logger.NewNullLogger()
type FilesystemReplicationResult struct {
Done bool
Retry bool
Unfixable bool
}
func handleGenericEndpointError(err error) FilesystemReplicationResult {
if _, ok := err.(net.Error); ok {
return FilesystemReplicationResult{Retry: true}
} }
return FilesystemReplicationResult{Unfixable: true} return l
} }
func driveFSReplication(ctx context.Context, ws *list.List, allTriedOnce chan struct{}, log Logger, f func(w *replicationWork) FilesystemReplicationResult) { type replicationStep struct {
initialLen, fCalls := ws.Len(), 0 from, to *FilesystemVersion
for ws.Len() > 0 { fswork *replicateFSWork
}
func (s *replicationStep) String() string {
if s.from == nil { // FIXME: ZFS semantics are that to is nil on non-incremental send
return fmt.Sprintf("%s%s (full)", s.fswork.fs.Path, s.to.RelName())
} else {
return fmt.Sprintf("%s(%s => %s)", s.fswork.fs.Path, s.from.RelName(), s.to.RelName())
}
}
func newReplicationStep(from, to *FilesystemVersion) *replicationStep {
return &replicationStep{from: from, to: to}
}
type replicateFSWork struct {
fs *Filesystem
steps []*replicationStep
currentStep int
errorCount int
}
func newReplicateFSWork(fs *Filesystem) *replicateFSWork {
if fs == nil {
panic("implementation error")
}
return &replicateFSWork{
fs: fs,
steps: make([]*replicationStep, 0),
}
}
func newReplicateFSWorkWithConflict(fs *Filesystem, conflict error) *replicateFSWork {
// FIXME ignore conflict for now, but will be useful later when we make the replicationPlan exportable
return &replicateFSWork{
fs: fs,
steps: make([]*replicationStep, 0),
}
}
func (r *replicateFSWork) AddStep(step *replicationStep) {
if step == nil {
panic("implementation error")
}
if step.fswork != nil {
panic("implementation error")
}
step.fswork = r
r.steps = append(r.steps, step)
}
func (w *replicateFSWork) CurrentStepDate() time.Time {
if len(w.steps) == 0 {
return time.Time{}
}
toTime, err := w.steps[w.currentStep].to.CreationAsTime()
if err != nil {
panic(err) // implementation inconsistent: should not admit invalid FilesystemVersion objects
}
return toTime
}
func (w *replicateFSWork) CurrentStep() *replicationStep {
if w.currentStep >= len(w.steps) {
return nil
}
return w.steps[w.currentStep]
}
func (w *replicateFSWork) CompleteStep() {
w.currentStep++
}
type replicationPlan struct {
fsws []*replicateFSWork
}
func newReplicationPlan() *replicationPlan {
return &replicationPlan{
fsws: make([]*replicateFSWork, 0),
}
}
func (p *replicationPlan) addWork(work *replicateFSWork) {
p.fsws = append(p.fsws, work)
}
func (p *replicationPlan) executeOldestFirst(ctx context.Context, doStep func(fs *Filesystem, from, to *FilesystemVersion) tryRes) {
log := getLogger(ctx)
for {
select { select {
case <-ctx.Done(): case <-ctx.Done():
log.Printf("aborting replication due to context error: %s", ctx.Err()) log.WithError(ctx.Err()).Info("aborting replication due to context error")
return return
default: default:
} }
w := ws.Remove(ws.Front()).(*replicationWork) // FIXME poor man's nested priority queue
res := f(w) pending := make([]*replicateFSWork, 0, len(p.fsws))
fCalls++ for _, fsw := range p.fsws {
if fCalls == initialLen { if fsw.CurrentStep() != nil {
select { pending = append(pending, fsw)
case allTriedOnce <- struct{}{}:
default:
} }
} }
if res.Done { sort.Slice(pending, func(i, j int) bool {
log.Printf("finished replication of %s", w.fs.Path) if pending[i].errorCount == pending[j].errorCount {
continue return pending[i].CurrentStepDate().Before(pending[j].CurrentStepDate())
}
return pending[i].errorCount < pending[j].errorCount
})
// pending is now sorted ascending by errorCount,CurrentStep().Creation
if len(pending) == 0 {
log.Info("replication complete")
return
} }
if res.Unfixable { fsw := pending[0]
log.Printf("aborting replication of %s after unfixable error", w.fs.Path) step := fsw.CurrentStep()
continue if step == nil {
panic("implementation error")
}
log.WithField("step", step).Info("begin replication step")
res := doStep(step.fswork.fs, step.from, step.to)
if res.done {
log.Info("replication step successful")
fsw.errorCount = 0
fsw.CompleteStep()
} else {
log.Error("replication step failed, queuing for retry result")
fsw.errorCount++
} }
log.Printf("queuing replication of %s for retry", w.fs.Path)
ws.PushBack(w)
} }
} }
func resolveConflict(conflict error) (path []*FilesystemVersion, msg string) { func resolveConflict(conflict error) (path []*FilesystemVersion, msg string) {
@ -137,7 +252,7 @@ func resolveConflict(conflict error) (path []*FilesystemVersion, msg string) {
// FIXME hard-coded replication policy: most recent // FIXME hard-coded replication policy: most recent
// snapshot as source // snapshot as source
var mostRecentSnap *FilesystemVersion var mostRecentSnap *FilesystemVersion
for n := len(noCommonAncestor.SortedSenderVersions) -1; n >= 0; n-- { for n := len(noCommonAncestor.SortedSenderVersions) - 1; n >= 0; n-- {
if noCommonAncestor.SortedSenderVersions[n].Type == FilesystemVersion_Snapshot { if noCommonAncestor.SortedSenderVersions[n].Type == FilesystemVersion_Snapshot {
mostRecentSnap = noCommonAncestor.SortedSenderVersions[n] mostRecentSnap = noCommonAncestor.SortedSenderVersions[n]
break break
@ -146,7 +261,7 @@ func resolveConflict(conflict error) (path []*FilesystemVersion, msg string) {
if mostRecentSnap == nil { if mostRecentSnap == nil {
return nil, "no snapshots available on sender side" return nil, "no snapshots available on sender side"
} }
return []*FilesystemVersion{mostRecentSnap}, fmt.Sprintf("start replication at most recent snapshot %s", mostRecentSnap) return []*FilesystemVersion{mostRecentSnap}, fmt.Sprintf("start replication at most recent snapshot %s", mostRecentSnap.RelName())
} }
} }
return nil, "no automated way to handle conflict type" return nil, "no automated way to handle conflict type"
@ -160,43 +275,144 @@ func resolveConflict(conflict error) (path []*FilesystemVersion, msg string) {
// If an error occurs when replicating a filesystem, that error is logged to the logger in ctx. // If an error occurs when replicating a filesystem, that error is logged to the logger in ctx.
// Replicate continues with the replication of the remaining file systems. // Replicate continues with the replication of the remaining file systems.
// Depending on the type of error, failed replications are retried in an unspecified order (currently FIFO). // Depending on the type of error, failed replications are retried in an unspecified order (currently FIFO).
func Replicate(ctx context.Context, ep EndpointPair, ipr IncrementalPathReplicator, allTriedOnce chan struct{}) { func Replicate(ctx context.Context, ep EndpointPair) {
log := ctx.Value(ContextKeyLog).(Logger) log := getLogger(ctx)
retryPlanTicker := time.NewTicker(15 * time.Second) // FIXME make configurable
defer retryPlanTicker.Stop()
var (
plan *replicationPlan
res tryRes
)
for {
log.Info("build replication plan")
plan, res = tryBuildReplicationPlan(ctx, ep)
if plan != nil {
break
}
log.WithField("result", res).Error("building replication plan failed, wait for retry timer result")
select {
case <-ctx.Done():
log.WithError(ctx.Err()).Info("aborting replication because context is done")
return
case <-retryPlanTicker.C:
// TODO also accept an external channel that allows us to tick
}
}
retryPlanTicker.Stop()
mainlog := log
plan.executeOldestFirst(ctx, func(fs *Filesystem, from, to *FilesystemVersion) tryRes {
log := mainlog.WithField("filesystem", fs.Path)
// FIXME refresh fs resume token
fs.ResumeToken = ""
var sr *SendReq
if fs.ResumeToken != "" {
sr = &SendReq{
Filesystem: fs.Path,
ResumeToken: fs.ResumeToken,
}
} else if from == nil {
sr = &SendReq{
Filesystem: fs.Path,
From: to.RelName(), // FIXME fix protocol to use To, like zfs does internally
}
} else {
sr = &SendReq{
Filesystem: fs.Path,
From: from.RelName(),
To: to.RelName(),
}
}
log.WithField("request", sr).Debug("initiate send request")
sres, sstream, err := ep.Sender().Send(ctx, sr)
if err != nil {
log.WithError(err).Error("send request failed")
return tryResFromEndpointError(err)
}
if sstream == nil {
log.Error("send request did not return a stream, broken endpoint implementation")
return tryRes{unfixable: true}
}
rr := &ReceiveReq{
Filesystem: fs.Path,
ClearResumeToken: !sres.UsedResumeToken,
}
log.WithField("request", rr).Debug("initiate receive request")
err = ep.Receiver().Receive(ctx, rr, sstream)
if err != nil {
log.WithError(err).Error("receive request failed (might also be error on sender)")
sstream.Close()
// This failure could be due to
// - an unexpected exit of ZFS on the sending side
// - an unexpected exit of ZFS on the receiving side
// - a connectivity issue
return tryResFromEndpointError(err)
}
log.Info("receive finished")
return tryRes{done: true}
})
}
type tryRes struct {
done bool
retry bool
unfixable bool
}
func tryResFromEndpointError(err error) tryRes {
if _, ok := err.(net.Error); ok {
return tryRes{retry: true}
}
return tryRes{unfixable: true}
}
func tryBuildReplicationPlan(ctx context.Context, ep EndpointPair) (*replicationPlan, tryRes) {
log := getLogger(ctx)
early := func(err error) (*replicationPlan, tryRes) {
return nil, tryResFromEndpointError(err)
}
sfss, err := ep.Sender().ListFilesystems(ctx) sfss, err := ep.Sender().ListFilesystems(ctx)
if err != nil { if err != nil {
log.Printf("error listing sender filesystems: %s", err) log.WithError(err).Error("error listing sender filesystems")
return return early(err)
} }
rfss, err := ep.Receiver().ListFilesystems(ctx) rfss, err := ep.Receiver().ListFilesystems(ctx)
if err != nil { if err != nil {
log.Printf("error listing receiver filesystems: %s", err) log.WithError(err).Error("error listing receiver filesystems")
return return early(err)
} }
wq := list.New() plan := newReplicationPlan()
mainlog := log
for _, fs := range sfss { for _, fs := range sfss {
wq.PushBack(&replicationWork{
fs: fs,
})
}
driveFSReplication(ctx, wq, allTriedOnce, log, func(w *replicationWork) FilesystemReplicationResult { log := mainlog.WithField("filesystem", fs.Path)
fs := w.fs
log.Printf("replicating %s", fs.Path) log.Info("assessing filesystem")
sfsvs, err := ep.Sender().ListFilesystemVersions(ctx, fs.Path) sfsvs, err := ep.Sender().ListFilesystemVersions(ctx, fs.Path)
if err != nil { if err != nil {
log.Printf("cannot get remote filesystem versions: %s", err) log.WithError(err).Error("cannot get remote filesystem versions")
return handleGenericEndpointError(err) return early(err)
} }
if len(sfsvs) <= 1 { if len(sfsvs) <= 1 {
log.Printf("sender does not have any versions") log.Error("sender does not have any versions")
return FilesystemReplicationResult{Unfixable: true} return nil, tryRes{unfixable: true}
} }
receiverFSExists := false receiverFSExists := false
@ -211,11 +427,11 @@ func Replicate(ctx context.Context, ep EndpointPair, ipr IncrementalPathReplicat
rfsvs, err = ep.Receiver().ListFilesystemVersions(ctx, fs.Path) rfsvs, err = ep.Receiver().ListFilesystemVersions(ctx, fs.Path)
if err != nil { if err != nil {
if _, ok := err.(FilteredError); ok { if _, ok := err.(FilteredError); ok {
log.Printf("receiver does not map %s", fs.Path) log.Info("receiver ignores filesystem")
return FilesystemReplicationResult{Done: true} continue
} }
log.Printf("receiver error %s", err) log.WithError(err).Error("receiver error")
return handleGenericEndpointError(err) return early(err)
} }
} else { } else {
rfsvs = []*FilesystemVersion{} rfsvs = []*FilesystemVersion{}
@ -223,130 +439,34 @@ func Replicate(ctx context.Context, ep EndpointPair, ipr IncrementalPathReplicat
path, conflict := IncrementalPath(rfsvs, sfsvs) path, conflict := IncrementalPath(rfsvs, sfsvs)
if conflict != nil { if conflict != nil {
log.Printf("conflict: %s", conflict)
var msg string var msg string
path, msg = resolveConflict(conflict) path, msg = resolveConflict(conflict) // no shadowing allowed!
if path != nil { if path != nil {
log.Printf("conflict resolved: %s", msg) log.WithField("conflict", conflict).Info("conflict")
log.WithField("resolution", msg).Info("automatically resolved")
} else { } else {
log.Printf("%s", msg) log.WithField("conflict", conflict).Error("conflict")
log.WithField("problem", msg).Error("cannot resolve conflict")
} }
} }
if path == nil { if path == nil {
return FilesystemReplicationResult{Unfixable: true} plan.addWork(newReplicateFSWorkWithConflict(fs, conflict))
continue
} }
return ipr.Replicate(ctx, ep.Sender(), ep.Receiver(), NewCopier(), fs, path) w := newReplicateFSWork(fs)
if len(path) == 1 {
step := newReplicationStep(nil, path[0])
w.AddStep(step)
} else {
for i := 0; i < len(path)-1; i++ {
step := newReplicationStep(path[i], path[i+1])
w.AddStep(step)
}
}
plan.addWork(w)
})
}
type Sender interface {
Send(ctx context.Context, r *SendReq) (*SendRes, io.ReadCloser, error)
}
type Receiver interface {
Receive(ctx context.Context, r *ReceiveReq, sendStream io.ReadCloser) (error)
}
type Copier interface {
Copy(writer io.Writer, reader io.Reader) (int64, error)
}
type copier struct{}
func (copier) Copy(writer io.Writer, reader io.Reader) (int64, error) {
return io.Copy(writer, reader)
}
func NewCopier() Copier {
return copier{}
}
type IncrementalPathReplicator interface {
Replicate(ctx context.Context, sender Sender, receiver Receiver, copier Copier, fs *Filesystem, path []*FilesystemVersion) FilesystemReplicationResult
}
type incrementalPathReplicator struct{}
func NewIncrementalPathReplicator() IncrementalPathReplicator {
return incrementalPathReplicator{}
}
func (incrementalPathReplicator) Replicate(ctx context.Context, sender Sender, receiver Receiver, copier Copier, fs *Filesystem, path []*FilesystemVersion) FilesystemReplicationResult {
log := ctx.Value(ContextKeyLog).(Logger)
if len(path) == 0 {
log.Printf("nothing to do")
return FilesystemReplicationResult{Done: true}
} }
if len(path) == 1 { return plan, tryRes{done: true}
log.Printf("full send of version %s", path[0])
sr := &SendReq{
Filesystem: fs.Path,
From: path[0].RelName(),
ResumeToken: fs.ResumeToken,
}
sres, sstream, err := sender.Send(ctx, sr)
if err != nil {
log.Printf("send request failed: %s", err)
return handleGenericEndpointError(err)
}
rr := &ReceiveReq{
Filesystem: fs.Path,
ClearResumeToken: fs.ResumeToken != "" && !sres.UsedResumeToken,
}
err = receiver.Receive(ctx, rr, sstream)
if err != nil {
log.Printf("receive request failed (might also be error on sender): %s", err)
sstream.Close()
// This failure could be due to
// - an unexpected exit of ZFS on the sending side
// - an unexpected exit of ZFS on the receiving side
// - a connectivity issue
return handleGenericEndpointError(err)
}
return FilesystemReplicationResult{Done: true}
}
usedResumeToken := false
for j := 0; j < len(path)-1; j++ {
rt := ""
if !usedResumeToken { // only send resume token for first increment
rt = fs.ResumeToken
usedResumeToken = true
}
sr := &SendReq{
Filesystem: fs.Path,
From: path[j].RelName(),
To: path[j+1].RelName(),
ResumeToken: rt,
}
sres, sstream, err := sender.Send(ctx, sr)
if err != nil {
log.Printf("send request failed: %s", err)
return handleGenericEndpointError(err)
}
// try to consume stream
rr := &ReceiveReq{
Filesystem: fs.Path,
ClearResumeToken: rt != "" && !sres.UsedResumeToken,
}
err = receiver.Receive(ctx, rr, sstream)
if err != nil {
log.Printf("receive request failed: %s", err)
return handleGenericEndpointError(err) // FIXME resume state on receiver -> update ResumeToken
}
// FIXME handle properties from sres
}
return FilesystemReplicationResult{Done: true}
} }

View File

@ -51,131 +51,131 @@ func (m *MockIncrementalPathRecorder) Finished() bool {
return m.Pos == len(m.Sequence) return m.Pos == len(m.Sequence)
} }
type DiscardCopier struct{} //type IncrementalPathReplicatorTest struct {
// Msg string
func (DiscardCopier) Copy(writer io.Writer, reader io.Reader) (int64, error) { // Filesystem *replication.Filesystem
return 0, nil // Path []*replication.FilesystemVersion
} // Steps []IncrementalPathSequenceStep
//}
type IncrementalPathReplicatorTest struct { //
Msg string //func (test *IncrementalPathReplicatorTest) Test(t *testing.T) {
Filesystem *replication.Filesystem //
Path []*replication.FilesystemVersion // t.Log(test.Msg)
Steps []IncrementalPathSequenceStep //
} // rec := &MockIncrementalPathRecorder{
// T: t,
func (test *IncrementalPathReplicatorTest) Test(t *testing.T) { // Sequence: test.Steps,
// }
t.Log(test.Msg) //
// ctx := replication.ContextWithLogger(context.Background(), testLog{t})
rec := &MockIncrementalPathRecorder{ //
T: t, // ipr := replication.NewIncrementalPathReplicator()
Sequence: test.Steps, // ipr.Replicate(
} // ctx,
// rec,
ctx := context.WithValue(context.Background(), replication.ContextKeyLog, testLog{t}) // rec,
// DiscardCopier{},
ipr := replication.NewIncrementalPathReplicator() // test.Filesystem,
ipr.Replicate( // test.Path,
ctx, // )
rec, //
rec, // assert.True(t, rec.Finished())
DiscardCopier{}, //
test.Filesystem, //}
test.Path,
)
assert.True(t, rec.Finished())
}
type testLog struct { type testLog struct {
t *testing.T t *testing.T
} }
func (t testLog) Printf(fmt string, args ...interface{}) { var _ replication.Logger = testLog{}
func (t testLog) Infof(fmt string, args ...interface{}) {
t.t.Logf(fmt, args)
}
func (t testLog) Errorf(fmt string, args ...interface{}) {
t.t.Logf(fmt, args) t.t.Logf(fmt, args)
} }
func TestIncrementalPathReplicator_Replicate(t *testing.T) {
tbl := []IncrementalPathReplicatorTest{ //func TestIncrementalPathReplicator_Replicate(t *testing.T) {
{ //
Msg: "generic happy place with resume token", // tbl := []IncrementalPathReplicatorTest{
Filesystem: &replication.Filesystem{ // {
Path: "foo/bar", // Msg: "generic happy place with resume token",
ResumeToken: "blafoo", // Filesystem: &replication.Filesystem{
}, // Path: "foo/bar",
Path: fsvlist("@a,1", "@b,2", "@c,3"), // ResumeToken: "blafoo",
Steps: []IncrementalPathSequenceStep{ // },
{ // Path: fsvlist("@a,1", "@b,2", "@c,3"),
SendRequest: &replication.SendReq{ // Steps: []IncrementalPathSequenceStep{
Filesystem: "foo/bar", // {
From: "@a,1", // SendRequest: &replication.SendReq{
To: "@b,2", // Filesystem: "foo/bar",
ResumeToken: "blafoo", // From: "@a,1",
}, // To: "@b,2",
SendResponse: &replication.SendRes{ // ResumeToken: "blafoo",
UsedResumeToken: true, // },
}, // SendResponse: &replication.SendRes{
}, // UsedResumeToken: true,
{ // },
ReceiveRequest: &replication.ReceiveReq{ // },
Filesystem: "foo/bar", // {
ClearResumeToken: false, // ReceiveRequest: &replication.ReceiveReq{
}, // Filesystem: "foo/bar",
}, // ClearResumeToken: false,
{ // },
SendRequest: &replication.SendReq{ // },
Filesystem: "foo/bar", // {
From: "@b,2", // SendRequest: &replication.SendReq{
To: "@c,3", // Filesystem: "foo/bar",
}, // From: "@b,2",
}, // To: "@c,3",
{ // },
ReceiveRequest: &replication.ReceiveReq{ // },
Filesystem: "foo/bar", // {
}, // ReceiveRequest: &replication.ReceiveReq{
}, // Filesystem: "foo/bar",
}, // },
}, // },
{ // },
Msg: "no action on empty sequence", // },
Filesystem: &replication.Filesystem{ // {
Path: "foo/bar", // Msg: "no action on empty sequence",
}, // Filesystem: &replication.Filesystem{
Path: fsvlist(), // Path: "foo/bar",
Steps: []IncrementalPathSequenceStep{}, // },
}, // Path: fsvlist(),
{ // Steps: []IncrementalPathSequenceStep{},
Msg: "full send on single entry path", // },
Filesystem: &replication.Filesystem{ // {
Path: "foo/bar", // Msg: "full send on single entry path",
}, // Filesystem: &replication.Filesystem{
Path: fsvlist("@justone,1"), // Path: "foo/bar",
Steps: []IncrementalPathSequenceStep{ // },
{ // Path: fsvlist("@justone,1"),
SendRequest: &replication.SendReq{ // Steps: []IncrementalPathSequenceStep{
Filesystem: "foo/bar", // {
From: "@justone,1", // SendRequest: &replication.SendReq{
To: "", // empty means full send // Filesystem: "foo/bar",
}, // From: "@justone,1",
SendResponse: &replication.SendRes{ // To: "", // empty means full send
UsedResumeToken: false, // },
}, // SendResponse: &replication.SendRes{
}, // UsedResumeToken: false,
{ // },
ReceiveRequest: &replication.ReceiveReq{ // },
Filesystem: "foo/bar", // {
ClearResumeToken: false, // ReceiveRequest: &replication.ReceiveReq{
}, // Filesystem: "foo/bar",
}, // ClearResumeToken: false,
}, // },
}, // },
} // },
// },
for _, test := range tbl { // }
test.Test(t) //
} // for _, test := range tbl {
// test.Test(t)
} // }
//
//}

View File

@ -14,7 +14,20 @@ const (
const DefaultUserFieldCapacity = 5 const DefaultUserFieldCapacity = 5
type Logger struct { type Logger interface {
WithOutlet(outlet Outlet, level Level) Logger
ReplaceField(field string, val interface{}) Logger
WithField(field string, val interface{}) Logger
WithFields(fields Fields) Logger
WithError(err error) Logger
Debug(msg string)
Info(msg string)
Warn(msg string)
Error(msg string)
Printf(format string, args ...interface{})
}
type loggerImpl struct {
fields Fields fields Fields
outlets *Outlets outlets *Outlets
outletTimeout time.Duration outletTimeout time.Duration
@ -22,8 +35,10 @@ type Logger struct {
mtx *sync.Mutex mtx *sync.Mutex
} }
func NewLogger(outlets *Outlets, outletTimeout time.Duration) *Logger { var _ Logger = &loggerImpl{}
return &Logger{
func NewLogger(outlets *Outlets, outletTimeout time.Duration) Logger {
return &loggerImpl{
make(Fields, DefaultUserFieldCapacity), make(Fields, DefaultUserFieldCapacity),
outlets, outlets,
outletTimeout, outletTimeout,
@ -36,7 +51,7 @@ type outletResult struct {
Error error Error error
} }
func (l *Logger) logInternalError(outlet Outlet, err string) { func (l *loggerImpl) logInternalError(outlet Outlet, err string) {
fields := Fields{} fields := Fields{}
if outlet != nil { if outlet != nil {
if _, ok := outlet.(fmt.Stringer); ok { if _, ok := outlet.(fmt.Stringer); ok {
@ -54,7 +69,7 @@ func (l *Logger) logInternalError(outlet Outlet, err string) {
l.outlets.GetLoggerErrorOutlet().WriteEntry(entry) l.outlets.GetLoggerErrorOutlet().WriteEntry(entry)
} }
func (l *Logger) log(level Level, msg string) { func (l *loggerImpl) log(level Level, msg string) {
l.mtx.Lock() l.mtx.Lock()
defer l.mtx.Unlock() defer l.mtx.Unlock()
@ -78,12 +93,12 @@ func (l *Logger) log(level Level, msg string) {
} }
func (l *Logger) WithOutlet(outlet Outlet, level Level) *Logger { func (l *loggerImpl) WithOutlet(outlet Outlet, level Level) Logger {
l.mtx.Lock() l.mtx.Lock()
defer l.mtx.Unlock() defer l.mtx.Unlock()
newOutlets := l.outlets.DeepCopy() newOutlets := l.outlets.DeepCopy()
newOutlets.Add(outlet, level) newOutlets.Add(outlet, level)
child := &Logger{ child := &loggerImpl{
fields: l.fields, fields: l.fields,
outlets: newOutlets, outlets: newOutlets,
outletTimeout: l.outletTimeout, outletTimeout: l.outletTimeout,
@ -93,9 +108,9 @@ func (l *Logger) WithOutlet(outlet Outlet, level Level) *Logger {
} }
// callers must hold l.mtx // callers must hold l.mtx
func (l *Logger) forkLogger(field string, val interface{}) *Logger { func (l *loggerImpl) forkLogger(field string, val interface{}) *loggerImpl {
child := &Logger{ child := &loggerImpl{
fields: make(Fields, len(l.fields)+1), fields: make(Fields, len(l.fields)+1),
outlets: l.outlets, outlets: l.outlets,
outletTimeout: l.outletTimeout, outletTimeout: l.outletTimeout,
@ -109,13 +124,13 @@ func (l *Logger) forkLogger(field string, val interface{}) *Logger {
return child return child
} }
func (l *Logger) ReplaceField(field string, val interface{}) *Logger { func (l *loggerImpl) ReplaceField(field string, val interface{}) Logger {
l.mtx.Lock() l.mtx.Lock()
defer l.mtx.Unlock() defer l.mtx.Unlock()
return l.forkLogger(field, val) return l.forkLogger(field, val)
} }
func (l *Logger) WithField(field string, val interface{}) *Logger { func (l *loggerImpl) WithField(field string, val interface{}) Logger {
l.mtx.Lock() l.mtx.Lock()
defer l.mtx.Unlock() defer l.mtx.Unlock()
if val, ok := l.fields[field]; ok && val != nil { if val, ok := l.fields[field]; ok && val != nil {
@ -125,16 +140,16 @@ func (l *Logger) WithField(field string, val interface{}) *Logger {
return l.forkLogger(field, val) return l.forkLogger(field, val)
} }
func (l *Logger) WithFields(fields Fields) (ret *Logger) { func (l *loggerImpl) WithFields(fields Fields) Logger {
// TODO optimize // TODO optimize
ret = l var ret Logger = l
for field, value := range fields { for field, value := range fields {
ret = ret.WithField(field, value) ret = ret.WithField(field, value)
} }
return ret return ret
} }
func (l *Logger) WithError(err error) *Logger { func (l *loggerImpl) WithError(err error) Logger {
val := interface{}(nil) val := interface{}(nil)
if err != nil { if err != nil {
val = err.Error() val = err.Error()
@ -142,22 +157,22 @@ func (l *Logger) WithError(err error) *Logger {
return l.WithField(FieldError, val) return l.WithField(FieldError, val)
} }
func (l *Logger) Debug(msg string) { func (l *loggerImpl) Debug(msg string) {
l.log(Debug, msg) l.log(Debug, msg)
} }
func (l *Logger) Info(msg string) { func (l *loggerImpl) Info(msg string) {
l.log(Info, msg) l.log(Info, msg)
} }
func (l *Logger) Warn(msg string) { func (l *loggerImpl) Warn(msg string) {
l.log(Warn, msg) l.log(Warn, msg)
} }
func (l *Logger) Error(msg string) { func (l *loggerImpl) Error(msg string) {
l.log(Error, msg) l.log(Error, msg)
} }
func (l *Logger) Printf(format string, args ...interface{}) { func (l *loggerImpl) Printf(format string, args ...interface{}) {
l.log(Error, fmt.Sprintf(format, args...)) l.log(Error, fmt.Sprintf(format, args...))
} }

22
logger/nulllogger.go Normal file
View File

@ -0,0 +1,22 @@
package logger
type nullLogger struct {}
var _ Logger = nullLogger{}
func NewNullLogger() Logger {
return nullLogger{}
}
func (n nullLogger) WithOutlet(outlet Outlet, level Level) Logger { return n }
func (n nullLogger) ReplaceField(field string, val interface{}) Logger { return n }
func (n nullLogger) WithField(field string, val interface{}) Logger { return n }
func (n nullLogger) WithFields(fields Fields) Logger { return n }
func (n nullLogger) WithError(err error) Logger { return n }
func (nullLogger) Debug(msg string) {}
func (nullLogger) Info(msg string) {}
func (nullLogger) Warn(msg string) {}
func (nullLogger) Error(msg string) {}
func (nullLogger) Printf(format string, args ...interface{}) {}