Make it work for streaming

This commit is contained in:
iamqizhao
2016-07-21 16:19:34 -07:00
parent 873cc272c2
commit 9ad4c58355
8 changed files with 120 additions and 37 deletions

View File

@ -625,19 +625,23 @@ func (ac *addrConn) transportMonitor() {
// the addrConn is idle (i.e., no RPC in flight). // the addrConn is idle (i.e., no RPC in flight).
case <-ac.shutdownChan: case <-ac.shutdownChan:
return return
case <-t.Done(): case <-t.GoAway():
ac.tearDown(errConnDrain)
ac.cc.newAddrConn(ac.addr, true)
return
case <-t.Error():
ac.mu.Lock() ac.mu.Lock()
if ac.state == Shutdown { if ac.state == Shutdown {
// ac.tearDown(...) has been invoked. // ac.tearDown(...) has been invoked.
ac.mu.Unlock() ac.mu.Unlock()
return return
} }
if t.Err() == transport.ErrConnDrain { //if t.Err() == transport.ErrConnDrain {
ac.mu.Unlock() // ac.mu.Unlock()
ac.tearDown(errConnDrain) // ac.tearDown(errConnDrain)
ac.cc.newAddrConn(ac.addr, true) // ac.cc.newAddrConn(ac.addr, true)
return // return
} //}
ac.state = TransientFailure ac.state = TransientFailure
ac.stateCV.Broadcast() ac.stateCV.Broadcast()
ac.mu.Unlock() ac.mu.Unlock()

View File

@ -391,6 +391,7 @@ func (s *Server) serveNewHTTP2Transport(c net.Conn, authInfo credentials.AuthInf
st.Close() st.Close()
return return
} }
grpclog.Println("DEBUG addConn ... ")
s.serveStreams(st) s.serveStreams(st)
} }
@ -808,7 +809,6 @@ func (s *Server) GracefulStop() {
s.events = nil s.events = nil
} }
s.mu.Unlock() s.mu.Unlock()
} }
func init() { func init() {

View File

@ -184,7 +184,7 @@ func NewClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth
// when there is no pending I/O operations on this stream. // when there is no pending I/O operations on this stream.
go func() { go func() {
select { select {
case <-t.Done(): case <-t.Error():
// Incur transport error, simply exit. // Incur transport error, simply exit.
case <-s.Done(): case <-s.Done():
// TODO: The trace of the RPC is terminated here when there is no pending // TODO: The trace of the RPC is terminated here when there is no pending

View File

@ -565,22 +565,12 @@ func testTimeoutOnDeadServer(t *testing.T, e env) {
awaitNewConnLogOutput() awaitNewConnLogOutput()
} }
func TestFailFast(t *testing.T) {
defer leakCheck(t)()
for _, e := range listTestEnv() {
testFailFast(t, e)
}
}
func TestServerGoAway(t *testing.T) { func TestServerGoAway(t *testing.T) {
defer leakCheck(t)() defer leakCheck(t)()
for _, e := range listTestEnv() { for _, e := range listTestEnv() {
if e.name == "handler-tls" { if e.name == "handler-tls" {
continue continue
} }
//if e.name != "tcp-clear" {
// continue
//}
testServerGoAway(t, e) testServerGoAway(t, e)
} }
} }
@ -621,6 +611,83 @@ func testServerGoAway(t *testing.T, e env) {
awaitNewConnLogOutput() awaitNewConnLogOutput()
} }
func TestServerGoAwayPendingRPC(t *testing.T) {
defer leakCheck(t)()
for _, e := range listTestEnv() {
if e.name == "handler-tls" {
continue
}
testServerGoAwayPendingRPC(t, e)
}
}
func testServerGoAwayPendingRPC(t *testing.T, e env) {
te := newTest(t, e)
te.userAgent = testAppUA
te.declareLogNoise(
"transport: http2Client.notifyError got notified that the client transport was broken EOF",
"grpc: Conn.transportMonitor exits due to: grpc: the client connection is closing",
"grpc: Conn.resetTransport failed to create client transport: connection error",
"grpc: Conn.resetTransport failed to create client transport: connection error: desc = \"transport: dial unix",
)
te.startServer(&testServer{security: e.security})
defer te.tearDown()
cc := te.clientConn()
tc := testpb.NewTestServiceClient(cc)
ctx, cancel := context.WithCancel(context.Background())
stream, err := tc.FullDuplexCall(ctx, grpc.FailFast(false))
if err != nil {
t.Fatalf("%v.FullDuplexCall(_) = _, %v, want <nil>", tc, err)
}
if _, err := tc.EmptyCall(context.Background(), &testpb.Empty{}, grpc.FailFast(false)); err != nil {
t.Fatalf("fadjflajdflkaflj")
}
ch := make(chan struct{})
go func() {
te.srv.GracefulStop()
close(ch)
}()
for {
ctx, _ := context.WithTimeout(context.Background(), 10*time.Millisecond)
if _, err := tc.EmptyCall(ctx, &testpb.Empty{}, grpc.FailFast(false)); err == nil {
continue
} else {
break
}
}
respParam := []*testpb.ResponseParameters{
{
Size: proto.Int32(1),
},
}
payload, err := newPayload(testpb.PayloadType_COMPRESSABLE, int32(100))
if err != nil {
t.Fatal(err)
}
req := &testpb.StreamingOutputCallRequest{
ResponseType: testpb.PayloadType_COMPRESSABLE.Enum(),
ResponseParameters: respParam,
Payload: payload,
}
if err := stream.Send(req); err != nil {
t.Fatalf("%v.Send(%v) = %v, want <nil>", stream, req, err)
}
if _, err := stream.Recv(); err != nil {
t.Fatalf("%v.Recv() = %v, want _, <nil>", stream, err)
}
cancel()
<-ch
awaitNewConnLogOutput()
}
func TestFailFast(t *testing.T) {
defer leakCheck(t)()
for _, e := range listTestEnv() {
testFailFast(t, e)
}
}
func testFailFast(t *testing.T, e env) { func testFailFast(t *testing.T, e env) {
te := newTest(t, e) te := newTest(t, e)
te.userAgent = testAppUA te.userAgent = testAppUA

View File

@ -71,7 +71,8 @@ type http2Client struct {
shutdownChan chan struct{} shutdownChan chan struct{}
// errorChan is closed to notify the I/O error to the caller. // errorChan is closed to notify the I/O error to the caller.
errorChan chan struct{} errorChan chan struct{}
err error //err error
goAway chan struct{}
framer *framer framer *framer
hBuf *bytes.Buffer // the buffer for HPACK encoding hBuf *bytes.Buffer // the buffer for HPACK encoding
@ -149,6 +150,7 @@ func newHTTP2Client(addr string, opts *ConnectOptions) (_ ClientTransport, err e
writableChan: make(chan int, 1), writableChan: make(chan int, 1),
shutdownChan: make(chan struct{}), shutdownChan: make(chan struct{}),
errorChan: make(chan struct{}), errorChan: make(chan struct{}),
goAway: make(chan struct{}),
framer: newFramer(conn), framer: newFramer(conn),
hBuf: &buf, hBuf: &buf,
hEnc: hpack.NewEncoder(&buf), hEnc: hpack.NewEncoder(&buf),
@ -408,13 +410,13 @@ func (t *http2Client) CloseStream(s *Stream, err error) {
if t.streamsQuota != nil { if t.streamsQuota != nil {
updateStreams = true updateStreams = true
} }
if t.state == draining && len(t.activeStreams) == 1 { delete(t.activeStreams, s.id)
if t.state == draining && len(t.activeStreams) == 0 {
// The transport is draining and s is the last live stream on t. // The transport is draining and s is the last live stream on t.
t.mu.Unlock() t.mu.Unlock()
t.Close() t.Close()
return return
} }
delete(t.activeStreams, s.id)
t.mu.Unlock() t.mu.Unlock()
if updateStreams { if updateStreams {
t.streamsQuota.add(1) t.streamsQuota.add(1)
@ -485,10 +487,14 @@ func (t *http2Client) GracefulClose() error {
} }
t.state = draining t.state = draining
// Notify the streams which were initiated after the server sent GOAWAY. // Notify the streams which were initiated after the server sent GOAWAY.
for i := t.goAwayID + 2; i < t.nextID; i += 2 { select {
if s, ok := t.activeStreams[i]; ok { case <-t.goAway:
close(s.goAway) for i := t.goAwayID + 2; i < t.nextID; i += 2 {
if s, ok := t.activeStreams[i]; ok {
close(s.goAway)
}
} }
default:
} }
active := len(t.activeStreams) active := len(t.activeStreams)
t.mu.Unlock() t.mu.Unlock()
@ -736,8 +742,7 @@ func (t *http2Client) handleGoAway(f *http2.GoAwayFrame) {
t.mu.Lock() t.mu.Lock()
if t.state == reachable { if t.state == reachable {
t.goAwayID = f.LastStreamID t.goAwayID = f.LastStreamID
t.err = ErrConnDrain close(t.goAway)
close(t.errorChan)
} }
t.mu.Unlock() t.mu.Unlock()
} }
@ -944,14 +949,18 @@ func (t *http2Client) controller() {
} }
} }
func (t *http2Client) Done() <-chan struct{} { func (t *http2Client) Error() <-chan struct{} {
return t.errorChan return t.errorChan
} }
func (t *http2Client) Err() error { func (t *http2Client) GoAway() <-chan struct{} {
return t.err return t.goAway
} }
//func (t *http2Client) Err() error {
// return t.err
//}
func (t *http2Client) notifyError(err error) { func (t *http2Client) notifyError(err error) {
t.mu.Lock() t.mu.Lock()
defer t.mu.Unlock() defer t.mu.Unlock()

View File

@ -737,6 +737,9 @@ func (t *http2Server) Close() (err error) {
func (t *http2Server) closeStream(s *Stream) { func (t *http2Server) closeStream(s *Stream) {
t.mu.Lock() t.mu.Lock()
delete(t.activeStreams, s.id) delete(t.activeStreams, s.id)
if t.state == draining && len(t.activeStreams) == 0 {
defer t.Close()
}
t.mu.Unlock() t.mu.Unlock()
// In case stream sending and receiving are invoked in separate // In case stream sending and receiving are invoked in separate
// goroutines (e.g., bi-directional streaming), cancel needs to be // goroutines (e.g., bi-directional streaming), cancel needs to be

View File

@ -431,7 +431,7 @@ type ClientTransport interface {
// this in order to take action (e.g., close the current transport // this in order to take action (e.g., close the current transport
// and create a new one) in error case. It should not return nil // and create a new one) in error case. It should not return nil
// once the transport is initiated. // once the transport is initiated.
//Error() <-chan struct{} Error() <-chan struct{}
// Done returns a channel that is closed when some I/O error // Done returns a channel that is closed when some I/O error
// happens or ClientTranspor receives the draining signal from the server // happens or ClientTranspor receives the draining signal from the server
@ -439,10 +439,10 @@ type ClientTransport interface {
// a goroutine to monitor this in order to take action (e.g., close // a goroutine to monitor this in order to take action (e.g., close
// the current transport and create a new one) in error case. It should // the current transport and create a new one) in error case. It should
// not return nil once the transport is initiated. // not return nil once the transport is initiated.
Done() <-chan struct{} GoAway() <-chan struct{}
// Err returns ... // Err returns ...
Err() error //Err() error
} }
// ServerTransport is the common interface for all gRPC server-side transport // ServerTransport is the common interface for all gRPC server-side transport

View File

@ -271,8 +271,8 @@ func TestClientSendAndReceive(t *testing.T) {
func TestClientErrorNotify(t *testing.T) { func TestClientErrorNotify(t *testing.T) {
server, ct := setUp(t, 0, math.MaxUint32, normal) server, ct := setUp(t, 0, math.MaxUint32, normal)
go server.stop() go server.stop()
// ct.reader should detect the error and activate ct.Done(). // ct.reader should detect the error and activate ct.Error().
<-ct.Done() <-ct.Error()
ct.Close() ct.Close()
} }
@ -309,7 +309,7 @@ func TestClientMix(t *testing.T) {
s.stop() s.stop()
}(s) }(s)
go func(ct ClientTransport) { go func(ct ClientTransport) {
<-ct.Done() <-ct.Error()
ct.Close() ct.Close()
}(ct) }(ct)
for i := 0; i < 1000; i++ { for i := 0; i < 1000; i++ {
@ -709,7 +709,7 @@ func TestClientWithMisbehavedServer(t *testing.T) {
} }
} }
// http2Client.errChan is closed due to connection flow control window size violation. // http2Client.errChan is closed due to connection flow control window size violation.
<-conn.Done() <-conn.Error()
ct.Close() ct.Close()
server.stop() server.stop()
} }