Commit e0fdc0e7 authored by Kirill Smelkov's avatar Kirill Smelkov

.

parent 3a76054d
...@@ -17,17 +17,12 @@ ...@@ -17,17 +17,12 @@
// See COPYING file for full licensing terms. // See COPYING file for full licensing terms.
// See https://www.nexedi.com/licensing for rationale and options. // See https://www.nexedi.com/licensing for rationale and options.
// Package xcontext provides addons to std package context. // Package xcontext is staging place for go123/xcontext.
//
// XXX docs:
// - Canceled
// - WhenDone
package xcontext package xcontext
import ( import (
"context" "context"
"errors" "errors"
"io"
) )
// Cancelled reports whether an error is due to a canceled context. // Cancelled reports whether an error is due to a canceled context.
...@@ -51,6 +46,7 @@ func Canceled(err error) bool { ...@@ -51,6 +46,7 @@ func Canceled(err error) bool {
} }
/*
// WhenDone arranges for f to be called either when ctx is cancelled or // WhenDone arranges for f to be called either when ctx is cancelled or
// surrounding function returns. // surrounding function returns.
// //
...@@ -59,7 +55,7 @@ func Canceled(err error) bool { ...@@ -59,7 +55,7 @@ func Canceled(err error) bool {
// func myfunc(ctx, ...) { // func myfunc(ctx, ...) {
// defer xcontext.WhenDone(ctx, func() { ... })() // defer xcontext.WhenDone(ctx, func() { ... })()
// //
// XXX -> use WithCloseOnErrCancel instead? // XXX -> use WithCloseOn{Err,Ret}Cancel instead?
func WhenDone(ctx context.Context, f func()) func() { func WhenDone(ctx context.Context, f func()) func() {
done := make(chan struct{}) done := make(chan struct{})
go func() { go func() {
...@@ -77,41 +73,4 @@ func WhenDone(ctx context.Context, f func()) func() { ...@@ -77,41 +73,4 @@ func WhenDone(ctx context.Context, f func()) func() {
close(done) close(done)
} }
} }
*/
// WithCloseOnErrCancel closes c on ctx cancel while f is run, or if f returns with an error.
//
// It is usually handy to propagate cancellation to interrupt IO. XXX when f creates/leaves link alive.
// XXX naming?
// XXX don't close on f return?
func WithCloseOnErrCancel(ctx context.Context, c io.Closer, f func() error) (err error) {
closed := false
fdone := make(chan error)
defer func() {
<-fdone // wait for f to complete
if err != nil {
if !closed {
c.Close() // XXX log err?
}
}
}()
go func() (err error) {
defer func() {
fdone <- err
close(fdone)
}()
return f()
}()
select {
case <-ctx.Done():
c.Close() // interrupt IO XXX log err?
closed = true
return ctx.Err()
case err := <-fdone:
return err
}
}
...@@ -17,15 +17,16 @@ ...@@ -17,15 +17,16 @@
// See COPYING file for full licensing terms. // See COPYING file for full licensing terms.
// See https://www.nexedi.com/licensing for rationale and options. // See https://www.nexedi.com/licensing for rationale and options.
// Package xio provides addons to standard package io. // Package xio is staging place for go123/xio.
package xio package xio
import ( import (
"context" "context"
"errors"
"io" "io"
"lab.nexedi.com/kirr/neo/go/internal/log" "lab.nexedi.com/kirr/neo/go/internal/log"
"lab.nexedi.com/kirr/neo/go/internal/xcontext" // "lab.nexedi.com/kirr/neo/go/internal/xcontext"
) )
...@@ -45,6 +46,7 @@ func EOFok(err error) error { ...@@ -45,6 +46,7 @@ func EOFok(err error) error {
return err return err
} }
/*
// CloseWhenDone arranges for c to be closed either when ctx is cancelled or // CloseWhenDone arranges for c to be closed either when ctx is cancelled or
// surrounding function returns. // surrounding function returns.
// //
...@@ -64,6 +66,7 @@ func CloseWhenDone(ctx context.Context, c io.Closer) func() { ...@@ -64,6 +66,7 @@ func CloseWhenDone(ctx context.Context, c io.Closer) func() {
} }
}) })
} }
*/
// LClose closes c and logs closing error if there was any. // LClose closes c and logs closing error if there was any.
// the error is otherwise ignored // the error is otherwise ignored
...@@ -75,3 +78,65 @@ func LClose(ctx context.Context, c io.Closer) { ...@@ -75,3 +78,65 @@ func LClose(ctx context.Context, c io.Closer) {
log.Error(ctx, err) log.Error(ctx, err)
} }
} }
// WithCloseOnErrCancel closes c on ctx cancel while f is run, or if f returns with an error.
//
// It is usually handy to propagate cancellation to interrupt IO.
func WithCloseOnErrCancel(ctx context.Context, c io.Closer, f func() error) (err error) {
closed := false
fdone := make(chan error)
defer func() {
errf, ok := <-fdone // wait for f to complete XXX return f's error
if ok {
// it was ctx cancel and `return ctx.Err()` vvv
// -> change return to be what f returned
err = errf
}
if err != nil {
if !closed {
LClose(ctx, c)
}
}
}()
go func() (err error) {
defer func() {
fdone <- err
close(fdone)
}()
return f()
}()
select {
case <-ctx.Done():
LClose(ctx, c) // interrupt IO
closed = true
return ctx.Err()
case err := <-fdone:
return err
}
}
// WithCloseOnRetCancel closes c on ctx cancel while f is run, or when f returns.
//
// It is usually handy to propagate cancellation to interrupt IO.
func WithCloseOnRetCancel(ctx context.Context, c io.Closer, f func() error) error {
var errf error
err := WithCloseOnErrCancel(ctx, c, func() error {
errf = f()
e := errf
if e == nil {
e = retOK // force c close
}
return e
})
if err == retOK {
err = errf
}
return err
}
var retOK = errors.New("ok")
...@@ -32,6 +32,7 @@ import ( ...@@ -32,6 +32,7 @@ import (
"github.com/golang/glog" "github.com/golang/glog"
"github.com/pkg/errors" "github.com/pkg/errors"
"lab.nexedi.com/kirr/go123/mem" "lab.nexedi.com/kirr/go123/mem"
"lab.nexedi.com/kirr/go123/xerr"
"lab.nexedi.com/kirr/go123/xcontext" "lab.nexedi.com/kirr/go123/xcontext"
"lab.nexedi.com/kirr/go123/xnet" "lab.nexedi.com/kirr/go123/xnet"
"lab.nexedi.com/kirr/go123/xsync" "lab.nexedi.com/kirr/go123/xsync"
...@@ -98,10 +99,8 @@ func (c *Client) Close() (err error) { ...@@ -98,10 +99,8 @@ func (c *Client) Close() (err error) {
// close networker if configured to do so // close networker if configured to do so
if c.ownNet { if c.ownNet {
err2 := c.node.Net.Close() __ := c.node.Net.Close()
if err == nil { err = xerr.First(err, __)
err = err2
}
} }
return err return err
} }
......
...@@ -30,6 +30,7 @@ import ( ...@@ -30,6 +30,7 @@ import (
"github.com/soheilhy/cmux" "github.com/soheilhy/cmux"
"lab.nexedi.com/kirr/go123/xerr"
"lab.nexedi.com/kirr/go123/xnet" "lab.nexedi.com/kirr/go123/xnet"
"lab.nexedi.com/kirr/go123/xsync" "lab.nexedi.com/kirr/go123/xsync"
"lab.nexedi.com/kirr/neo/go/internal/log" "lab.nexedi.com/kirr/neo/go/internal/log"
...@@ -90,27 +91,30 @@ func neoMatch(r io.Reader) bool { ...@@ -90,27 +91,30 @@ func neoMatch(r io.Reader) bool {
// default HTTP mux. // default HTTP mux.
// //
// Default HTTP mux can be assumed to contain /debug/pprof and the like. // Default HTTP mux can be assumed to contain /debug/pprof and the like.
func listenAndServe(ctx context.Context, net xnet.Networker, laddr string, serve func(ctx context.Context, l xnet.Listener) error) error { func listenAndServe(ctx context.Context, net xnet.Networker, laddr string, serve func(ctx context.Context, l xnet.Listener) error) (err error) {
l, err := net.Listen(ctx, laddr) l, err := net.Listen(ctx, laddr)
if err != nil { if err != nil {
return err return err
} }
defer func() { // just in case if mux.Close does not close l
// XXX who closes l? __ := l.Close()
err = xerr.First(err, __)
}()
log.Infof(ctx, "listening at %s ...", l.Addr()) log.Infof(ctx, "listening at %s ...", l.Addr())
log.Flush() // XXX ok? log.Flush()
mux := cmux.New(xnet.BindCtxL(l, ctx)) mux := cmux.New(xnet.BindCtxL(l, ctx))
neoL := mux.Match(neoMatch) neoL := mux.Match(neoMatch)
httpL := mux.Match(cmux.HTTP1(), cmux.HTTP2()) // XXX verify http2 works httpL := mux.Match(cmux.HTTP1(), cmux.HTTP2())
miscL := mux.Match(cmux.Any()) miscL := mux.Match(cmux.Any())
wg := xsync.NewWorkGroup(ctx) wg := xsync.NewWorkGroup(ctx)
wg.Go(func(ctx context.Context) error { wg.Go(func(ctx context.Context) error {
// XXX shutdown serve on ctx cancel (-> WithCloseOnErrCancel(mux); cmux recently added mux.Close) return xio.WithCloseOnRetCancel(ctx, &noErrCloser{mux},
return mux.Serve() mux.Serve,
)
}) })
wg.Go(func(ctx context.Context) error { wg.Go(func(ctx context.Context) error {
...@@ -118,35 +122,47 @@ func listenAndServe(ctx context.Context, net xnet.Networker, laddr string, serve ...@@ -118,35 +122,47 @@ func listenAndServe(ctx context.Context, net xnet.Networker, laddr string, serve
}) })
wg.Go(func(ctx context.Context) error { wg.Go(func(ctx context.Context) error {
// XXX shutdown http on ctx cancel srv := &http.Server{}
return http.Serve(httpL, nil) return xio.WithCloseOnRetCancel(ctx, srv, func() error {
// TODO ^^^ better Shutdown instead of Close
return srv.Serve(httpL)
})
}) })
wg.Go(func(ctx context.Context) error { wg.Go(func(ctx context.Context) error {
// XXX shutdown on ctx cancel return xio.WithCloseOnRetCancel(ctx, miscL, func() error {
for { for {
conn, err := miscL.Accept() conn, err := miscL.Accept()
if err != nil { if err != nil {
return err return err
}
// got something unexpected - grab the header (which we
// already have read), log it and reject the connection.
b := make([]byte, 1024)
// must not block as some data is already there in cmux buffer
n, _ := conn.Read(b)
subj := fmt.Sprintf("strange connection from %s:", conn.RemoteAddr())
serr := "peer sent nothing"
if n > 0 {
serr = fmt.Sprintf("peer sent %q", b[:n])
}
log.Warningf(ctx, "%s: %s", subj, serr)
xio.LClose(ctx, conn)
} }
})
// got something unexpected - grab the header (which we
// already have read), log it and reject the connection.
b := make([]byte, 1024)
// must not block as some data is already there in cmux buffer
n, _ := conn.Read(b)
subj := fmt.Sprintf("strange connection from %s:", conn.RemoteAddr())
serr := "peer sent nothing"
if n > 0 {
serr = fmt.Sprintf("peer sent %q", b[:n])
}
log.Infof(ctx, "%s: %s", subj, serr)
xio.LClose(ctx, conn)
}
}) })
return wg.Wait()
}
// noErrCloser turns `Close()` -> `Close() err` that alwasys returns nil.
type noErrCloser struct {
c interface { Close() }
}
err = wg.Wait() func (c *noErrCloser) Close() error {
return err c.c.Close()
return nil
} }
...@@ -54,6 +54,7 @@ import ( ...@@ -54,6 +54,7 @@ import (
"time" "time"
"lab.nexedi.com/kirr/go123/xcontext" "lab.nexedi.com/kirr/go123/xcontext"
"lab.nexedi.com/kirr/go123/xerr"
"lab.nexedi.com/kirr/go123/xnet" "lab.nexedi.com/kirr/go123/xnet"
"lab.nexedi.com/kirr/go123/xsync" "lab.nexedi.com/kirr/go123/xsync"
...@@ -232,6 +233,10 @@ func (m *Master) Run(ctx context.Context, l xnet.Listener) (err error) { ...@@ -232,6 +233,10 @@ func (m *Master) Run(ctx context.Context, l xnet.Listener) (err error) {
// wrap listener with link / identification hello checker // wrap listener with link / identification hello checker
lli := xneo.NewListener(neonet.NewLinkListener(l)) lli := xneo.NewListener(neonet.NewLinkListener(l))
defer func() {
__ := lli.Close()
err = xerr.First(err, __)
}()
// accept: accept incoming connections and pass them to main driver // accept: accept incoming connections and pass them to main driver
m.mainWG.Go(func(ctx context.Context) (err error) { m.mainWG.Go(func(ctx context.Context) (err error) {
...@@ -286,8 +291,6 @@ func (m *Master) Run(ctx context.Context, l xnet.Listener) (err error) { ...@@ -286,8 +291,6 @@ func (m *Master) Run(ctx context.Context, l xnet.Listener) (err error) {
err = ctx.Err() err = ctx.Err()
} }
xio.LClose(ctx, lli) // XXX here ok? (probably not)
return err return err
} }
...@@ -1007,22 +1010,22 @@ func (m *Master) serveClient(ctx context.Context, cli *_MasteredPeer) (err error ...@@ -1007,22 +1010,22 @@ func (m *Master) serveClient(ctx context.Context, cli *_MasteredPeer) (err error
defer task.Runningf(&ctx, "%s: serve client", cli.node.NID)(&err) defer task.Runningf(&ctx, "%s: serve client", cli.node.NID)(&err)
clink := cli.node.Link() clink := cli.node.Link()
defer xio.CloseWhenDone(ctx, clink)()
// M <- C requests handler // M <- C requests handler
for { return xio.WithCloseOnRetCancel(ctx, clink, func() error {
req, err := clink.Recv1() for {
if err != nil { req, err := clink.Recv1()
return err if err != nil {
} return err
}
resp := m.serveClient1(ctx, req.Msg) resp := m.serveClient1(ctx, req.Msg)
err = req.Reply(resp) err = req.Reply(resp)
req.Close() req.Close()
if err != nil { if err != nil {
return err return err
}
} }
} })
} }
// serveClient1 prepares response for 1 request from client. // serveClient1 prepares response for 1 request from client.
...@@ -1121,9 +1124,8 @@ func (m *Master) identify(ctx context.Context, n nodeCome) (peer *_MasteredPeer, ...@@ -1121,9 +1124,8 @@ func (m *Master) identify(ctx context.Context, n nodeCome) (peer *_MasteredPeer,
if err != nil { if err != nil {
log.Infof(ctx, "%s: rejecting: %s", subj, err) log.Infof(ctx, "%s: rejecting: %s", subj, err)
m.mainWG.Go(func(ctx context.Context) error { m.mainWG.Go(func(ctx context.Context) error {
xxcontext.WithCloseOnErrCancel(ctx, link, func() error { xio.WithCloseOnRetCancel(ctx, link, func() error {
n.req.Reply(err) return n.req.Reply(err)
return fmt.Errorf("X") // to close link
}) })
return nil // not to cancel main by a failing reject return nil // not to cancel main by a failing reject
}) })
...@@ -1200,7 +1202,7 @@ func (m *Master) identify(ctx context.Context, n nodeCome) (peer *_MasteredPeer, ...@@ -1200,7 +1202,7 @@ func (m *Master) identify(ctx context.Context, n nodeCome) (peer *_MasteredPeer,
}) })
// XXX compensate a bit for lack of ctx handling in Send/Recv // XXX compensate a bit for lack of ctx handling in Send/Recv
return xxcontext.WithCloseOnErrCancel(ctx, link, func() error { return xio.WithCloseOnErrCancel(ctx, link, func() error {
// send accept and indicate to run that initial acceptance is done // send accept and indicate to run that initial acceptance is done
err := peer.accept(ctx) err := peer.accept(ctx)
......
...@@ -31,7 +31,7 @@ import ( ...@@ -31,7 +31,7 @@ import (
"lab.nexedi.com/kirr/neo/go/internal/log" "lab.nexedi.com/kirr/neo/go/internal/log"
"lab.nexedi.com/kirr/neo/go/internal/task" "lab.nexedi.com/kirr/neo/go/internal/task"
"lab.nexedi.com/kirr/neo/go/internal/xcontext" "lab.nexedi.com/kirr/neo/go/internal/xio"
"lab.nexedi.com/kirr/neo/go/neo/neonet" "lab.nexedi.com/kirr/neo/go/neo/neonet"
"lab.nexedi.com/kirr/neo/go/neo/proto" "lab.nexedi.com/kirr/neo/go/neo/proto"
"lab.nexedi.com/kirr/neo/go/neo/xneo" "lab.nexedi.com/kirr/neo/go/neo/xneo"
...@@ -149,8 +149,7 @@ func (node *_MasteredNode) talkMaster1(ctx, ctxPreTalkM context.Context, f func( ...@@ -149,8 +149,7 @@ func (node *_MasteredNode) talkMaster1(ctx, ctxPreTalkM context.Context, f func(
return err return err
} }
// XXX close on RetCancel return xio.WithCloseOnRetCancel(ctx, mlink, func() (err error) {
return xcontext.WithCloseOnErrCancel(ctx, mlink, func() (err error) {
if accept.YourNID != node.MyInfo.NID { if accept.YourNID != node.MyInfo.NID {
log.Infof(ctx, "master %s told us to be %s", accept.MyNID, accept.YourNID) log.Infof(ctx, "master %s told us to be %s", accept.MyNID, accept.YourNID)
node.MyInfo.NID = accept.YourNID // XXX locking ? -> opMu ? node.MyInfo.NID = accept.YourNID // XXX locking ? -> opMu ?
......
...@@ -32,7 +32,6 @@ import ( ...@@ -32,7 +32,6 @@ import (
"lab.nexedi.com/kirr/go123/xerr" "lab.nexedi.com/kirr/go123/xerr"
"lab.nexedi.com/kirr/go123/xnet" "lab.nexedi.com/kirr/go123/xnet"
"lab.nexedi.com/kirr/neo/go/internal/xcontext"
"lab.nexedi.com/kirr/neo/go/internal/xio" "lab.nexedi.com/kirr/neo/go/internal/xio"
"lab.nexedi.com/kirr/neo/go/neo/internal/tneonet" "lab.nexedi.com/kirr/neo/go/neo/internal/tneonet"
"lab.nexedi.com/kirr/neo/go/neo/proto" "lab.nexedi.com/kirr/neo/go/neo/proto"
...@@ -114,7 +113,7 @@ func _handshakeClient(ctx context.Context, conn net.Conn, version uint32, encPre ...@@ -114,7 +113,7 @@ func _handshakeClient(ctx context.Context, conn net.Conn, version uint32, encPre
rxbuf = newXBufReader(conn, /*any non-small limit*/1024) rxbuf = newXBufReader(conn, /*any non-small limit*/1024)
var peerEnc proto.Encoding var peerEnc proto.Encoding
err = xcontext.WithCloseOnErrCancel(ctx, conn, func() error { err = xio.WithCloseOnErrCancel(ctx, conn, func() error {
// tx client hello // tx client hello
err := txHello("tx hello", conn, version, encPrefer) err := txHello("tx hello", conn, version, encPrefer)
if err != nil { if err != nil {
...@@ -154,7 +153,7 @@ func _handshakeServer(ctx context.Context, conn net.Conn, version uint32) (enc p ...@@ -154,7 +153,7 @@ func _handshakeServer(ctx context.Context, conn net.Conn, version uint32) (enc p
rxbuf = newXBufReader(conn, /*any non-small limit*/1024) rxbuf = newXBufReader(conn, /*any non-small limit*/1024)
var peerEnc proto.Encoding var peerEnc proto.Encoding
err = xcontext.WithCloseOnErrCancel(ctx, conn, func() error { err = xio.WithCloseOnErrCancel(ctx, conn, func() error {
// rx client hello // rx client hello
var peerVer uint32 var peerVer uint32
var err error var err error
......
...@@ -87,9 +87,18 @@ func (stor *Storage) Run(ctx context.Context, l xnet.Listener) (err error) { ...@@ -87,9 +87,18 @@ func (stor *Storage) Run(ctx context.Context, l xnet.Listener) (err error) {
// wrap listener with link / identificaton hello checker // wrap listener with link / identificaton hello checker
stor.lli = xneo.NewListener(neonet.NewLinkListener(l)) stor.lli = xneo.NewListener(neonet.NewLinkListener(l))
defer func() {
__ := stor.lli.Close()
err = xerr.First(err, __)
}()
defer func() {
__ := stor.back.Close()
err = xerr.First(err, __)
}()
// connect to master and let it drive us via commands and updates // connect to master and let it drive us via commands and updates
err = stor.node.TalkMaster(ctx, func(ctx context.Context, mlink *_MasterLink) error { return stor.node.TalkMaster(ctx, func(ctx context.Context, mlink *_MasterLink) error {
// XXX move -> SetNumReplicas handler // XXX move -> SetNumReplicas handler
// // NumReplicas: neo/py meaning for n(replica) = `n(real-replica) - 1` // // NumReplicas: neo/py meaning for n(replica) = `n(real-replica) - 1`
// if !(accept.NumPartitions == 1 && accept.NumReplicas == 0) { // if !(accept.NumPartitions == 1 && accept.NumReplicas == 0) {
...@@ -105,15 +114,6 @@ func (stor *Storage) Run(ctx context.Context, l xnet.Listener) (err error) { ...@@ -105,15 +114,6 @@ func (stor *Storage) Run(ctx context.Context, l xnet.Listener) (err error) {
// we got StartOperation command. Let master drive us during service phase. // we got StartOperation command. Let master drive us during service phase.
return stor.m1serve(ctx, mlink, reqStart) return stor.m1serve(ctx, mlink, reqStart)
}) })
// XXX should Storage do it, or should it leave back non-closed?
// TODO -> Storage should not close backend.
err2 := stor.back.Close()
if err == nil {
err = err2
}
return err
} }
// m1initialize drives storage by master messages during initialization phase // m1initialize drives storage by master messages during initialization phase
...@@ -286,7 +286,9 @@ func (stor *Storage) serve(ctx context.Context) (err error) { ...@@ -286,7 +286,9 @@ func (stor *Storage) serve(ctx context.Context) (err error) {
wg.Add(1) wg.Add(1)
go func() { go func() {
defer wg.Done() defer wg.Done()
err := stor.serveLink(ctx, req, idReq) err := xio.WithCloseOnRetCancel(ctx, req.Link(), func() error {
return stor.serveLink(ctx, req, idReq)
})
if err == nil { if err == nil {
if ctx.Err() == nil { if ctx.Err() == nil {
// the error is not due to serve cancel // the error is not due to serve cancel
...@@ -334,7 +336,6 @@ func (stor *Storage) identify_(idReq *proto.RequestIdentification) (proto.Msg, * ...@@ -334,7 +336,6 @@ func (stor *Storage) identify_(idReq *proto.RequestIdentification) (proto.Msg, *
func (stor *Storage) serveLink(ctx context.Context, req *neonet.Request, idReq *proto.RequestIdentification) (err error) { func (stor *Storage) serveLink(ctx context.Context, req *neonet.Request, idReq *proto.RequestIdentification) (err error) {
link := req.Link() link := req.Link()
defer task.Runningf(&ctx, "serve %s", idReq.NID)(&err) defer task.Runningf(&ctx, "serve %s", idReq.NID)(&err)
defer xio.CloseWhenDone(ctx, link)()
// first process identification // first process identification
// TODO -> .Accept() that would listen, handshake and identify a node // TODO -> .Accept() that would listen, handshake and identify a node
......
...@@ -30,7 +30,7 @@ import ( ...@@ -30,7 +30,7 @@ import (
"lab.nexedi.com/kirr/neo/go/internal/log" "lab.nexedi.com/kirr/neo/go/internal/log"
"lab.nexedi.com/kirr/neo/go/internal/task" "lab.nexedi.com/kirr/neo/go/internal/task"
"lab.nexedi.com/kirr/neo/go/internal/xcontext" "lab.nexedi.com/kirr/neo/go/internal/xio"
"lab.nexedi.com/kirr/neo/go/neo/neonet" "lab.nexedi.com/kirr/neo/go/neo/neonet"
"lab.nexedi.com/kirr/neo/go/neo/proto" "lab.nexedi.com/kirr/neo/go/neo/proto"
) )
...@@ -75,7 +75,7 @@ func (l *listener) Accept(ctx context.Context) (_ *neonet.Request, msgID *proto. ...@@ -75,7 +75,7 @@ func (l *listener) Accept(ctx context.Context) (_ *neonet.Request, msgID *proto.
// the first conn must come with RequestIdentification packet // the first conn must come with RequestIdentification packet
defer xerr.Context(&err, "identify") defer xerr.Context(&err, "identify")
var req neonet.Request var req neonet.Request
err = xcontext.WithCloseOnErrCancel(ctx, link, func() error { err = xio.WithCloseOnErrCancel(ctx, link, func() error {
var err error var err error
req, err = link.Recv1(/*XXX ctx*/) req, err = link.Recv1(/*XXX ctx*/)
if err != nil { if err != nil {
...@@ -123,7 +123,7 @@ func Dial(ctx context.Context, typ proto.NodeType, net xnet.Networker, addr stri ...@@ -123,7 +123,7 @@ func Dial(ctx context.Context, typ proto.NodeType, net xnet.Networker, addr stri
defer xerr.Contextf(&err, "%s: request identification", link) defer xerr.Contextf(&err, "%s: request identification", link)
accept := &proto.AcceptIdentification{} accept := &proto.AcceptIdentification{}
err = xcontext.WithCloseOnErrCancel(ctx, link, func() error { err = xio.WithCloseOnErrCancel(ctx, link, func() error {
// FIXME error if peer sends us something with another connID // FIXME error if peer sends us something with another connID
// (currently we ignore and serveRecv will deadlock) // (currently we ignore and serveRecv will deadlock)
// //
......
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment