Skip to content
This repository has been archived by the owner on Mar 9, 2022. It is now read-only.

Commit

Permalink
reload cni network config if has fs change events
Browse files Browse the repository at this point in the history
With go RWMutex design, no goroutine should expect to be able to
acquire a read lock until the read lock has been released, if one
goroutine call lock.

The original design is to reload cni network config on every single
Status CRI gRPC call. If one RunPodSandbox request holds read lock
to allocate IP for too long, all other RunPodSandbox/StopPodSandbox
requests will wait for the RunPodSandbox request to release read lock.
And the Status CRI call will fail and kubelet becomes NOTReady.

Reload cni network config at every single Status CRI call is not
necessary and also brings NOTReady situation. To lower the possibility
of NOTReady, CRI will reload cni network config if there is any valid fs
change events from the cni network config dir.

Signed-off-by: Wei Fu <fuweid89@gmail.com>
  • Loading branch information
fuweid committed Mar 8, 2020
1 parent 1a00c06 commit 8499fd7
Show file tree
Hide file tree
Showing 5 changed files with 144 additions and 17 deletions.
116 changes: 116 additions & 0 deletions pkg/server/cni_conf_syncer.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,116 @@
/*
Copyright The Containerd 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 server

import (
"sync"

cni "github.com/containerd/go-cni"
"github.com/fsnotify/fsnotify"
"github.com/pkg/errors"
"github.com/sirupsen/logrus"
)

// cniNetConfSyncer is used to reload cni network conf triggered by fs change
// events.
type cniNetConfSyncer struct {
// only used for lastSyncStatus
sync.RWMutex
lastSyncStatus error

watcher *fsnotify.Watcher
confDir string
netPlugin cni.CNI
loadOpts []cni.CNIOpt
}

// newCNINetConfSyncer creates cni network conf syncer.
func newCNINetConfSyncer(confDir string, netPlugin cni.CNI, loadOpts []cni.CNIOpt) (*cniNetConfSyncer, error) {
watcher, err := fsnotify.NewWatcher()
if err != nil {
return nil, errors.Wrap(err, "failed to create fsnotify watcher")
}

if err := watcher.Add(confDir); err != nil {
return nil, errors.Wrapf(err, "failed to watch cni conf dir %s", confDir)
}

syncer := &cniNetConfSyncer{
watcher: watcher,
confDir: confDir,
netPlugin: netPlugin,
loadOpts: loadOpts,
}

if err := syncer.netPlugin.Load(syncer.loadOpts...); err != nil {
logrus.WithError(err).Error("failed to load cni during init, please check CRI plugin status before setting up network for pods")
syncer.updateLastStatus(err)
}
return syncer, nil
}

// syncLoop monitors any fs change events from cni conf dir and tries to reload
// cni configuration.
func (syncer *cniNetConfSyncer) syncLoop() error {
for {
select {
case event := <-syncer.watcher.Events:
// Only reload config when receiving write/rename/remove
// events
//
// FIXME(fuweid): Might only reload target cni config
// files to prevent no-ops.
if event.Op&(fsnotify.Chmod|fsnotify.Create) > 0 {
logrus.Debugf("ignore event from cni conf dir: %s", event)
continue
}
logrus.Debugf("receiving change event from cni conf dir: %s", event)

lerr := syncer.netPlugin.Load(syncer.loadOpts...)
if lerr != nil {
logrus.WithError(lerr).
Errorf("failed to reload cni configuration after receiving fs change event(%s)", event)
}
syncer.updateLastStatus(lerr)

case err := <-syncer.watcher.Errors:
if err != nil {
logrus.WithError(err).Error("failed to continue sync cni conf change")
return err
}
}
}
}

// lastStatus retrieves last sync status.
func (syncer *cniNetConfSyncer) lastStatus() error {
syncer.RLock()
defer syncer.RUnlock()
return syncer.lastSyncStatus
}

// updateLastStatus will be called after every single cni load.
func (syncer *cniNetConfSyncer) updateLastStatus(err error) {
syncer.Lock()
defer syncer.Unlock()
syncer.lastSyncStatus = err
}

// stop stops watcher in the syncLoop.
func (syncer *cniNetConfSyncer) stop() error {
return syncer.watcher.Close()
}
24 changes: 22 additions & 2 deletions pkg/server/service.go
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,9 @@ type criService struct {
// initialized indicates whether the server is initialized. All GRPC services
// should return error before the server is initialized.
initialized atomic.Bool
// cniNetConfMonitor is used to reload cni network conf if there is
// any valid fs change events from cni network conf dir.
cniNetConfMonitor *cniNetConfSyncer
}

// NewCRIService returns a new instance of CRIService
Expand Down Expand Up @@ -128,6 +131,11 @@ func NewCRIService(config criconfig.Config, client *containerd.Client) (CRIServi

c.eventMonitor = newEventMonitor(c)

c.cniNetConfMonitor, err = newCNINetConfSyncer(c.config.NetworkPluginConfDir, c.netPlugin, c.cniLoadOptions())
if err != nil {
return nil, errors.Wrap(err, "failed to create cni conf monitor")
}

return c, nil
}

Expand Down Expand Up @@ -169,6 +177,14 @@ func (c *criService) Run() error {
)
snapshotsSyncer.start()

// Start CNI network conf syncer
logrus.Info("Start cni network conf syncer")
cniNetConfMonitorErrCh := make(chan error, 1)
go func() {
defer close(cniNetConfMonitorErrCh)
cniNetConfMonitorErrCh <- c.cniNetConfMonitor.syncLoop()
}()

// Start streaming server.
logrus.Info("Start streaming server")
streamServerErrCh := make(chan error)
Expand All @@ -183,11 +199,12 @@ func (c *criService) Run() error {
// Set the server as initialized. GRPC services could start serving traffic.
c.initialized.Set()

var eventMonitorErr, streamServerErr error
var eventMonitorErr, streamServerErr, cniNetConfMonitorErr error
// Stop the whole CRI service if any of the critical service exits.
select {
case eventMonitorErr = <-eventMonitorErrCh:
case streamServerErr = <-streamServerErrCh:
case cniNetConfMonitorErr = <-cniNetConfMonitorErrCh:
}
if err := c.Close(); err != nil {
return errors.Wrap(err, "failed to stop cri service")
Expand Down Expand Up @@ -222,6 +239,9 @@ func (c *criService) Run() error {
if streamServerErr != nil {
return errors.Wrap(streamServerErr, "stream server error")
}
if cniNetConfMonitorErr != nil {
return errors.Wrap(cniNetConfMonitorErr, "cni network conf monitor error")
}
return nil
}

Expand All @@ -233,7 +253,7 @@ func (c *criService) Close() error {
if err := c.streamServer.Stop(); err != nil {
return errors.Wrap(err, "failed to stop stream server")
}
return nil
return c.cniNetConfMonitor.stop()
}

func (c *criService) register(s *grpc.Server) error {
Expand Down
5 changes: 0 additions & 5 deletions pkg/server/service_unix.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,11 +60,6 @@ func (c *criService) initPlatform() error {
return errors.Wrap(err, "failed to initialize cni")
}

// Try to load the config if it exists. Just log the error if load fails
// This is not disruptive for containerd to panic
if err := c.netPlugin.Load(c.cniLoadOptions()...); err != nil {
logrus.WithError(err).Error("Failed to load cni during init, please check CRI plugin status before setting up network for pods")
}
return nil
}

Expand Down
6 changes: 0 additions & 6 deletions pkg/server/service_windows.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ package server
import (
cni "github.com/containerd/go-cni"
"github.com/pkg/errors"
"github.com/sirupsen/logrus"
)

// windowsNetworkAttachCount is the minimum number of networks the PodSandbox
Expand All @@ -44,11 +43,6 @@ func (c *criService) initPlatform() error {
return errors.Wrap(err, "failed to initialize cni")
}

// Try to load the config if it exists. Just log the error if load fails
// This is not disruptive for containerd to panic
if err := c.netPlugin.Load(c.cniLoadOptions()...); err != nil {
logrus.WithError(err).Error("Failed to load cni during init, please check CRI plugin status before setting up network for pods")
}
return nil
}

Expand Down
10 changes: 6 additions & 4 deletions pkg/server/status.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,10 +41,6 @@ func (c *criService) Status(ctx context.Context, r *runtime.StatusRequest) (*run
Type: runtime.NetworkReady,
Status: true,
}
// Load the latest cni configuration to be in sync with the latest network configuration
if err := c.netPlugin.Load(c.cniLoadOptions()...); err != nil {
log.G(ctx).WithError(err).Errorf("Failed to load cni configuration")
}
// Check the status of the cni initialization
if err := c.netPlugin.Status(); err != nil {
networkCondition.Status = false
Expand Down Expand Up @@ -76,6 +72,12 @@ func (c *criService) Status(ctx context.Context, r *runtime.StatusRequest) (*run
log.G(ctx).WithError(err).Errorf("Failed to marshal CNI config %v", err)
}
resp.Info["cniconfig"] = string(cniConfig)

lastCNILoadStatus := "OK"
if lerr := c.cniNetConfMonitor.lastStatus(); lerr != nil {
lastCNILoadStatus = lerr.Error()
}
resp.Info["lastCNILoadStatus"] = lastCNILoadStatus
}
return resp, nil
}

0 comments on commit 8499fd7

Please sign in to comment.