Initial commit

This commit is contained in:
Mahak Mukhi
2017-03-09 15:33:28 -08:00
parent 4eaacfed97
commit d9b58b5ce9
7 changed files with 120 additions and 25 deletions

View File

@ -16,3 +16,12 @@ type ClientParameters struct {
// If true, client runs keepalive checks even with no active RPCs. // If true, client runs keepalive checks even with no active RPCs.
PermitWithoutStream bool PermitWithoutStream bool
} }
// TODO(mmukhi) : documentation
type ServerParameters struct {
MaxConnectionIdle time.Duration
MaxConnectionAge time.Duration
MaxConnectionAgeGrace time.Duration
Time time.Duration
Timeout time.Duration
}

View File

@ -53,6 +53,7 @@ import (
"google.golang.org/grpc/credentials" "google.golang.org/grpc/credentials"
"google.golang.org/grpc/grpclog" "google.golang.org/grpc/grpclog"
"google.golang.org/grpc/internal" "google.golang.org/grpc/internal"
"google.golang.org/grpc/keepalive"
"google.golang.org/grpc/metadata" "google.golang.org/grpc/metadata"
"google.golang.org/grpc/stats" "google.golang.org/grpc/stats"
"google.golang.org/grpc/tap" "google.golang.org/grpc/tap"
@ -117,6 +118,7 @@ type options struct {
maxConcurrentStreams uint32 maxConcurrentStreams uint32
useHandlerImpl bool // use http.Handler-based server useHandlerImpl bool // use http.Handler-based server
unknownStreamDesc *StreamDesc unknownStreamDesc *StreamDesc
keepaliveParams keepalive.ServerParameters
} }
var defaultMaxMsgSize = 1024 * 1024 * 4 // use 4MB as the default message size limit var defaultMaxMsgSize = 1024 * 1024 * 4 // use 4MB as the default message size limit
@ -124,6 +126,13 @@ var defaultMaxMsgSize = 1024 * 1024 * 4 // use 4MB as the default message size l
// A ServerOption sets options. // A ServerOption sets options.
type ServerOption func(*options) type ServerOption func(*options)
// TODO(mmukhi) : Documentation
func KeepaliveParams(kp keepalive.ServerParameters) ServerOption {
return func(o *options) {
o.keepaliveParams = kp
}
}
// CustomCodec returns a ServerOption that sets a codec for message marshaling and unmarshaling. // CustomCodec returns a ServerOption that sets a codec for message marshaling and unmarshaling.
func CustomCodec(codec Codec) ServerOption { func CustomCodec(codec Codec) ServerOption {
return func(o *options) { return func(o *options) {
@ -469,6 +478,7 @@ func (s *Server) serveHTTP2Transport(c net.Conn, authInfo credentials.AuthInfo)
AuthInfo: authInfo, AuthInfo: authInfo,
InTapHandle: s.opts.inTapHandle, InTapHandle: s.opts.inTapHandle,
StatsHandler: s.opts.statsHandler, StatsHandler: s.opts.statsHandler,
keepaliveParams: s.opts.keepaliveParams,
} }
st, err := transport.NewServerTransport("http2", c, config) st, err := transport.NewServerTransport("http2", c, config)
if err != nil { if err != nil {

View File

@ -49,9 +49,14 @@ const (
initialWindowSize = defaultWindowSize // for an RPC initialWindowSize = defaultWindowSize // for an RPC
initialConnWindowSize = defaultWindowSize * 16 // for a connection initialConnWindowSize = defaultWindowSize * 16 // for a connection
infinity = time.Duration(math.MaxInt64) infinity = time.Duration(math.MaxInt64)
defaultKeepaliveTime = infinity defaultClientKeepaliveTime = infinity
defaultKeepaliveTimeout = time.Duration(20 * time.Second) defaultClientKeepaliveTimeout = time.Duration(20 * time.Second)
defaultMaxStreamsClient = 100 defaultMaxStreamsClient = 100
defaultMaxConnectionIdle = infinity
defaultMaxConnectionAge = infinity
defaultMaxConnectionAgeGrace = infinity
defaultServerKeepaliveTime = time.Duration(2 * time.Hour)
defaultServerKeepaliveTimeout = time.Duration(20 * time.Second)
) )
// The following defines various control items which could flow through // The following defines various control items which could flow through

View File

@ -194,10 +194,10 @@ func newHTTP2Client(ctx context.Context, addr TargetInfo, opts ConnectOptions) (
kp := opts.KeepaliveParams kp := opts.KeepaliveParams
// Validate keepalive parameters. // Validate keepalive parameters.
if kp.Time == 0 { if kp.Time == 0 {
kp.Time = defaultKeepaliveTime kp.Time = defaultClientKeepaliveTime
} }
if kp.Timeout == 0 { if kp.Timeout == 0 {
kp.Timeout = defaultKeepaliveTimeout kp.Timeout = defaultClientKeepaliveTimeout
} }
var buf bytes.Buffer var buf bytes.Buffer
t := &http2Client{ t := &http2Client{

View File

@ -41,6 +41,7 @@ import (
"net" "net"
"strconv" "strconv"
"sync" "sync"
"time"
"golang.org/x/net/context" "golang.org/x/net/context"
"golang.org/x/net/http2" "golang.org/x/net/http2"
@ -48,6 +49,7 @@ import (
"google.golang.org/grpc/codes" "google.golang.org/grpc/codes"
"google.golang.org/grpc/credentials" "google.golang.org/grpc/credentials"
"google.golang.org/grpc/grpclog" "google.golang.org/grpc/grpclog"
"google.golang.org/grpc/keepalive"
"google.golang.org/grpc/metadata" "google.golang.org/grpc/metadata"
"google.golang.org/grpc/peer" "google.golang.org/grpc/peer"
"google.golang.org/grpc/stats" "google.golang.org/grpc/stats"
@ -90,11 +92,15 @@ type http2Server struct {
stats stats.Handler stats stats.Handler
// TODO(mmukhi): Documentation
kp keepalive.ServerParameters
mu sync.Mutex // guard the following mu sync.Mutex // guard the following
state transportState state transportState
activeStreams map[uint32]*Stream activeStreams map[uint32]*Stream
// the per-stream outbound flow control window size set by the peer. // the per-stream outbound flow control window size set by the peer.
streamSendQuota uint32 streamSendQuota uint32
idle time.Time
} }
// newHTTP2Server constructs a ServerTransport based on HTTP2. ConnectionError is // newHTTP2Server constructs a ServerTransport based on HTTP2. ConnectionError is
@ -128,6 +134,22 @@ func newHTTP2Server(conn net.Conn, config *ServerConfig) (_ ServerTransport, err
return nil, connectionErrorf(true, err, "transport: %v", err) return nil, connectionErrorf(true, err, "transport: %v", err)
} }
} }
kp := config.keepaliveParams
if kp.MaxConnectionIdle == 0 {
kp.MaxConnectionIdle = defaultMaxConnectionIdle
}
if kp.MaxConnectionAge == 0 {
kp.MaxConnectionAge = defaultMaxConnectionAge
}
if kp.MaxConnectionAgeGrace == 0 {
kp.MaxConnectionAgeGrace = defaultMaxConnectionAgeGrace
}
if kp.Time == 0 {
kp.Time = defaultServerKeepaliveTime
}
if kp.Timeout == 0 {
kp.Timeout = defaultServerKeepaliveTimeout
}
var buf bytes.Buffer var buf bytes.Buffer
t := &http2Server{ t := &http2Server{
ctx: context.Background(), ctx: context.Background(),
@ -149,6 +171,8 @@ func newHTTP2Server(conn net.Conn, config *ServerConfig) (_ ServerTransport, err
activeStreams: make(map[uint32]*Stream), activeStreams: make(map[uint32]*Stream),
streamSendQuota: defaultWindowSize, streamSendQuota: defaultWindowSize,
stats: config.StatsHandler, stats: config.StatsHandler,
kp: kp,
idle: time.Now(),
} }
if t.stats != nil { if t.stats != nil {
t.ctx = t.stats.TagConn(t.ctx, &stats.ConnTagInfo{ t.ctx = t.stats.TagConn(t.ctx, &stats.ConnTagInfo{
@ -248,6 +272,9 @@ func (t *http2Server) operateHeaders(frame *http2.MetaHeadersFrame, handle func(
t.maxStreamID = s.id t.maxStreamID = s.id
s.sendQuotaPool = newQuotaPool(int(t.streamSendQuota)) s.sendQuotaPool = newQuotaPool(int(t.streamSendQuota))
t.activeStreams[s.id] = s t.activeStreams[s.id] = s
if len(t.activeStreams) == 1 {
t.idle = time.Time{}
}
t.mu.Unlock() t.mu.Unlock()
s.windowHandler = func(n int) { s.windowHandler = func(n int) {
t.updateWindow(s, uint32(n)) t.updateWindow(s, uint32(n))
@ -735,6 +762,37 @@ func (t *http2Server) applySettings(ss []http2.Setting) {
} }
} }
// TODO(mmukhi): Documentation
func (t *http2Server) keepalive() {
//p := &ping{data: [8]byte{}}
maxIdle := time.NewTimer(t.kp.MaxConnectionIdle)
maxAge := time.NewTimer(t.kp.MaxConnectionAge)
keepalive := time.NewTimer(t.kp.Time)
t.mu.Lock()
idle := t.idle
t.mu.Unlock()
for {
select {
case <-maxIdle.C:
if idle == t.idle {
// send go away
continue
}
if idle.IsZero() {
maxIdle.Reset(t.kp.MaxConnectionIdle)
continue
}
maxIdle.Reset(t.kp.MaxConnectionIdle - time.Since(idle))
case <-maxAge.C:
case <-keepalive.C:
case <-t.shutdownChan:
// TODO(mmukhi): clean-up
return
}
}
}
// controller running in a separate goroutine takes charge of sending control // controller running in a separate goroutine takes charge of sending control
// frames (e.g., window update, reset stream, setting, etc.) to the server. // frames (e.g., window update, reset stream, setting, etc.) to the server.
func (t *http2Server) controller() { func (t *http2Server) controller() {
@ -816,6 +874,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 len(t.activeStreams) == 0 {
t.idle = time.Now()
}
if t.state == draining && len(t.activeStreams) == 0 { if t.state == draining && len(t.activeStreams) == 0 {
defer t.Close() defer t.Close()
} }

View File

@ -366,6 +366,7 @@ type ServerConfig struct {
AuthInfo credentials.AuthInfo AuthInfo credentials.AuthInfo
InTapHandle tap.ServerInHandle InTapHandle tap.ServerInHandle
StatsHandler stats.Handler StatsHandler stats.Handler
keepaliveParams keepalive.ServerParameters
} }
// NewServerTransport creates a ServerTransport with conn or non-nil error // NewServerTransport creates a ServerTransport with conn or non-nil error

View File

@ -156,7 +156,7 @@ func (h *testStreamHandler) handleStreamInvalidHeaderField(t *testing.T, s *Stre
} }
// start starts server. Other goroutines should block on s.readyChan for further operations. // start starts server. Other goroutines should block on s.readyChan for further operations.
func (s *server) start(t *testing.T, port int, maxStreams uint32, ht hType) { func (s *server) start(t *testing.T, port int, serverConfig *ServerConfig, ht hType) {
var err error var err error
if port == 0 { if port == 0 {
s.lis, err = net.Listen("tcp", "localhost:0") s.lis, err = net.Listen("tcp", "localhost:0")
@ -180,10 +180,7 @@ func (s *server) start(t *testing.T, port int, maxStreams uint32, ht hType) {
if err != nil { if err != nil {
return return
} }
config := &ServerConfig{ transport, err := NewServerTransport("http2", conn, serverConfig)
MaxStreams: maxStreams,
}
transport, err := NewServerTransport("http2", conn, config)
if err != nil { if err != nil {
return return
} }
@ -252,12 +249,12 @@ func (s *server) stop() {
} }
func setUp(t *testing.T, port int, maxStreams uint32, ht hType) (*server, ClientTransport) { func setUp(t *testing.T, port int, maxStreams uint32, ht hType) (*server, ClientTransport) {
return setUpWithOptions(t, port, maxStreams, ht, ConnectOptions{}) return setUpWithOptions(t, port, &ServerConfig{MaxStreams: maxStreams}, ht, ConnectOptions{})
} }
func setUpWithOptions(t *testing.T, port int, maxStreams uint32, ht hType, copts ConnectOptions) (*server, ClientTransport) { func setUpWithOptions(t *testing.T, port int, serverConfig *ServerConfig, ht hType, copts ConnectOptions) (*server, ClientTransport) {
server := &server{startedErr: make(chan error, 1)} server := &server{startedErr: make(chan error, 1)}
go server.start(t, port, maxStreams, ht) go server.start(t, port, serverConfig, ht)
server.wait(t, 2*time.Second) server.wait(t, 2*time.Second)
addr := "localhost:" + server.port addr := "localhost:" + server.port
var ( var (
@ -301,6 +298,18 @@ func setUpWithNoPingServer(t *testing.T, copts ConnectOptions, done chan net.Con
return tr return tr
} }
// TestMaxConnectionIdle tests that a server will send GoAway to a idle client.
// An idle client is one who doesn't make any RPC calls for a duration of
// MaxConnectionIdle time.
func TestMaxConnectionIdle(t *testing.T) {
serverConfig := &ServerConfig{
keepaliveParams: keepalive.ServerParams{
MaxConnectionIdle: 2 * time.Second,
},
}
server, client := setUpWithOptions(t, 0, serverConfig, normal, ConnectOptions{})
}
func TestKeepaliveClientClosesIdleTransport(t *testing.T) { func TestKeepaliveClientClosesIdleTransport(t *testing.T) {
done := make(chan net.Conn, 1) done := make(chan net.Conn, 1)
tr := setUpWithNoPingServer(t, ConnectOptions{KeepaliveParams: keepalive.ClientParameters{ tr := setUpWithNoPingServer(t, ConnectOptions{KeepaliveParams: keepalive.ClientParameters{
@ -377,7 +386,7 @@ func TestKeepaliveClientClosesWithActiveStreams(t *testing.T) {
} }
func TestKeepaliveClientStaysHealthyWithResponsiveServer(t *testing.T) { func TestKeepaliveClientStaysHealthyWithResponsiveServer(t *testing.T) {
s, tr := setUpWithOptions(t, 0, math.MaxUint32, normal, ConnectOptions{KeepaliveParams: keepalive.ClientParameters{ s, tr := setUpWithOptions(t, 0, &ServerConfig{MaxStreams: math.MaxUint32}, normal, ConnectOptions{KeepaliveParams: keepalive.ClientParameters{
Time: 2 * time.Second, // Keepalive time = 2 sec. Time: 2 * time.Second, // Keepalive time = 2 sec.
Timeout: 1 * time.Second, // Keepalive timeout = 1 sec. Timeout: 1 * time.Second, // Keepalive timeout = 1 sec.
PermitWithoutStream: true, // Run keepalive even with no RPCs. PermitWithoutStream: true, // Run keepalive even with no RPCs.