Skip to content

Commit

Permalink
Implement a new shared "Drainer" handler.
Browse files Browse the repository at this point in the history
This implements a new `http.Handler` called `Drainer`, which is intended to wrap some inner `http.Handler` business logic with a new outer handler that can respond to Kubelet probes (successfully until told to "Drain()").

This takes over the webhook's relatively new probe handling and lame duck logic with one key difference.  Previously the webhook waited for a fixed period after SIGTERM before exitting, but the new logic waits for this same grace period AFTER THE LAST REQUEST.  So if the handler keeps getting (non-probe) requests, the timer will continually reset, and once it stops receiving requests for the configured grace period, "Drain()" will return and the webhook will exit.

The goal of this work is to try to better cope with what we believe to be high tail latencies of the API server seeing that a webhook replica is shutting down.

Related: #1509
  • Loading branch information
mattmoor committed Jul 17, 2020
1 parent a9ea699 commit c76b2f1
Show file tree
Hide file tree
Showing 6 changed files with 318 additions and 25 deletions.
2 changes: 2 additions & 0 deletions network/error_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,8 @@ import (

// ErrorHandler sets up a handler suitable for use with the ErrorHandler field on
// httputil's reverse proxy.
// TODO(mattmoor): Move the implementation into handlers/error.go once downstream consumers
// have adopted the alias.
func ErrorHandler(logger *zap.SugaredLogger) func(http.ResponseWriter, *http.Request, error) {
return func(w http.ResponseWriter, req *http.Request, err error) {
ss := readSockStat(logger)
Expand Down
18 changes: 18 additions & 0 deletions network/handlers/doc.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,18 @@
/*
Copyright 2020 The Knative Authors
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/

// Package handlers holds useful HTTP handler implementations.
package handlers
105 changes: 105 additions & 0 deletions network/handlers/drain.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,105 @@
/*
Copyright 2020 The Knative Authors
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/

package handlers

import (
"net/http"
"sync"
"time"

"knative.dev/pkg/network"
)

// Drainer wraps an inner http.Handler to support responding to kubelet
// probes with a "200 OK" until the handler is told to Drain.
// When the Drainer is told to Drain, it will immediately start to fail
// probes with a "500 shutting down", and the call will block until no
// requests have been received for QuietPeriod (defaults to
// network.DefaultDrainTimeout).
type Drainer struct {
// Mutex guards the initialization and resets of the timer
sync.Mutex

// Inner is the http.Handler to which we delegate actual requests.
Inner http.Handler

// QuietPeriod is the duration that must elapse without any requests
// after Drain is called before it may return.
QuietPeriod time.Duration

// once is used to initialize timer
once sync.Once

// timer is used to orchestrate the drain.
timer *time.Timer
}

// Ensure Drainer implements http.Handler
var _ http.Handler = (*Drainer)(nil)

// ServeHTTP implements http.Handler
func (d *Drainer) ServeHTTP(w http.ResponseWriter, r *http.Request) {
if network.IsKubeletProbe(r) { // Respond to probes regardless of path.
if d.draining() {
http.Error(w, "shutting down", http.StatusInternalServerError)
} else {
w.WriteHeader(http.StatusOK)
}
return
}

d.reset()
d.Inner.ServeHTTP(w, r)
}

// Drain blocks until QuietPeriod has elapsed since the last request,
// starting when this is invoked.
func (d *Drainer) Drain() {
d.once.Do(func() {
t := func() *time.Timer {
d.Lock()
defer d.Unlock()
if d.QuietPeriod == 0 {
d.QuietPeriod = network.DefaultDrainTimeout
}
d.timer = time.NewTimer(d.QuietPeriod)
return d.timer
}()

<-t.C
})
}

// reset resets the drain timer to the full amount of time.
func (d *Drainer) reset() {
d.Lock()
defer d.Unlock()
if d.timer == nil {
return
}
if d.timer.Stop() {
d.timer.Reset(d.QuietPeriod)
}
}

// draining returns whether we are draining the handler.
func (d *Drainer) draining() bool {
d.Lock()
defer d.Unlock()

return d.timer != nil
}
160 changes: 160 additions & 0 deletions network/handlers/drain_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,160 @@
/*
Copyright 2020 The Knative Authors
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/

package handlers

import (
"context"
"net/http"
"net/http/httptest"
"testing"
"time"

"knative.dev/pkg/network"
)

func TestDrainMechanics(t *testing.T) {
var (
w http.ResponseWriter
req = &http.Request{}
probe = &http.Request{
Header: http.Header{
"User-Agent": []string{network.KubeProbeUAPrefix},
},
}
)

inner := http.HandlerFunc(func(http.ResponseWriter, *http.Request) {})

drainer := &Drainer{
Inner: inner,
QuietPeriod: 100 * time.Millisecond,
}

// Works before Drain is called.
drainer.ServeHTTP(w, req)
drainer.ServeHTTP(w, req)
drainer.ServeHTTP(w, req)

// Check for 200 OK
resp := httptest.NewRecorder()
drainer.ServeHTTP(resp, probe)
if got, want := resp.Code, http.StatusOK; got != want {
t.Errorf("probe status = %d, wanted %d", got, want)
}

// Start to drain, and cancel the context when it returns.
ctx, cancel := context.WithCancel(context.Background())
go func() {
drainer.Drain()
cancel()
}()

select {
case <-time.After(50 * time.Millisecond):
// Drain is blocking.
case <-ctx.Done():
t.Error("Drain terminated prematurely.")
}
// Now send a request to reset things.
drainer.ServeHTTP(w, req)

// Check for 500 shutting down
resp = httptest.NewRecorder()
drainer.ServeHTTP(resp, probe)
if got, want := resp.Code, http.StatusInternalServerError; got != want {
t.Errorf("probe status = %d, wanted %d", got, want)
}

select {
case <-time.After(50 * time.Millisecond):
// Drain is blocking.
case <-ctx.Done():
t.Error("Drain terminated prematurely.")
}
// Now send a request to reset things.
drainer.ServeHTTP(w, req)

select {
case <-time.After(50 * time.Millisecond):
// Drain is blocking.
case <-ctx.Done():
t.Error("Drain terminated prematurely.")
}
// Now send a request to reset things.
drainer.ServeHTTP(w, req)

select {
case <-time.After(50 * time.Millisecond):
// Drain is blocking.
case <-ctx.Done():
t.Error("Drain terminated prematurely.")
}
// Probing does not reset the clock.
// Check for 500 shutting down
resp = httptest.NewRecorder()
drainer.ServeHTTP(resp, probe)
if got, want := resp.Code, http.StatusInternalServerError; got != want {
t.Errorf("probe status = %d, wanted %d", got, want)
}

// Big finish, test that multiple invocations of Drain all block.
ctx1, cancel1 := context.WithCancel(context.Background())
go func() {
drainer.Drain()
cancel1()
}()
ctx2, cancel2 := context.WithCancel(context.Background())
go func() {
drainer.Drain()
cancel2()
}()
ctx3, cancel3 := context.WithCancel(context.Background())
go func() {
drainer.Drain()
cancel3()
}()

select {
case <-time.After(60 * time.Millisecond):
t.Error("Timed out waiting for Drain to return.")

case <-ctx.Done():
case <-ctx1.Done():
case <-ctx2.Done():
case <-ctx3.Done():
// Once the first context is cancelled, check that all of them are cancelled.
}

// Check that a 4th and final one after things complete finishes instantly.
ctx4, cancel4 := context.WithCancel(context.Background())
go func() {
drainer.Drain()
cancel4()
}()

// Give the rest a short window to complete.
time.Sleep(time.Millisecond)

for idx, ictx := range []context.Context{ctx, ctx1, ctx2, ctx3, ctx4} {
select {
case <-ictx.Done():
// Should be done.
default:
t.Errorf("Drain[%d] did not complete.", idx)
}
}
}
23 changes: 23 additions & 0 deletions network/handlers/error.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,23 @@
/*
Copyright 2020 The Knative Authors
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/

package handlers

import "knative.dev/pkg/network"

// Error sets up a handler suitable for use with the ErrorHandler field on
// httputil's reverse proxy, which logs /proc/net/sockstat data.
var Error = network.ErrorHandler
35 changes: 10 additions & 25 deletions webhook/webhook.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
// Injection stuff
kubeclient "knative.dev/pkg/client/injection/kube/client"
kubeinformerfactory "knative.dev/pkg/injection/clients/namespacedkube/informers/factory"
"knative.dev/pkg/network/handlers"

"go.uber.org/zap"
"golang.org/x/sync/errgroup"
Expand Down Expand Up @@ -83,8 +84,6 @@ type Webhook struct {
// synced is function that is called when the informers have been synced.
synced context.CancelFunc

// stopCh is closed when we should start failing readiness probes.
stopCh chan struct{}
// grace period is how long to wait after failing readiness probes
// before shutting down.
gracePeriod time.Duration
Expand Down Expand Up @@ -137,7 +136,6 @@ func New(
secretlister: secretInformer.Lister(),
Logger: logger,
synced: cancel,
stopCh: make(chan struct{}),
gracePeriod: network.DefaultDrainTimeout,
}

Expand Down Expand Up @@ -176,8 +174,13 @@ func (wh *Webhook) Run(stop <-chan struct{}) error {
logger := wh.Logger
ctx := logging.WithLogger(context.Background(), logger)

drainer := &handlers.Drainer{
Inner: wh,
QuietPeriod: wh.gracePeriod,
}

server := &http.Server{
Handler: wh,
Handler: drainer,
Addr: fmt.Sprintf(":%d", wh.Options.Port),
TLSConfig: &tls.Config{
GetCertificate: func(*tls.ClientHelloInfo) (*tls.Certificate, error) {
Expand Down Expand Up @@ -215,19 +218,12 @@ func (wh *Webhook) Run(stop <-chan struct{}) error {
select {
case <-stop:
eg.Go(func() error {
// Start failing readiness probes immediately.
logger.Info("Starting to fail readiness probes...")
close(wh.stopCh)

// As we start to shutdown, disable keep-alives to avoid clients hanging onto connections.
server.SetKeepAlivesEnabled(false)

// Wait for a grace period for the above to take effect and this Pod's
// endpoint to be removed from the webhook service's Endpoints.
// For this to be effective, it must be greater than the probe's
// periodSeconds times failureThreshold by a margin suitable to
// propagate the new Endpoints data across the cluster.
time.Sleep(wh.gracePeriod)
// Start failing readiness probes immediately.
logger.Info("Starting to fail readiness probes...")
drainer.Drain()

return server.Shutdown(context.Background())
})
Expand All @@ -241,17 +237,6 @@ func (wh *Webhook) Run(stop <-chan struct{}) error {
}

func (wh *Webhook) ServeHTTP(w http.ResponseWriter, r *http.Request) {
// Respond to probes regardless of path.
if network.IsKubeletProbe(r) {
select {
case <-wh.stopCh:
http.Error(w, "shutting down", http.StatusInternalServerError)
default:
w.WriteHeader(http.StatusOK)
}
return
}

// Verify the content type is accurate.
contentType := r.Header.Get("Content-Type")
if contentType != "application/json" {
Expand Down

0 comments on commit c76b2f1

Please sign in to comment.