< home

Kubelet Initialization

Note: This document contains my personal notes from exploring the Kubelet initialization process in Kubernetes. Due to its nature as working notes, it may not follow a strictly formal structure and might contain areas marked for further investigation or clarification. The content is technical and assumes familiarity with Kubernetes internals.

To keep consistency, the code is maintained at …

git log --oneline | head -1
7bfdda4696f Merge pull request #129380 from jdtuhui/testifylint/len@component-base

The Kubelet initialization begins at cmd/kubelet/kubelet.go: 34

// cmd/kubelet/kubelet.go
...
func main() {
	command := app.NewKubeletCommand()
	code := cli.Run(command)
	os.Exit(code)
}

The app.NewKubeletCommand() function, defined at line 136 in cmd/kubelet/app/server.go, sets defaults using options.NewKubeletFlags() and options.NewKubeletConfiguration().

...
func NewKubeletCommand() *cobra.Command {
	cleanFlagSet := pflag.NewFlagSet(server.ComponentKubelet, pflag.ContinueOnError)
	cleanFlagSet.SetNormalizeFunc(cliflag.WordSepNormalizeFunc)
	kubeletFlags := options.NewKubeletFlags()

	kubeletConfig, err := options.NewKubeletConfiguration()
...

Now what are the defaults set in options.NewKubeletFlags()(more or less initialization)?

// cmd/kubelet/app/options/options.go: 139
func NewKubeletFlags() *KubeletFlags {
	return &KubeletFlags{ // declared in cmd/kubelet/app/options/options.go: 54
		ContainerRuntimeOptions: *NewContainerRuntimeOptions(),
		CertDirectory:           "/var/lib/kubelet/pki",
		RootDirectory:           filepath.Clean(defaultRootDir), // /var/lib/kubelet
		MaxContainerCount:       -1,
		MaxPerPodContainerCount: 1,
		MinimumGCAge:            metav1.Duration{Duration: 0},
		RegisterSchedulable:     true,
		NodeLabels:              make(map[string]string),
	}
}
...

DEEPER DIVE

What are the defaults set in NewContainerRuntimeOptions()?

// cmd/kubelet/app/options/container_runtime.go: 36
...
func NewContainerRuntimeOptions() *config.ContainerRuntimeOptions {
	return &config.ContainerRuntimeOptions{ // declared in pkg/kubelet/config/flags.go: 26
		PodSandboxImage: defaultPodSandboxImage, // k8s.gcr.io/pause:3.10
	}
}

/DEEPER DIVE

Now let’s see what options.NewKubeletConfiguration() does.

// cmd/kubelet/app/options/options.go: 200
...
func NewKubeletConfiguration() (*kubeletconfig.KubeletConfiguration, error) {
	scheme, _, err := kubeletscheme.NewSchemeAndCodecs() // NewSchemeAndCodecs is a utility function that returns a Scheme and CodecFactory
	if err != nil {
		return nil, err
	}
	versioned := &v1beta1.KubeletConfiguration{} // Lookup staging/src/k8s.io/kubelet/config/v1beta1/types.go: 89
	scheme.Default(versioned)
	config := &kubeletconfig.KubeletConfiguration{} // Lookup pkg/kubelet/apis/config/types.go: 82
	if err := scheme.Convert(versioned, config, nil); err != nil { // Convert will attempt to convert in into out. Both must be pointers
		return nil, err
	}
	applyLegacyDefaults(config) // applies legacy default values to the KubeletConfiguration
	return config, nil
}

...

The kubelet configuration has these options

DEEPER DIVE

The applyLegacyDefaults function is defined in cmd/kubelet/app/options/options.go: 218.

func applyLegacyDefaults(kc *kubeletconfig.KubeletConfiguration) {
	// --anonymous-auth
	kc.Authentication.Anonymous.Enabled = true
	// --authentication-token-webhook
	kc.Authentication.Webhook.Enabled = false
	// --authorization-mode
	kc.Authorization.Mode = kubeletconfig.KubeletAuthorizationModeAlwaysAllow
	// --read-only-port
	kc.ReadOnlyPort = ports.KubeletReadOnlyPort
}

/DEEPER DIVE

Going further on in function NewKubeletConfiguration(), we would notice that it setup the CLI using Cobra and Pflag.

// cmd/kubelet/app/server.go: 148...
	cmd := &cobra.Command{
		Use: server.ComponentKubelet,
		Long: `The kubelet is the primary "node agent" that runs on each
...

Then the following validates the initial KubeletFlags

// cmd/kubelet/app/server.go: 205
			if err := options.ValidateKubeletFlags(kubeletFlags); err != nil {
				return fmt.Errorf("failed to validate kubelet flags: %w", err)
			}

Then the following loads the configuration file if it is provided in kubeletFlags

// cmd/kubelet/app/server.go: 215
			if len(kubeletFlags.KubeletConfigFile) > 0 { // comes from the `--config` flag
				kubeletConfig, err = loadConfigFile(kubeletFlags.KubeletConfigFile)
				if err != nil {
					return fmt.Errorf("failed to load kubelet config file, path: %s, error: %w", kubeletFlags.KubeletConfigFile, err)
				}
			}

ASIDE

In case EKS Nodes, the kubelet service is managed by systemd, and the Kubelet configuration file is passed as an argument to the –config flag as you can see in the kubelet.service file.

The Kubelet’s configuration is primarily established during node initialization through the bootstrap.sh script.

The bootstrap script configures two critical environment variables for the Kubelet service: KUBELET_ARGS and KUBELET_EXTRA_ARGS. These variables are defined in systemd service configuration files located at /etc/systemd/system/kubelet.service.d/

# https://github.com/awslabs/amazon-eks-ami/blob/3b667df7797233b170bec20033a20a47aaa5d0fc/templates/al2/runtime/bootstrap.sh#L625-L627
# /etc/systemd/system/kubelet.service.d/10-kubelet-args.conf
[Service]
Environment='KUBELET_ARGS=$KUBELET_ARGS'

and

# https://github.com/awslabs/amazon-eks-ami/blob/3b667df7797233b170bec20033a20a47aaa5d0fc/templates/al2/runtime/bootstrap.sh#L630-L635
# /etc/systemd/system/kubelet.service.d/30-kubelet-extra-args.conf
[Service]
Environment='KUBELET_EXTRA_ARGS=$KUBELET_EXTRA_ARGS'

…respectively.

NOTE: On CPU specific nodes, the bootstrap script extends itself, with this.

/ASIDE

Now coming back to the cmd/kubelet/app/server.go file, post setting up of logging the configuration elements are validated.

// cmd/kubelet/app/server.go: 248
			// We always validate the local configuration (command line + config file).
			// This is the default "last-known-good" config for dynamic config, and must always remain valid.
			if err := kubeletconfigvalidation.ValidateKubeletConfiguration(kubeletConfig, utilfeature.DefaultFeatureGate); err != nil {
				return fmt.Errorf("failed to validate kubelet configuration, error: %w, path: %s", err, kubeletConfig)
			}

Then the KubeletServer constructed from the kubeletConfig and kubeletFlags.

// KubeletServer encapsulates all of the parameters necessary for starting up
// a kubelet. These can either be set via command line or directly.
// cmd/kubelet/app/server.go: 259
			kubeletServer := &options.KubeletServer{
				KubeletFlags:         *kubeletFlags,
				KubeletConfiguration: *kubeletConfig,
			}

Then kubeletServer is used to construct the default KubeletDeps

// cmd/kubelet/app/server.go: 265
			kubeletDeps, err := UnsecuredDependencies(kubeletServer, utilfeature.DefaultFeatureGate)

DEEPER DIVE

The UnsecuredDependencies function is defined in cmd/kubelet/app/server.go: 275.

// UnsecuredDependencies returns a Dependencies suitable for being run, or an error if the server setup
// is not valid.  It will not start any background processes, and does not include authentication/authorization
func UnsecuredDependencies(s *options.KubeletServer, featureGate featuregate.FeatureGate) (*kubelet.Dependencies, error) {
	// Initialize the TLS Options
	tlsOptions, err := InitializeTLS(&s.KubeletFlags, &s.KubeletConfiguration)
	...
	return &kubelet.Dependencies{
		...
		TLSOptions:          tlsOptions}, nil
}

The kubeDeps are runtime dependencies that are necessary for the Kubelet to run, the struct is define in pkg/kubelet/kubelet.go: 287.

type Dependencies struct {
	Options []Option

	// Injected Dependencies
	Auth                      server.AuthInterface
	CAdvisorInterface         cadvisor.Interface
	Cloud                     cloudprovider.Interface
	ContainerManager          cm.ContainerManager
	EventClient               v1core.EventsGetter
	HeartbeatClient           clientset.Interface
	OnHeartbeatFailure        func()
	KubeClient                clientset.Interface
	Mounter                   mount.Interface
	HostUtil                  hostutil.HostUtils
	OOMAdjuster               *oom.OOMAdjuster
	OSInterface               kubecontainer.OSInterface
	PodConfig                 *config.PodConfig
	ProbeManager              prober.Manager
	Recorder                  record.EventRecorder
	Subpather                 subpath.Interface
	TracerProvider            trace.TracerProvider
	VolumePlugins             []volume.VolumePlugin
	DynamicPluginProber       volume.DynamicPluginProber
	TLSOptions                *server.TLSOptions
	RemoteRuntimeService      internalapi.RuntimeService
	RemoteImageService        internalapi.ImageManagerService
	PodStartupLatencyTracker  util.PodStartupLatencyTracker
	NodeStartupLatencyTracker util.NodeStartupLatencyTracker
	// remove it after cadvisor.UsingLegacyCadvisorStats dropped.
	useLegacyCadvisorStats bool
}

This sets up the TLS for the Kubelet, using the InitializeTLS function.

// InitializeTLS checks for a configured TLSCertFile and TLSPrivateKeyFile: if unspecified a new self-signed
// certificate and key file are generated. Returns a configured server.TLSOptions object.
// cmd/kubelet/app/server.go: 1124
func InitializeTLS(kf *options.KubeletFlags, kc *kubeletconfiginternal.KubeletConfiguration) (*server.TLSOptions, error) {
	if !kc.ServerTLSBootstrap && kc.TLSCertFile == "" && kc.TLSPrivateKeyFile == "" {
		kc.TLSCertFile = filepath.Join(kf.CertDirectory, "kubelet.crt")
		kc.TLSPrivateKeyFile = filepath.Join(kf.CertDirectory, "kubelet.key")
		...
	}
	...
	tlsOptions := &server.TLSOptions{
		Config: &tls.Config{
			MinVersion:   minTLSVersion,
			CipherSuites: tlsCipherSuites,
		},
		CertFile: kc.TLSCertFile,
		KeyFile:  kc.TLSPrivateKeyFile,
	}

	if len(kc.Authentication.X509.ClientCAFile) > 0 {
		clientCAs, err := certutil.NewPool(kc.Authentication.X509.ClientCAFile)
		if err != nil {
			return nil, fmt.Errorf("unable to load client CA file %s: %w", kc.Authentication.X509.ClientCAFile, err)
		}
		// Specify allowed CAs for client certificates
		tlsOptions.Config.ClientCAs = clientCAs
		// Populate PeerCertificates in requests, but don't reject connections without verified certificates
		tlsOptions.Config.ClientAuth = tls.RequestClientCert
	}

	return tlsOptions, nil
}

Since in EKS default configuration, the ServerTLSBootstrap is set to true, the Kubelet will request a certificate from the certificates.k8s.io API. This requires an approver to approve the certificate signing requests. The RotateKubeletServerCertificate feature must be enabled.

/DEEPER DIVE

The kubelet must always with uid 0, and this is checked here.

// cmd/kubelet/app/server.go: 270
			if err := checkPermissions(); err != nil {
				klog.ErrorS(err, "kubelet running with insufficient permissions")
			}

Before eventually starting the Kubelet server, the singnal handlers are setup and feature enablement metrics is setup.

// cmd/kubelet/app/server.go: 283
			ctx := genericapiserver.SetupSignalContext()

			utilfeature.DefaultMutableFeatureGate.AddMetrics()

The following section will go into the details Run function of the called from cmd/kubelet/app/server.go:287.

Run calls run

Run runs the specified KubeletServer with the given Dependencies. This should never exit. The kubeDeps argument may be nil - if so, it is initialized from the settings on KubeletServer. Otherwise, the caller is assumed to have set up the Dependencies object and a default one will not be generated.

The Run fuction actually calls the run` function.

// cmd/kubelet/app/server.go: 287
			if err := run(ctx, s, kubeDeps, featureGate); err != nil {
				return fmt.Errorf("failed to run Kubelet: %w", err)
			}

So the run function is defined in cmd/kubelet/app/server.go: 292, which does the following as it builds up to creating the Kubelet server.

  • cmd/kubelet/app/server.go: 599 - Validate the KubeletServer by calling the ValidateKubeletServer
  • cmd/kubelet/app/server.go: 627 - Register the current configuration to be exposed as /configz by calling initConfigz function
  • cmd/kubelet/app/server.go: 633 - Register the current configuration to be exposed as /configz by calling initConfigz function
  • cmd/kubelet/app/server.go: 674 - Sets up kubeDeps.KubeClient, kubeDeps.EventClient and kubeDeps.HeartbeatClient, if any of them is nil.
  • cmd/kubelet/app/server.go: 715 - Creates an authenticator, an authorizer, and a matching authorizer attributes getter compatible with the kubelet’s needs, by calling BuildAuth, if kubeDeps.Auth is nil.
  • cmd/kubelet/app/server.go: 723 - Ensure the container runtime is up and running and sets up the KubeDeps references to those by calling the PreInitRuntimeService function.

DEEPER DIVE

The PreInitRuntimeService function is defined in pkg/kubelet/kubelet.go: 359. This function in the Kubelet is responsible for initializing the container runtime service before the main kubelet initialization.

// pkg/kubelet/kubelet.go: 360
remoteImageEndpoint := kubeCfg.ImageServiceEndpoint
if remoteImageEndpoint == "" && kubeCfg.ContainerRuntimeEndpoint != "" {
    remoteImageEndpoint = kubeCfg.ContainerRuntimeEndpoint
}

Sets up image service endpoint, and falls back to container runtime endpoint if image endpoint is not specified.

// pkg/kubelet/kubelet.go: 372
	if kubeDeps.RemoteRuntimeService, err = remote.NewRemoteRuntimeService(
	    kubeCfg.ContainerRuntimeEndpoint,
	    kubeCfg.RuntimeRequestTimeout.Duration,
	    tp,
	    &logger); err != nil {
	    return err
	}

Creates connection to container runtime (e.g., containerd), sets up runtime request timeout, integrates with tracing, and sets up logging.

This function is crucial as it sets up the core services needed for container and image operations before the main kubelet initialization.

/DEEPER DIVE

Coming back to the run function in cmd/kubelet/app/server.go: 735, this sets cgroupRoot to /.

    nodeAllocatableRoot := cm.NodeAllocatableRoot(s.CgroupRoot, s.CgroupsPerQOS, s.CgroupDriver)

Internally the nodeAllocatableRoot is set to / if s.CgroupRoot is empty.

Coming back to the run function in cmd/kubelet/app/server.go: 727, the following happens…

// Get cgroup driver setting from CRI
	if utilfeature.DefaultFeatureGate.Enabled(features.KubeletCgroupDriverFromCRI) {
		if err := getCgroupDriverFromCRI(ctx, s, kubeDeps); err != nil {
			return err
		}
	}

This feature behind a feature gate. And the kubelet uses the Cgroup driver from the CRI, if the feature is enabled. The current Cgroup driver is set in the kubeletServer.CgroupDriver field. And if it is linux is could be on these two values systemd or cgroupfs, and kubeDeps.RemoteRuntimeService is used to get the Cgroup driver from the CRI.

JUST NOTE

There is a utility function called the ToSystemd and here is a snippet, the comments explain it well.

// cgroupName.ToSystemd converts the internal cgroup name to a systemd name.
// For example, the name {"kubepods", "burstable", "pod1234-abcd-5678-efgh"} becomes
// "/kubepods.slice/kubepods-burstable.slice/kubepods-burstable-pod1234_abcd_5678_efgh.slice"
// This function always expands the systemd name into the cgroupfs form. If only
// the last part is needed, use path.Base(...) on it to discard the rest.
func (cgroupName CgroupName) ToSystemd() string {
	if len(cgroupName) == 0 || (len(cgroupName) == 1 && cgroupName[0] == "") {
		return "/"
	}
	newparts := []string{}
	for _, part := range cgroupName {
		part = escapeSystemdCgroupName(part)
		newparts = append(newparts, part)
	}

	result, err := cgroupsystemd.ExpandSlice(strings.Join(newparts, "-") + systemdSuffix)
	if err != nil {
		// Should never happen...
		panic(fmt.Errorf("error converting cgroup name [%v] to systemd format: %v", cgroupName, err))
	}
	return result
}

/JUST NOTE

Then the CAdvisor interface is initialized and asssigned to kubeDeps.CAdvisorInterface.

// cmd/kubelet/app/server.go: 754
    if kubeDeps.CAdvisorInterface == nil {
        ...
        kubeDeps.CAdvisorInterface = cadvisor.New(...
    }

The kubelet requires the cAdvisor interface for several critical monitoring and resource tracking functions. If the interface is not provided, the kubelet will create a new cAdvisor interface. The interface provides container metrics, machine metrics, and other monitoring data, which include:

  • Container metrics: CPU, memory, filesystem, and network usage
  • Machine metrics: CPU, memory, filesystem, and network usage
  • Other monitoring data: Container and machine events, container and machine specifications

Then the kubeDeps.Recorder is initialized at the cmd/kubelet/app/server.go: 763, with a call to makeEventRecorder.

Then the kubeDeps.ContainerManager is initialized at the cmd/kubelet/app/server.go: 840, is it is nil. But before the following values are updates…

  • The reservations for resources such as Kubelet and System are set
  • The eviction thresholds are set. (defaults exists in pkg/kubelet/eviction/defaults_linux.go: 23.)
  • The cpuManagerPolicy is set to None if it is not set.
  • The topologyManagerPolicy is set to None if it is not set.
  • The nodeSwap is set if the experimental feature is enabled and the node is enabled cgroupv2
// cmd/kubelet/app/server.go: 765
    if kubeDeps.ContainerManager == nil {
        ...
        // line: 840
        kubeDeps.ContainerManager, err = cm.NewContainerManager(...
    }

DEEPER DIVE

The NewContainerManager function is defined in pkg/kubelet/cm/container_manager.go: 47. This function is responsible for creating a new container manager, which is the core component of the kubelet that manages containers and their lifecycle.

  • Get the cgroup subsystem v1/v2 mode
  • Check if swap is on
  • Construcrt the capacity on the node the kubelet is dealing with from the MachineInfo(vendor/github.com/google/cadvisor/info/v1/machine.go: 180)
// pkg/kubelet/cm/container_manager_linux.go: 239
	capacity := cadvisor.CapacityFromMachineInfo(machineInfo)
	for k, v := range capacity {
		internalCapacity[k] = v
	}
  • Set the pid limits to the internal capacity
// pkg/kubelet/cm/container_manager_linux.go: 243
	pidlimits, err := pidlimit.Stats() // pkg/kubelet/stats/pidlimit/pidlimit_linux.go: 35
	if err == nil && pidlimits != nil && pidlimits.MaxPID != nil {
		internalCapacity[pidlimit.PIDs] = *resource.NewQuantity(
			int64(*pidlimits.MaxPID),
			resource.DecimalSI)
	}
  • Initializes the cgroup hierarchy by:
    • Configuring the cgroupRoot directory
    • Creating a cgroupManager factory with the appropriate subsystem (cgroup v1 or v2)
    • Specifying the cgroup driver implementation (systemd or cgroupfs)
  • Updates the nodeConfig.cgroupVersion to reflect the active cgroup version from the manager
// pkg/kubelet/cm/container_manager_linux.go: 251
    cgroupRoot := ParseCgroupfsToCgroupName(nodeConfig.CgroupRoot)
	cgroupManager := NewCgroupManager(subsystems, nodeConfig.CgroupDriver)
	nodeConfig.CgroupVersion = cgroupManager.Version()
  • Based on CgroupsPerQOS, the new cgroupRoot is set to kubepods
// pkg/kubelet/cm/container_manager_linux.go: 255
	if nodeConfig.CgroupsPerQOS { // default is true staging/src/k8s.io/kubelet/config/v1beta1/types.go: 346
	   ...
		// line: 271
		cgroupRoot = NewCgroupName(cgroupRoot, defaultNodeAllocatableCgroupName) // defaultNodeAllocatableCgroupName = "kubepods"
	}
  • The new QOSContainerManager is initialized into which implements qosContainerManagerImpl
// pkg/kubelet/cm/container_manager_linux.go: 275
qosContainerManager, err := NewQOSContainerManager(subsystems, cgroupRoot, nodeConfig, cgroupManager)
  • Create a new containerMnaagerImpl and set the qosContainerManager and cgroupManager on it
// pkg/kubelet/cm/container_manager_linux.go: 280
	cm := &containerManagerImpl{
		cadvisorInterface:   cadvisorInterface,
		mountUtil:           mountUtil,
		NodeConfig:          nodeConfig,
		subsystems:          subsystems,
		cgroupManager:       cgroupManager,
		capacity:            capacity,
		internalCapacity:    internalCapacity,
		cgroupRoot:          cgroupRoot,
		recorder:            recorder,
		qosContainerManager: qosContainerManager,
	}

  • Assigned the instance of topologyManager to cm.topologyManager.
// pkg/kubelet/cm/container_manager_linux.go: 293
	cm.topologyManager, err = topologymanager.NewManager(
		machineInfo.Topology,
		nodeConfig.TopologyManagerPolicy,
		nodeConfig.TopologyManagerScope,
		nodeConfig.TopologyManagerPolicyOptions,
	)
  • A new instance of the devicemanager.MangerImpl is created and assigned to cm.deviceManager
// pkg/kubelet/cm/container_manager_linux.go: 305
	cm.deviceManager, err = devicemanager.NewManagerImpl(machineInfo.Topology, cm.topologyManager)
  • The Dynamic Resource Allocation is new feature in Kubernetes, and is behind a feature gate, and disabled by default.
// pkg/kubelet/cm/container_manager_linux.go: 312
	if utilfeature.DefaultFeatureGate.Enabled(kubefeatures.DynamicResourceAllocation) {
		klog.InfoS("Creating Dynamic Resource Allocation (DRA) manager")
		cm.draManager, err = dra.NewManagerImpl(kubeClient, nodeConfig.KubeletRootDir, nodeConfig.NodeName)
		if err != nil {
			return nil, err
		}
	}
  • The passed argument kubeClient is assigned to cm.kubeClient(pkg/kubelet/cm/container_manager_linux.go: 319)
  • The instancce of cpuManager is created and assigned to cm.cpuManager
  • The instance of memoryManager is created and assigned to cm.memoryManager
	// Initialize CPU manager
	cm.cpuManager, err = cpumanager.NewManager(
		...
	)
	...

	cm.memoryManager, err = memorymanager.NewManager(
		...
	)
	...
  • You will notice that each of these managers, topologyManager, deviceManager, cpuManager, memoryManager, have a method to initialize HintProvider, which is a function that returns a HintProvider interface. The HintProvider interface is used to provide hints to the kubelet about the resources that are available on the node, and play a significant role in resource allocation in NUMA locality and topology aware resource allocation.

/DEEPER DIVE

Now back to the lines after the call to NewContainerManager in cmd/kubelet/app/server.go, two kudeDeps variables PodStartupLatencyTracker and NodeStartupLatencyTracker are intitialized.

...
	// cmd/kubelet/app/server.go: 887
	kubeDeps.PodStartupLatencyTracker = kubeletutil.NewPodStartupLatencyTracker()
	// >>> pkg/kubelet/kubeletutil/pod_startup_latency_tracker.go:60
	// >>> returns an instance to struct `basicPodStartupLatencyTracker`
	// >>> pkg/kubelet/kubeletutil/pod_startup_latency_tracker.go:40
	// >>> struct `basicPodStartupLatencyTracker` implements `PodStartupLatencyTracker`
	// >>> pkg/kubelet/kubeletutil/pod_startup_latency_tracker.go:50
	// >>> struct `basicPodStartupLatencyTracker` has `perPodState` map with...
	// >>> ... `firstStartedPulling`, `lastFinishedPulling`, `observedRunningTime`
	// cmd/kubelet/app/server.go: 891
	kubeDeps.NodeStartupLatencyTracker = kubeletutil.NewNodeStartupLatencyTracker()
	// >>> pkg/kubelet/kubeletutil/node_startup_latency_tracker.go:49
	// >>> returns an instance to struct `basicNodeStartupLatencyTracker`
	// >>> `bascicNodeStartupLatencyTracker` implements `NodeStartupLatencyTracker`
	// >>> pkg/kubelet/kubeletutil/node_startup_latency_tracker.go:36
	// >>> struct `basicNodeStartupLatencyTracker` has  node `bootTime`, ...
	// >>> ... `kubeletStartTime`, `firstRegistrationAttemptTime`, ...
	// >>> ... `firstRegisteredNewNodeTime`, `firstNodeReadyTime`
...

The PodStartupLatencyTracker is used to track the startup latency of pods, and the NodeStartupLatencyTracker is used to track the startup latency of nodes. Also note the bootTime is got from btime in /proc/stat.

Then the kubelet’s OOM Score is adjusted based on what is provided in tghe kubelet configuration for the oomScoreAdj field(default: -999), by calling the ApplyOOMScoreAdj function(cmd/kubelet/app/server.go: 895). pkg/util/oom/oom_linux.go: 51 implements applyOOMScoreAdj which is used to set the OOM score of a process.

NOTE: The highly negative score means the Kubelet is very unlikely to be killed by the Linux OOM killer,

The next call is to RunKubelet, passing it the Context(ctx), KubeletServer(s), and Dependencies(kubeDeps) instances as arguments at cmd/kubelet/app/server.go: 900.

DEEPER DIVE

RunKubelet is responsible for setting up and running a kubelet, and defined at cmd/kubelet/app/server.go: 1227. It takes the ctx.Context, options.KubeletServer and kubelet.Dependencies as arguments. And is the sequences of calls that lead to the actual running of the kubelet.

  • Set thhe hostname to OS given hostname of no s.hostnameOverride is specified.
  • Set the nodeName to cloud provider given name or else default to hostname.
  • Set the boolean flag hostnameOverridden bases on the s.hostnameOverride being set.
  • Setup the EventRecorder by calling makeEventRecorder. makeEventRecorder sets up kubeDeps.Recorder if it’s nil. It’s a no-op otherwise. cmd/kubelet/app/server.go: 555
  • Set the nodeIPs array by parsing the s.NodeIP and s.CloudProvider. cmd/kubelet/app/server.go: 1241
  • Initialize the capabilities to AllowPrivileged to true.
  • Set the preferred docker configuration path to s.RootDirectory, by calling credentialprovider.SetPreferredDockercfgPath.
  • Set the kubeDeps.OSInterface to an instance of RealOS struct, which implements the OSInterface interface. pkg/kubelet/container/os.go: 27

The next call is to createAndInitKubelet at cmd/kubelet/app/server.go: 1260, which is responsible for creating and initializing the kubelet. The createAndInitKubelet function is defined at cmd/kubelet/app/server.go: 1301. It takes s(instance of options.KubeletServer),kubeDeps(instance of kubelet.Dependencies), hostname, hostnameOverridden, nodeName, nodeIPs, as arguments, and returns an instance of kubelet.Bootstrap and an error. The createAndInitKubelet function in turn calls….

// cmd/kubelet/app/server.go: 1301
	k, err = kubelet.NewMainKubelet(&kubeServer.KubeletConfiguration,
		kubeDeps,
		&kubeServer.ContainerRuntimeOptions,
		hostname,
		hostnameOverridden,
		nodeName,
		nodeIPs,
		kubeServer.ProviderID,
		kubeServer.CloudProvider,
		kubeServer.CertDirectory,
		kubeServer.RootDirectory,
		kubeServer.PodLogsDir,
		kubeServer.ImageCredentialProviderConfigFile,
		kubeServer.ImageCredentialProviderBinDir,
		kubeServer.RegisterNode,
		kubeServer.RegisterWithTaints,
		kubeServer.AllowedUnsafeSysctls,
		kubeServer.ExperimentalMounterPath,
		kubeServer.KernelMemcgNotification,
		kubeServer.ExperimentalNodeAllocatableIgnoreEvictionThreshold,
		kubeServer.MinimumGCAge,
		kubeServer.MaxPerPodContainerCount,
		kubeServer.MaxContainerCount,
		kubeServer.RegisterSchedulable,
		kubeServer.NodeLabels,
		kubeServer.NodeStatusMaxImages,
		kubeServer.KubeletFlags.SeccompDefault || kubeServer.KubeletConfiguration.SeccompDefault)

Now lets dive into what happens in kubelet.NewMainKubelet.

DEEPER DIVE

The NewMainKubelet is defined in pkg/kubelet/kubelet.go: 386. NewMainKubelet instantiates a new Kubelet object along with all the required internal modules. No initialization of Kubelet and its modules happen here.

  • At this point the following things need to set for the kubelet to be initialized:
    • rootDirectory(default /var/lib/kubelet, cmd/kubelet/app/options/options.go: 44)
    • podLogsDir (default /var/log/pods, pkg/kubelet/apis/config/types.go: 92)
    • kubeCfg.SyncFrequency.Duration is greater than 0.
    • The cloudProvider must external if not empty.
      1. IMPORTANT: Check if kubelet is running in standalone mode or cluster mode, checking the nil-ness of kubeDeps.KubeClient. If it is nil, then it is running in standalone mode. Otherwise it is running in cluster mode. In cluster mode is sets up kubeInformers, and starts the sync with the API server.
      2. IMPORTANT: Sets up the kubeDeps.PodConfig(pkg/kubelet/kubelet.go: 457) instance, helps the kubelet to with the following functions:
    • Receive pod configurations from various sources
    • Merge these configurations consistently
    • Track changes to pod configurations
    • Notify components about pod configuration changes
    • Maintain pod state synchronization
    • NOTE: This object in-memory “source of truth” for pod configurations Lets dived into this a bit, and look at the makePodSourceConfig function that is defined in pkg/kubelet/kubelet.go: 323.

DEEPER DIVE

The makePodSourceConfig function establishes the various sources from which the Kubelet can receive Pod configurations. Here’s a detailed breakdown of its functionality:

  • The function takes several parameters:
    • kubeCfg: Kubelet configuration settings
    • kubeDeps: Dependencies required by kubelet
    • nodeName: The name of the node
    • nodeHasSynced: A function that checks if the node has synced
  • The function starts by setting up HTTP headers for static pod URLs:
    manifestURLHeader := make(http.Header)
    if len(kubeCfg.StaticPodURLHeader) > 0 {
      for k, v := range kubeCfg.StaticPodURLHeader {
          for i := range v {
              manifestURLHeader.Add(k, v[i])
          }
      }
    }
    
  • It creates a new PodConfig object that will handle pod configuration notifications:
    cfg := config.NewPodConfig(config.PodConfigNotificationIncremental, kubeDeps.Recorder, kubeDeps.PodStartupLatencyTracker)
    

    This creates an in memory “source of truth” for pod configurations, which will be used by the kubelet to monitor and apply pod configurations.

  • The function then sets up three possible sources for pod configurations:

    • File Source - for static pods defined in files:
       if kubeCfg.StaticPodPath != "" {
         config.NewSourceFile(kubeCfg.StaticPodPath, nodeName, kubeCfg.FileCheckFrequency.Duration, cfg.Channel(ctx, kubetypes.FileSource))
       }
      

    b. URL Source - for static pods defined via HTTP:

     if kubeCfg.StaticPodURL != "" {
         config.NewSourceURL(kubeCfg.StaticPodURL, manifestURLHeader, nodeName, kubeCfg.HTTPCheckFrequency.Duration, cfg.Channel(ctx, kubetypes.HTTPSource))
     }
    

    c. API Server Source - for pods managed through the Kubernetes API:

     if kubeDeps.KubeClient != nil {
         config.NewSourceApiserver(kubeDeps.KubeClient, nodeName, nodeHasSynced, cfg.Channel(ctx, kubetypes.ApiserverSource))
     }
    

This function is crucial for Kubernetes node operation as it establishes all the ways a node can receive pod configurations:

  • Through static files on disk
  • Through HTTP endpoints
  • Through the Kubernetes API server

Each source is configured with its own update frequency and communication channel. The function returns the configured PodConfig object which will be used by the kubelet to monitor and apply pod configurations from all these sources.

/DEEPER DIVE

  • Create an instance of kubecontainer.GCPolicy at pkg/kubelet/kubelet.go: 463, which is used to manage the garbage collection of dead containers.
  • IMPORTANT: Create an instance of v1.NodeDaemonEndpoints at pkg/kubelet/kubelet.go: 469, which is used to store the daemon endpoints of the node. Node Daemon Endpoints are specific endpoints that represent services running on a node in a Kubernetes cluster. They are defined by the NodeDaemonEndpoints struct, which currently contains a single endpoint type, KubeletEndpoint. This represents the endpoint where the Kubelet is listening. The Kubelet is the primary node agent that runs on each node in the cluster.
  • Create an instance of images.ImageGCPolicy at pkg/kubelet/kubelet.go: 473, which is used to manage the garbage collection of unused images on the node.
  • If the ImageMaximumGCAge feature is enabled, set the imageGcPolicy.MaxAge.Duration to the value of kubeCfg.ImageGcMaximumAge.Duration.
  • Create an instance of kubelet.LifecycleHandler at pkg/kubelet/kubelet.go: 477, which is used to handle the lifecycle of pods.
  • kubeCfg.EnforceNodeAllocatable is passed into a variable called enforceNodeAllocatable at pkg/kubelet/kubelet.go: 485, which is used to determine if the node allocatable should be enforced. This will be an empty array if we are not enforcing Evictions.
  • thresholds is set based on the parsing of the Node Eviction Thresholds in pkg/kubelet/kubelet.go: 488. These thresholds are critical for:
    • Maintaining node stability
    • Preventing out-of-resource scenarios
    • Ensuring system processes have enough resources
    • Managing pod evictions in a controlled manner
  • IMPORTANT: An instance of eviction.Config is created at pkg/kubelet/kubelet.go: 493, which is used to configure the eviction manager.
	// pkg/kubelet/eviction/types.go: 46
	type Config struct {
		// PressureTransitionPeriod is duration the kubelet has to wait before transitioning out of a pressure condition.
		PressureTransitionPeriod time.Duration
		// Maximum allowed grace period (in seconds) to use when terminating pods in response to a soft eviction threshold being met.
		MaxPodGracePeriodSeconds int64
		// Thresholds define the set of conditions monitored to trigger eviction.
		Thresholds []evictionapi.Threshold
		// KernelMemcgNotification if true will integrate with the kernel memcg notification to determine if memory thresholds are crossed.
		KernelMemcgNotification bool
		// PodCgroupRoot is the cgroup which contains all pods.
		PodCgroupRoot string
	}
  • Instance of corelisters.ServiceLister and cache.InformerSynced is created to list and and be informed about services ion the cluster.
	// pkg/kubelet/kubelet.go: 502
		var serviceLister corelisters.ServiceLister
		var serviceHasSynced cache.InformerSynced
		if kubeDeps.KubeClient != nil {
			// don't watch headless services, they are not needed since this informer is only used to create the environment variables for pods.
			// See https://issues.k8s.io/122394
			kubeInformers := informers.NewSharedInformerFactoryWithOptions(kubeDeps.KubeClient, 0, informers.WithTweakListOptions(func(options *metav1.ListOptions) {
				options.FieldSelector = fields.OneTermNotEqualSelector("spec.clusterIP", v1.ClusterIPNone).String()
			}))
			serviceLister = kubeInformers.Core().V1().Services().Lister()
			serviceHasSynced = kubeInformers.Core().V1().Services().Informer().HasSynced
			kubeInformers.Start(wait.NeverStop)
		} else {
			serviceIndexer := cache.NewIndexer(cache.MetaNamespaceKeyFunc, cache.Indexers{cache.NamespaceIndex: cache.MetaNamespaceIndexFunc})
			serviceLister = corelisters.NewServiceLister(serviceIndexer)
			serviceHasSynced = func() bool { return true }
		}
  • Create a node reference object nodeRef of v1.ObjectReference at pkg/kubelet/kubelet.go: 520, this will be used for events.
  • Create a OOMWatcher instance oomWatcher at pkg/kubelet/kubelet.go: 527, by calling oomwatcher.NewOOMWatcher which is used to monitor the OOM events on the node.
  • Create an string slice clusterDNS at pkg/kubelet/kubelet.go: 544, and populate it based on the kubeCfg.ClusterDNS ip addresses.
  • IMPORTANT: Create an insecure http client instance insecureContainerLifecycleHTTPClient at pkg/kubelet/kubelet.go: 559, which is used to make HTTP requests to the container lifecycle handler. The comments here are interesting:
	// A TLS transport is needed to make HTTPS-based container lifecycle requests,
	// but we do not have the information necessary to do TLS verification.
	//
	// This client must not be modified to include credentials, because it is
	// critical that credentials not leak from the client to arbitrary hosts.
  • An instance of kubeDeps.TracerProvider.Tracer is created, tracer, at pkg/kubelet/kubelet.go: 566.
  • The klet an instance of the Kubelet struct is created at pkg/kubelet/kubelet.go: 571. Here is the code…
klet := &Kubelet{
	hostname:                       hostname,
	hostnameOverridden:             hostnameOverridden,
	nodeName:                       nodeName,
	kubeClient:                     kubeDeps.KubeClient,
	heartbeatClient:                kubeDeps.HeartbeatClient,
	onRepeatedHeartbeatFailure:     kubeDeps.OnHeartbeatFailure,
	rootDirectory:                  filepath.Clean(rootDirectory),
	podLogsDirectory:               podLogsDirectory,
	resyncInterval:                 kubeCfg.SyncFrequency.Duration,
	sourcesReady:                   config.NewSourcesReady(kubeDeps.PodConfig.SeenAllSources),
	registerNode:                   registerNode,
	registerWithTaints:             registerWithTaints,
	registerSchedulable:            registerSchedulable,
	dnsConfigurer:                  dns.NewConfigurer(kubeDeps.Recorder, nodeRef, nodeIPs, clusterDNS, kubeCfg.ClusterDomain, kubeCfg.ResolverConfig),
	serviceLister:                  serviceLister,
	serviceHasSynced:               serviceHasSynced,
	nodeLister:                     nodeLister,
	nodeHasSynced:                  nodeHasSynced,
	streamingConnectionIdleTimeout: kubeCfg.StreamingConnectionIdleTimeout.Duration,
	recorder:                       kubeDeps.Recorder,
	cadvisor:                       kubeDeps.CAdvisorInterface,
	cloud:                          kubeDeps.Cloud,
	externalCloudProvider:          cloudprovider.IsExternal(cloudProvider),
	providerID:                     providerID,
	nodeRef:                        nodeRef,
	nodeLabels:                     nodeLabels,
	nodeStatusUpdateFrequency:      kubeCfg.NodeStatusUpdateFrequency.Duration,
	nodeStatusReportFrequency:      kubeCfg.NodeStatusReportFrequency.Duration,
	os:                             kubeDeps.OSInterface,
	oomWatcher:                     oomWatcher,
	cgroupsPerQOS:                  kubeCfg.CgroupsPerQOS,
	cgroupRoot:                     kubeCfg.CgroupRoot,
	mounter:                        kubeDeps.Mounter,
	hostutil:                       kubeDeps.HostUtil,
	subpather:                      kubeDeps.Subpather,
	maxPods:                        int(kubeCfg.MaxPods),
	podsPerCore:                    int(kubeCfg.PodsPerCore),
	syncLoopMonitor:                atomic.Value{},
	daemonEndpoints:                daemonEndpoints,
	containerManager:               kubeDeps.ContainerManager,
	nodeIPs:                        nodeIPs,
	nodeIPValidator:                validateNodeIP,
	clock:                          clock.RealClock{},
	enableControllerAttachDetach:   kubeCfg.EnableControllerAttachDetach,
	makeIPTablesUtilChains:         kubeCfg.MakeIPTablesUtilChains,
	nodeStatusMaxImages:            nodeStatusMaxImages,
	tracer:                         tracer,
	nodeStartupLatencyTracker:      kubeDeps.NodeStartupLatencyTracker,
}

TODO: Annotate each of the values of those variables in the Kubelet struct with references to where they are defined.

  • If klet.Cloud is not nil, the klet.CloudResourceSyncManager is set to a new instance of cloudprovider.CloudResourceSyncManager at pkg/kubelet/kubelet.go: 620.
  • IMPORTANT: klet.secretManager and klet.configMapManager are set. As you can see below
    // pkg/kubelet/kubelet.go: 623
    var secretManager secret.Manager
		var configMapManager configmap.Manager
		if klet.kubeClient != nil {
			switch kubeCfg.ConfigMapAndSecretChangeDetectionStrategy {
			case kubeletconfiginternal.WatchChangeDetectionStrategy:
				secretManager = secret.NewWatchingSecretManager(klet.kubeClient, klet.resyncInterval)
				configMapManager = configmap.NewWatchingConfigMapManager(klet.kubeClient, klet.resyncInterval)
			case kubeletconfiginternal.TTLCacheChangeDetectionStrategy:
				secretManager = secret.NewCachingSecretManager(
					klet.kubeClient, manager.GetObjectTTLFromNodeFunc(klet.GetNode))
				configMapManager = configmap.NewCachingConfigMapManager(
					klet.kubeClient, manager.GetObjectTTLFromNodeFunc(klet.GetNode))
			case kubeletconfiginternal.GetChangeDetectionStrategy:
				secretManager = secret.NewSimpleSecretManager(klet.kubeClient)
				configMapManager = configmap.NewSimpleConfigMapManager(klet.kubeClient)
			default:
				return nil, fmt.Errorf("unknown configmap and secret manager mode: %v", kubeCfg.ConfigMapAndSecretChangeDetectionStrategy)
			}

			klet.secretManager = secretManager
			klet.configMapManager = configMapManager
		}

And by default kubeCfg.ConfigMapAndSecretChangeDetectionStrategy is set to Watch.

  • An instance of MachineInfo is created, machineInfo, at pkg/kubelet/kubelet.go: 646, which is used to store the machine information. And the machineInfo is assigned klet.machineInfo.
  • An instance of BackOff is created, imageBackOff, at pkg/kubelet/kubelet.go: 655, which is used to manage the backoff for image pull failures. This variable will be passed to the container runtime.
  • Manager(s) for the liveness, readiness and startup probes is created into klet.livenessManager, klet.readinessManager, and klet.startupManager respectively at pkg/kubelet/kubelet.go: 657-659 by call proberesults.NewManager.
  • An instance of cache is created, klet.podCache, at pkg/kubelet/kubelet.go: 660, which is used to store the pod information.
  • An instance of MirrorClient(pkg/kubelet/pod/mirror_client.go: 34) interface is created into klet.mirrorPodClient by calling kubepod.NewBasicMirrorClient at pkg/kubelet/kubelet.go: 662, which is used to manage the pod mirroring of Kubelet’s static pods. MirrorClient knows how to create/delete a mirror pod in the API server.

DEFINITION

Mirror pods in Kubernetes are special pod objects that exist for static pods. Here’s how they work:

  • Static Pod -> Mirror Pod Relationship:
    • When you create a static pod (by placing a pod manifest in the node’s static pod directory)
    • The kubelet automatically creates a mirror pod on the API server
    • The mirror pod is essentially a read-only copy/representation of the static pod
  • Key Characteristics:
    • Mirror pods are only visible on the API server (you can see them with kubectl)
    • You cannot modify mirror pods directly through the API server
    • They have the same name as the static pod but are bound to the specific node
    • The kubelet adds a label kubernetes.io/config.mirror to identify them
  • Purpose:
    • They make static pods visible to the control plane
    • Allow monitoring and visibility of static pods through standard Kubernetes tools
    • Help in tracking the status of static pods at the cluster level
  • Example Use Case:
    • Common for system components like the kubelet itself
    • Often used for monitoring agents or networking plugins that need to run on every node
    • Useful when you need pods that must exist even if the API server is down

Think of mirror pods as the API server’s “window” into static pods running on nodes. They let you see what static pods exist, but you can only modify the actual static pods through their manifest files on the node.

/DEFINITION

  • An instance of basicManager(pkg/kubelet/pod/pod_manager.go: 108) struct, which implements the Manager(pkg/kubelet/pod/pod_manager.go: 45) interface is created, klet.podManager, at pkg/kubelet/kubelet.go: 664, which is used to manage the pods on the node.

DEEPER DIVE

The podManager is a critical component of the kubelet that manages the pods on the node. Here’s a deeper look at its role and functionality:

  1. Storage Role:
    • Stores the desired/intended set of:
      • Admitted pods
      • Mirror pods
    • Does NOT store the actual running pods (that’s handled by podWorkers)
  2. Configuration:
    • Gets populated by kubelet config loops
    • Receives pod configurations from multiple sources:
      • API server (for regular pods)
      • Local filesystem or HTTP (for static pods)
  3. Important Distinctions:
    • Not all desired pods in podManager are actually running
    • Not all running pods are in podManager (e.g., force-deleted pods may still be terminating)
  4. Future Improvements (TODO):
    • There’s a planned review to move components from using podManager to podWorkers when they need actual pod state
    • This change might add some latency but will:
      • Avoid race conditions
      • Better handle terminating pods
      • Better handle updated static pods

/DEEPER DIVE

  • An instance of status manager(pkg/kubelet/status/status_manager.go: 71) which implements the Manager(pkg/kubelet/status/status_manager.go: 121) interface, is created, klet.statusManager, at pkg/kubelet/kubelet.go: 666, which is used to manage the status of the pods on the node.
  • An instance of resourceAnalyzer(pkg/kubelet/server/stats/resource_analyzer.go: 33) struct which implements the ResourceAnalyzer(pkg/kubelet/server/stats/resource_analyzer.go: 25) interface, is created, klet.resourceAnalyzer, at pkg/kubelet/kubelet.go: 667, which is used to analyze the resources on the node.
  • kubeDeps.RemoteRuntimeService is assigned to klet.runtimeService at pkg/kubelet/kubelet.go: 669, which is used to manage the container runtime service.
  • An instance of containerLogManager(pkg/kubelet/logs/container_log_manager.go: 143) struct which implements the ContainerLogManager(pkg/kubelet/logs/container_log_manager.go: 55) interface, is created, klet.containerLogManager, at pkg/kubelet/kubelet.go: 676-687, which is used to manage the container logs on the node.
    // pkg/kubelet/kubelet.go: 676
   	containerLogManager, err := logs.NewContainerLogManager(
    		klet.runtimeService,
    		kubeDeps.OSInterface,
    		kubeCfg.ContainerLogMaxSize,
    		int(kubeCfg.ContainerLogMaxFiles),
    		int(kubeCfg.ContainerLogMaxWorkers),
    		kubeCfg.ContainerLogMonitorInterval,
   	)
   	if err != nil {
    		return nil, fmt.Errorf("failed to initialize container log manager: %v", err)
   	}
  	klet.containerLogManager = containerLogManager
  • An instance of ReasonCache(pkg/kubelet/reason_cache.go: 38) struct is created, klet.reasonCache, at pkg/kubelet/kubelet.go: 673, which is used to to store the failure reason for the latest container start, it’s an lru cache hence it might not be reliable, but goal is propogate the reason to container stature.
  • An instance of basicWorkQueue(pkg/kubelet/util/queue/work_queue.go: 36) struct which implements the WorkQueue(pkg/kubelet/util/queue/work_queue.go: 29), is created, klet.workQueue, at pkg/kubelet/kubelet.go: 690, WorkQueue allows queuing items with a timestamp and an item is considered ready to process if the timestamp has expired.
  • An instance of podWorkers(pkg/kubelet/pod_workers.go: 551) struct which implements the PodWorkers(pkg/kubelet/pod_workers.go: 36), is created, klet.podWorkers, at pkg/kubelet/kubelet.go: 692. The PodWorkers interface is used to manage the workers that process pods on the node. It provides methods to add, remove, and get workers, as well as to get the number of workers and the number of pods being processed by each worker. And here is a deeper look at this.

DEEPER DIVE

Let look the call the following call.

// pkg/kubelet/kubelet.go: 692
klet.podWorkers = newPodWorkers(
		// an instance of Kubelet struct, pkg/kubelet/kubelet.go: 568
		klet,
		// an instance of record.EventRecorder, pkg/kubelet/kubelet.go: 570
		kubeDeps.Recorder,
		// a queue used to trigger pod workers.
		klet.workQueue,
		// resyncInterval is the interval between periodic full reconciliations of
		klet.resyncInterval,
		// backOffPeriod is the period to back off when pod syncing results in an error.
		backOffPeriod,
		// Cache stores the PodStatus for the pods. It represents *all* the visible
		// pods/containers in the container runtime. All cache entries are at least as
		// new or newer than the global timestamp (set by UpdateTime()), while
		// individual entries may be slightly newer than the global timestamp. If a pod
		// has no states known by the runtime, Cache returns an empty PodStatus object
		// with ID populated.
		//
		// Cache provides two methods to retrieve the PodStatus: the non-blocking Get()
		// and the blocking GetNewerThan() method. The component responsible for
		// populating the cache is expected to call Delete() to explicitly free the
		// cache entries.
		// an instance of Cache interface, pkg/kubelet/container/cache.go: 39
		klet.podCache,
	)

The podWorkers is a core component in the kubelet that manages the lifecycle of pods. Here are its key responsibilities:

  • State Management:
    • It manages pods through three main states:
      • syncing: pod should be running
      • terminating: pod should be stopped
      • terminated: pod should have all resources cleaned up
  • Exclusive Pod Execution:
    • Once a pod is accepted, no other pod with the same UID (or name+namespace for static pods) can start until the first pod has fully terminated and been cleaned up.
    • This means even if a pod is desired in the API and admitted, it might have to wait for a prior pod to terminate.
  • Event-Driven Architecture:
    • It works as an event-driven controller through UpdatePod
    • Requires periodic resyncing through SyncKnownPods to maintain desired state
    • Handles force-deleted pods that still need to complete their termination
  • Authority:
    • Acts as the authoritative source within kubelet for:
      • Which pods are actually running
      • Current state of each pod
    • Other kubelet components must consult podWorkers to know pod phases for resource management

In essence, it’s the traffic controller for pod lifecycle management within the kubelet, ensuring orderly execution and cleanup of pods.

/DEEPER DIVE

  • A variable singleProcessOOMKill is set pkg/kubelet/kubelet.go: 700, which is used to determine if the kubelet should group kill all process within a container or kill on the concerned process. The default value for this cgroupv1 is true, and cannot be false, but in the cgroupv2 it could be either true or false, and the default is false.

JUST NOTE

When we talk about group kill in the context of OOM (Out Of Memory) in containers, it means all processes within the same container (not the pod) are killed when an OOM event occurs.

To clarify the different scopes:

  • When SingleProcessOOMKill=true, the Out-Of-Memory (OOM) killer behavior is configured to:
    • Selectively terminate only the specific process that exceeds its memory limit
    • Preserve other processes within the same container
    • Maintain the integrity of other containers within the pod

This granular OOM control helps maintain system stability while minimizing the impact of memory-related terminations.

  • When SingleProcessOOMKill=false, the OOM killer operates in group termination mode:
    • Terminates all processes within the memory-exceeding container
    • This conservative approach ensures complete cleanup of potentially unstable containers
    • Helps prevent memory leaks and cascading failures
    • Maintains pod-level isolation by preserving other containers in the pod
    • Container restart behavior is governed by the pod’s restartPolicy specification

The term “group” in this context specifically denotes the process group within a single container’s namespace, not the broader pod scope. This container-level isolation ensures that OOM events are handled independently for each container, maintaining the pod’s overall stability and resource boundaries.

Consider a pod with two containers A and B:

  • An OOM event in container A triggers the configured OOM killer behavior (single-process or group) exclusively within container A’s namespace
  • Container B remains unaffected, maintaining its resource allocation and runtime state This isolation is fundamental to Kubernetes’ container orchestration model, ensuring that resource-related issues in one container do not cascade to other containers within the same pod.

/JUST NOTE

  • A generic runtime manager is created, runtime, at pkg/kubelet/kubelet.go: 714, which is used to manage the runtime of the containers. runtime is an instance to kubeGenericRuntimeManager(pkg/kubelet/kuberuntime/kuberuntime_manager.go: 98) which implements interface KubeGenericRuntime(pkg/kubelet/kuberuntime/kuberuntime_manager.go: 182). KubeGenericRuntime is a interface contains interfaces for container runtime and command.

DEEPER DIVE

	// pkg/kubelet/kubelet.go: 714
	runtime, err := kuberuntime.NewKubeGenericRuntimeManager(
		// FilterEventRecorder creates an event recorder to record object's event except implicitly required container's, like infra container.
		kubecontainer.FilterEventRecorder(kubeDeps.Recorder),
		// Manages container health check results.
		klet.livenessManager,
		klet.readinessManager,
		klet.startupManager,
		// /var/lib/kubelet is the root directory of the kubelet.
		rootDirectory,
		// podLogsDir is a custom root directory path kubelet will use to place pod's log files.
		// /var/log/pods
		podLogsDirectory,
		// An instance of the MachineInfo struct.
		machineInfo,
		// serves as the authoritative source for pod state within the kubelet
		klet.podWorkers,
		// An iterface to OS specific operations.
		kubeDeps.OSInterface,
		// An instance of the kubelet object
		klet,
		// HTTP client to make requests to the container lifecycle handler.
		insecureContainerLifecycleHTTPClient,
		// Period for the exponential backoff for image pulls.
		imageBackOff,
		// Tells kubelet to serialize image pulls, default: true
		kubeCfg.SerializeImagePulls,
		// The maximum number of concurrent image pull operations.
		kubeCfg.MaxParallelImagePulls,
		// RegistryPullQPS is the limit of registry pulls per second.
		float32(kubeCfg.RegistryPullQPS),
		// RegistryBurst is the maximum size of bursty pulls,
		// temporarily allows pulls to burst to this number,
		// while still not exceeding RegistryPullQPS
		int(kubeCfg.RegistryBurst),
		imageCredentialProviderConfigFile,
		imageCredentialProviderBinDir,
		// As menstioned above, the singleProcessOOMKill is set to true.
		singleProcessOOMKill,
		// Boolean, enables CPU CFS quota enforcement for containers that specify CPU limits.
		kubeCfg.CPUCFSQuota,
		// CPU CFS quota period value, cpu.cfs_period_us. Default: 100ms
		kubeCfg.CPUCFSQuotaPeriod,
		// An instance of thread-safe instance to RuntimeService interface.
		// returned from staging/src/k8s.io/cri-client/pkg/remote_runtime.go: 83
		kubeDeps.RemoteRuntimeService,
		// An instance to thread-safe instance of ImageManagerService interface.
	 	// returned from staging/src/k8s.io/cri-client/pkg/remote_image.go: 50
		kubeDeps.RemoteImageService,
		// Interface to manage containers on the machine
		// returned from pkg/kubelet/cm/container_manager_linux.go: 206
		kubeDeps.ContainerManager,
		// Manager for container logs
		klet.containerLogManager,
		// Handles RuntimeClass objects for the Kubelet.
		klet.runtimeClassManager,
		// seccomp profile for all workloads, default: false
		seccompDefault,
		// swapBehavior configures swap memory available to container workloads
		// For cgroupsv1 this is not configured
		kubeCfg.MemorySwap.SwapBehavior,
		// Returns the absolute value of the Node Allocatable
		// which is primarily useful for enforcement
		kubeDeps.ContainerManager.GetNodeAllocatableAbsolute,
		// MemoryThrottlingFactor specifies the factor multiplied by the memory limit or node allocatable memory
		// when setting the cgroupv2 memory.high value to enforce MemoryQoS.
		// Decreasing this factor will set lower high limit for container cgroups and put heavier reclaim pressure
		// while increasing will put less reclaim pressure.
		*kubeCfg.MemoryThrottlingFactor,
		// An instance of PodStartupLatencyTracker interface.
		kubeDeps.PodStartupLatencyTracker,
		// TracerProvider provides Tracers that are used by instrumentation code
		kubeDeps.TracerProvider,
	)

The NewKubeGenericRuntimeManager constructor function initializes the Kubernetes Container Runtime Interface (CRI) manager, which serves as the primary interface between the Kubelet and the container runtime.

Architectural Components:

  • Purpose:
    • Creates a container runtime manager that handles container operations in Kubernetes
    • Interfaces with the underlying container runtime (like Docker or containerd) through CRI (Container Runtime Interface)
  • Important Parameters:
    • recorder: For recording events
    • Various managers for handling container probes (liveness, readiness, startup)
    • runtimeService: Interface to the container runtime
    • imageService: Service for handling container images
    • Configuration options like:
      • serializeImagePulls: Controls parallel image pulling
      • maxParallelImagePulls: Limits concurrent image pulls
      • seccompDefault: Security computing mode settings
      • memorySwapBehavior: Memory swap configuration
  • Main Operations:
    • Creates an instrumented runtime and image service, with metrics
    • Initializes a new kubeGenericRuntimeManager with the provided parameters
    • Verifies runtime API version compatibility
    • Sets up credential providers for container image pulling
    • Initializes several sub-components:
      • Image puller
      • Lifecycle handler runner
      • Container garbage collector
      • Version cache
  • Version Checking:
if typedVersion.Version != kubeRuntimeAPIVersion {
    return nil, ErrVersionNotSupported
}
  • Ensures the runtime API version matches the supported version
  • Returns an error if versions are incompatible

  • Component Initialization:
    • Sets up image pulling management:
    • Sets up lifecycle handler(postStart and preStop) for container
    • Sets up new container garbage collector
    • The podStateProvider is assigned the podWorkers instance
kubeRuntimeManager.imagePuller = images.NewImageManager(
    kubecontainer.FilterEventRecorder(recorder),
    kubeRuntimeManager,
    imageBackOff,
    serializeImagePulls,
    maxParallelImagePulls,
    imagePullQPS,
    imagePullBurst,
    podPullingTimeRecorder)
kubeRuntimeManager.runner = lifecycle.NewHandlerRunner(insecureContainerLifecycleHTTPClient, kubeRuntimeManager, kubeRuntimeManager, recorder)
kubeRuntimeManager.containerGC = newContainerGC(runtimeService, podStateProvider, kubeRuntimeManager, tracer)
kubeRuntimeManager.podStateProvider = podStateProvider

The lifecycle handler is an interface of HandlerRunner for a container(pkg/kubelet/container/helpers.go: 42). This interface implements a Run function(pkg/kubelet/lifecycle/handlers.go: 72) which is responsible for executing different handler types on the container.

  1. Error Handling:
    • Handles initialization errors gracefully
    • Logs errors with detailed information
    • Exits if critical components (like credential provider plugins) fail to initialize

This function is a crucial part of Kubernetes’ container runtime abstraction layer, allowing it to work with different container runtimes while maintaining a consistent interface for the rest of the system.

/DEEPER DIVE

  • The runtime is assigned to:
    • klet.containerRuntime at pkg/kubelet/kubelet.go: 751, which is used to manage the container runtime.
    • klet.streamingRuntime at pkg/kubelet/kubelet.go: 752, which handles container streaming.
    • klet.runner at pkg/kubelet/kubelet.go: 753, which will be used to run commands inside of the container.
  • Create an instance of RuntimeCache in runtimeCache at pkg/kubelet/kubelet.go: 755, runtimeCache caches a list of pods. It records a timestamp (cacheTime) right before updating the pods, so the timestamp is at most as new as the pods (and can be slightly older). The timestamp always moves forward. Callers are expected not to modify the pods returned from GetPods. And then that is assigned to klet.runtimeCache.
  • Create an instance of HostStatsProvider that defines an interface for providing host stats associated with pods that are managed by the kubelet, and assigns it hostStatsProvider at pkg/kubelet/kubelet.go: 762.
  • The container runtime stats providers is initialized into klet.StatsProvider, creating an instance of Provider(pkg/kubelet/stats/provider.go: 91) which provides the stats of the node and the pod-managed containers, at pkg/kubelet/kubelet.go: 766.
  • An eventChannel is created for PodLifecycleEvent at pkg/kubelet/kubelet.go: 786. And since the EventedPLEG feature flag is still in Alpha a generic pleg interface is created at pkg/kubelet/kubelet.go: 812 with an instance of PodLifecycleEventGenerator(pkg/kubelet/pleg/pleg.go), which container functions for generating pod lifecycle events, and stored in klet.gleg.
  • A newRuntimeState is assiged to klet.runtimeState, by calling the function with the same, passing the maxWaitForContainerRuntime which defines the maximum wait time for the runtime to be available. Ref, pkg/kubelet/kubelet.go: 815
  • A PLEG health check is added to klet.runtimeState, by calling the addHealthCheck at pkg/kubelet/kubelet.go: 816
  • IMPORTANT: The runtimeState.cidr is update with CIDR obtained from kudeCfg.PodCIDR, by calling the klet.updatePodCIDR function at pkg/kubelet/kubelet.go: 820. updatePodCIDR(pkg/kubelet/kubelet_network.go: 44) updates the pod CIDR in the runtime state if it is different from the current CIDR. Return true if pod CIDR is actually changed.
  • An instance to GC(pkg/kubelet/container/container_gc.go: 43) interface, which manages garbage collection of dead containers, is created by calling NewContainerGC at pkg/kubelet/kubelet.go: 825. And upon error, this assigned to klet.containerGC at pkg/kubelet/kubelet.go: 829
  • An instance of podContainerDeletor struct(pkg/kubelet/pod_container_deletor.go: 36), created into klet.containerDeletor by calling the newPodContainerDeletor at pkg/kubelet/kubelet.go: 830. A podContainerDeletor, is an asynchronous worker(a goroutine) that receives containers ids from a channel, and calls runtime.DeleteContaier(pkg/kubelet/pod_container_deletor.go: 52) which in turn calls removeContainer(pkg/kubelet/kuberuntime/kuberuntime_container.go: 1314), to remove container and container logs.
  • An instance of ImageGCManager interface(pkg/kubelet/images/image_gc_manager.go: 69), which managed the lifecycle of all images, is create by calling NewImageGCManager(pkg/kubelet/images/images_gc_manager.go: 184) at pkg/kubelet/kubelet.go: 833. And upon no error that instance is assigned to klet.imageManager(yes, not *GCManager).
  • Based on the kubeDeps.TLSOptions, the klet.serverCertificateManager is setup at pkg/kubelet/kubelet.go: 839-851.
  • The klet.probeManager(pkg/kubelet/kubelet.go: 867) is initialized to an instance of prode manager interface, Manager defined at pkg/kubelet/prober/prober_manager.go: 71 which creates a probe worker for every container that specifies a probe, and it also updates each of the pod container status.
  • A tokenManager is created using the NewManager function from the token package at pkg/kubelet/kubelet.go: 876, this must be used later one. The tokenManager is responsible for managing the service account tokens.
  • At pkg/kubelet/kubelet.go: 878-891,
    • clusterTrustBundleManger is initialized to manager ClusterTrustBundle(staging/src/k8s.io/api/certificate/v1alpha1/types.go: 44)
      • If ClusterTrustBundleProjection feature flag is enabled, an informer based manager is created. Otherwise, a no-op manager is used instead.
      • TODO: Check the above in the existing clusters as default for ClusterTrustBundleProjection is false, and its still an alpha feature.
  • At pkg/kubelet/kubelet.go: 896
    • The klet.volumePluginMgr is initialized using the NewInitializedVolumePluginMgr function.
      • The volumePluginMgr, manages all volume plugins (e.g., CSI, FlexVolume, in-tree plugins) that the Kubelet uses to provision and manage storage for pods.
      • if err, then this function returns
  • At pkg/kubelet/kubelet.go: 901, create a PluginManager instance by calling pluginmanager.NewPluginManager, and assigns it to klet.pluginManager. pluginManager, manages the lifecycle of kubelet plugins.
  • At pkg/kubelet/kubelet.go: 908-912, if experimentalMounterPath is set it configures DNS in the containerized mounter’s environment.
  • At pkg/kubelet/kubelet.go: 915-927, creates a VolumeManager instance by calling volumemanager.NewVolumeManager and assigns it to klet.volumeManager. volumeManager, is responsible for managing the lifecycle of volumes (e.g., attaching, mounting, unmounting, detaching) for pods running on the node.
  • At pkg/kubelet/kubelet.go: 929 variable boMax is initialized to MaxContainerBackOff(which is 300 * time.Second).
  • At pkg/kubelet/kubelet.go: 930 variable base is initialized to containerBackOffPeriod(which is time.Second * 10)
  • At pkg/kubelet/kubelet.go: 931-936 if the KubeletCrashLoopBackOffMax feature gate(currently in Alpha and default false) is enabled, boMax is reset accordingly.
  • At pkg/kubelet/kubelet.go: 937 creates a BackOff instance by calling flowcontrol.NewBackOff and assigns it klet.backOff. And at pkg/kubelet/kubelet.go: 938, assign an anonymous function to the klet.backOff.HasExpiredFunc, which returns a boolean on the expiry backOff which reset the backOff counter.
  • At pkg/kubelet/kubelet.go: 943 creates an Manager instance(pkg/kubelet/eviction/types.go) which is implemented by the managerImpl(pkg/kubelet/eviction/eviction_manager.go: 66) by calling the eviction.NewManager. The instance of Manager is passed in both evictionManager and evictionAdmitHandler. And at pkg/kubelet/kubelet.go: 946-947, these are assigned to klet.evictionManager.evictionManager and klet.admitHandlers respectively.
  • At pkg/kubelet/kubelet.go: 951 the slice safeAndUnsafeSysctls is assigned the slice containing both sysctl.SafeSysctlAllowlist(), and user provided(though configuration) allowedUnsafeSysctls. And at pkg/kubelet/kubelet.go: 952, the safeAndUnsafeSysctls is verified by passing to NewAllowList(pkg/kubelet/sysctl/allowlist.go: 44), and the returns patternAllowlist(pkg/kubelet/sysctl/allowlist.go: 36) in to sysctlsAllowList. On err in this case this function returns with an error.
  • At pkg/kubelet/kubelet.go: 956 the sysctlsAllowlist is add to klet.admitHandlers.
  • At pkg/kubelet/kubelet.go: 959 an instance of activeDeadlineHandler(pkg/kubelet/active_deadline.go: 36) is created by calling newActiveDeadlineHandler(pkg/kubelet/active_deadline.go: 46), and assigned to activeDeadlineHandler. A deadline handler, manages the pod deadline. And this is added to klet’s PodSyncLoopHandler and PodSyncHandler at pkg/kubelet/kubelet.go: 963 and pkg/kubelet/kubelet.go: 964 respectively. On err in this case this function returns with an error.
  • At pkg/kubelet/kubelet.go: 966, an admit handler responsible for allocating resources to pod GetAllocateResourcesPodAdmitHandler(pkg/kubelet/cm/container_manager.go: 136), added to klet.admitHandler.
  • At pkg/kubelet/kubelet.go: 968-969, an instance of the CriticalPodAdmissionHandler(pkg/kubelet/preemption/preemption.go: 45) is created by calling the NewCriticalPodAdmissionHandler(pkg/kubelet/preemption/preemption.go: 63) and assigned to criticalPodAdmissionHandler, if not error and then it added to the klet.admitHandlers with predicates. Its critical to understand the CriticalPodAdmissionHandler is in fact an AdmissionFailureHandler(pkg/kubelet/lifecycle/predicate.go: 87)
  • If linux, at pkg/kubelet/kubelet.go: 977 an instance of Validator(pkg/security/apparmor/validate.go: 34) interface which validates that a pod with AppArmor profile can be run by that Node, and assigned to klet.appArmorValidator
  • If linux, at pkg/kubelet/kubelet.go: 977 an instance of Validator(pkg/security/apparmor/validate.go: 34) interface which validates that a pod with AppArmor profile can be run by that Node, and assigned to klet.appArmorValidator, this is added to the klet.adminHandlers at pkg/kubelet/kubelet.go: 978.
  • IMPORTANT: At pkg/kubelet/kubelet.go: 981-992, the klet.nodeLeaseController is initialized, this controller is responsible for renewing lease of the kubelet with the api-server. A lease is the indicator the node’s health. The leaseDuration(pkg/kubelet/kubelet.go: 981) default to 40s, and renewInterval(pkg/kubelet/kubelet.go: 982) is one-fourth of the leasDuration.
  • IMPORTANT: At pkg/kubelet/kubelet.go: 995, the node shutdownManager is assigned an instance of Manager(pkg/kubelet/nodeshutdown/nodeshutdown_manager.go: 42) interface, which will handle aspects of the node shutdown, including honoring the graceful shutdown periods, for critical pods or pods by priority. The shutdownManager, is then assigned to the klet.shutdownManager(pkg/kubelet/kubelet.go: 1009)
  • At pkg/kubelet/kubelet.go: 1010 an instance of the UsernsManager(pkg/kubelet/userns/userns_manager.go: 54), is created which ensures avoiding the UID and GID with those of on the node, and effective manage User Namespaces, and assigned to klet.usernsManager.
  • At pkg/kubelet/kubelet.go: 1014 the shutdownManager(from above) is added to the klet.admitHandlers
  • At pkg/kubelet/kubelet.go: 1018 the latest kubeCfg, is assigned to the klet.kubeletConfiguration.
  • At pkg/kubelet/kubelet.go: 1022 the node status functions are generated and assinged to klet.setNodeStatusFuncs by calling the defaultNodeStatusFuncs(pkg/kubelet/kubelet_node_status.go: 760), which returns an array of callback functions.
  • If the SystemdWatchdog feature flag is enabled, which in most mordern Kubernetes clusters(>= 1.32), this allows using systemd watchdog to monitor the health status of kubelet. At pkg/kubelet/kubelet.go: 1028 the klet.healthChecker is updated with the new checkers.

Then at pkg/kubelet/kubelet.go: 1033 the function returns created klet instance.

/DEEPER DIVE

Upon successful initialization at pkg/kubelet/kubelet.go: 1310, the Kubelet performs two critical operations:

  1. Emits a startup event via k.BirthCry() (pkg/kubelet/kubelet.go: 1341)
  2. Initiates garbage collection routines through k.StartGarbageCollection() (pkg/kubelet/kubelet.go: 1343)

The createAndInitKubelet function at pkg/kubelet/kubelet.go: 1345 returns the initialized Kubelet instance k to the RunKubelet function. Before proceeding to the startKubelet function (cmd/kubelet/app/server.go:1281), two configuration parameters are set:

  • podCfg is initialized with kubeDeps.PodConfig
  • The Linux resource limit (rlimit) for maximum open files is configured via kubeServer.MaxOpenFiles (cmd/kubelet/app/server.go: 1277)

Now back to the startKubelet function at cmd/kubelet/app/server.go:1281. Lets dive deeper into the startKubelet function.

DEEPER DIVE

This function, startKubelet, is responsible for starting the Kubelet, which is a core Kubernetes component that runs on each node in the cluster. The Kubelet’s primary role is to ensure that containers are running in Pods as expected. The function starts the Kubelet’s main logic and optionally enables its HTTP servers for serving metrics, debugging, and other endpoints. Here’s a detailed breakdown of what this function does:


Start the Kubelet’s Main Logic

// cmd/kubelet/app/server.go:1289
go k.Run(podCfg.Updates())
  • k.Run(podCfg.Updates()) starts the Kubelet’s main event loop.
    • podCfg.Updates() returns a channel that provides updates to the Kubelet about changes in Pod configurations (e.g., new Pods, updates to existing Pods, or deletions).
    • The Kubelet listens to this channel and takes appropriate actions, such as creating, updating, or deleting Pods on the node.
  • The go keyword runs this logic in a separate goroutine, allowing the Kubelet to operate asynchronously.
  • TODO: Define the logic for handling Pod updates.

Start the Kubelet’s Secure HTTP Server (if enabled)

// cmd/kubelet/app/server.go:1292
if enableServer {
    go k.ListenAndServe(kubeCfg, kubeDeps.TLSOptions, kubeDeps.Auth, kubeDeps.TracerProvider)
}
  • If enableServer is true, the Kubelet starts a secure HTTP server.
    • k.ListenAndServe starts the server, which listens for HTTPS requests.
    • The server is configured using:
      • kubeCfg: The Kubelet’s configuration.
      • kubeDeps.TLSOptions: TLS settings for securing the server.
      • kubeDeps.Auth: Authentication settings for securing access to the server.
      • kubeDeps.TracerProvider: Used for distributed tracing (e.g., OpenTelemetry).
  • This server typically serves endpoints for metrics, debugging, and other administrative functions.
  • The go keyword runs the server in a separate goroutine.
  • TODO: Dive deeper into the implementation details of the secure HTTP server.

Start the Kubelet’s Read-Only HTTP Server (if enabled)

// cmd/kubelet/app/server.go:1295
if kubeCfg.ReadOnlyPort > 0 {
    go k.ListenAndServeReadOnly(netutils.ParseIPSloppy(kubeCfg.Address), uint(kubeCfg.ReadOnlyPort), kubeDeps.TracerProvider)
}
  • If the ReadOnlyPort in the Kubelet’s configuration is greater than 0, a read-only HTTP server is started.
    • k.ListenAndServeReadOnly starts the server, which listens for HTTP requests.
    • The server binds to the IP address specified in kubeCfg.Address and the port specified in kubeCfg.ReadOnlyPort.
    • This server provides read-only access to metrics and debugging endpoints, typically without authentication or encryption.
    • kubeDeps.TracerProvider is used for distributed tracing.
  • The go keyword runs the server in a separate goroutine.

Start the Kubelet’s Pod Resources Server

// cmd/kubelet/app/server.go:1298
go k.ListenAndServePodResources()
  • k.ListenAndServePodResources() starts a gRPC server that serves the Pod Resources API.
    • This API provides information about the resources (e.g., CPU, memory, devices) allocated to Pods running on the node.
    • It is used by components like the Kubernetes Device Plugins and other resource management tools.
  • The go keyword runs this server in a separate goroutine.
  • TODO: Dive into the implementation details of the Pod Resources API.

By running these components in separate goroutines, the Kubelet can handle multiple tasks concurrently, ensuring that it can manage Pods, serve metrics, and provide debugging information efficiently.

/DEEPER DIVE

NOTE Come out of the startKubelet function.

/DEEPER DIVE

References:

Some Important Data Structures

  1. Kubelet
  2. KubeletConfiguration
  3. Kubelet Dependencies
  4. Kubelet Flags
  5. Kube Generic Runtime Manager
  6. EventedPLEG
  7. GenericPLEG

Contribution Opportunities:

  • Can definitely improve the logic in this code for parseNodeIP function, it so difficult to read and digest.
  • Possible simple contribution to Kubernetes codebase, fix the error log message on line pkg/kubelet/util/boottime_util_linux.go: 39 from

      klog.InfoS("Failed to get boot time from /proc/uptime. Will retry with unix.Sysinfo.", "error", err)
    

    to

      klog.InfoS("Failed to get boot time from /proc/stat. Will retry with unix.Sysinfo.", "error", err)
    

Good to Explore

  1. CPUManagerPolicy for CPU bound workloads, and for stateful workloads
  2. TopologyManagerPolicy for NUMA aware workloads
  3. KubeletTracing to be enabled for tracing kubelet calls

Just Random Notes

Container stats provider

There is a feature flag in Alpha to move container stats from CAdvisor to CRI stats provider, this will be populated to the kubeDeps.useLegacyCadvisorStats. Code is for checking this flag is enabled is here.

// pkg/kubelet/cadvisor/util.go: 77

// UsingLegacyCadvisorStats returns true if container stats are provided by cadvisor instead of through the CRI.
// CRI integrations should get container metrics via CRI.
// TODO: cri-o relies on cadvisor as a temporary workaround. The code should
// be removed. Related issue:
// https://github.com/kubernetes/kubernetes/issues/51798
func UsingLegacyCadvisorStats(runtimeEndpoint string) bool {
	// If PodAndContainerStatsFromCRI feature is enabled, then assume the user
	// wants to use CRI stats, as the aforementioned workaround isn't needed
	// when this feature is enabled.
	if utilfeature.DefaultFeatureGate.Enabled(features.PodAndContainerStatsFromCRI) {
		return false
	}
	return strings.HasSuffix(runtimeEndpoint, CrioSocketSuffix)
}

How to get the kubelet’s runtime configuration?

Here’s how you can curl the kubelet’s runtime configuration:

Prerequisites

  • Ensure you have access to the kubelet API
    • The kubelet API is usually protected by authentication and authorization mechanisms. You may need to provide a valid token or client certificate to access it.
    • If you’re running this command from a node where the kubelet is running, you can access it via localhost.

Accessing Configuration

  • Use curl to access the /configz endpoint
     curl -sSk -H "Authorization: Bearer $(cat /var/run/secrets/kubernetes.io/serviceaccount/token)" \
     https://localhost:10250/configz
    
    • -sSk: Silent mode, skip SSL certificate verification (since the kubelet uses a self-signed certificate by default).
    • -H "Authorization: Bearer $(cat /var/run/secrets/kubernetes.io/serviceaccount/token)": This adds the authorization header using the service account token. This assumes you’re running this command from a pod with access to the service account token.
    • https://localhost:10250/configz: The kubelet’s /configz endpoint.

3. Alternative: Use kubectl proxy

If you don’t have direct access to the kubelet API, you can use kubectl proxy to access it securely.

  • Start a proxy:
    kubectl proxy --port=8080
    
  • Access the kubelet’s /configz endpoint via the proxy:
    curl http://localhost:8080/api/v1/nodes/<node-name>/proxy/configz
    

    Replace <node-name> with the name of the node where the kubelet is running.

4. Parse the output

The output will be in JSON format. You can use tools like jq to parse and format it:

   curl -sSk -H "Authorization: Bearer $(cat /var/run/secrets/kubernetes.io/serviceaccount/token)" \
   https://localhost:10250/configz | jq .

Notes:

  • If the kubelet is not configured to allow anonymous access, you may need to provide a valid token or client certificate.
  • Be cautious when accessing the kubelet API, as it has privileged access to the node and its workloads.
  • Ensure that your network policies or firewall rules allow access to the kubelet’s port (default: 10250).

Feature Gates in Kubernetes

In the Kubernetes codebase, feature gates are defined in the pkg/features/kube_features.go file. And pkg/features/versioned_kube_features.go file contains the versioned feature gates and their default states (enabled or disabled) for each Kubernetes release.

Steps to Find Versioned Feature Gates:

  1. Navigate to the Kubernetes GitHub Repository:
  2. Locate the pkg/features/kube_features.go File:
  3. Search for Feature Gate Definitions:
    • In the kube_features.go file, you’ll find a const definitions for all known kubernetes specific feature keys. Example:

       // owner: @psch
       //
       // Enables a StatefulSet to start from an arbitrary non zero ordinal
       StatefulSetStartOrdinal featuregate.Feature = "StatefulSetStartOrdinal"
      
      • In the versioned_kube_features.go file you will find all the known kubernetes-specific feature keys with VersionedSpecs. Example:
       var defaultVersionedKubernetesFeatureGates = map[featuregate.Feature]featuregate.VersionedSpecs{
       ...
         ...
               StatefulSetStartOrdinal: {
               {Version: version.MustParse("1.26"), Default: false, PreRelease: featuregate.Alpha},
               {Version: version.MustParse("1.27"), Default: true, PreRelease: featuregate.Beta},
               {Version: version.MustParse("1.31"), Default: true, PreRelease: featuregate.GA, LockToDefault: true}, // GA in 1.31, remove in 1.33
           },
         ...
         ...
      
    • Each feature gate is associated with an array of FeatureSpec that specifies:

      • Default: Whether the feature is enabled by default.
      • PreRelease: The maturity stage of the feature (e.g., Alpha, Beta, GA).
        • LockToDefault: Indicates that the feature is locked to its default and cannot be changed
        • Version: ndicates the earliest version from which this FeatureSpec is valid.
  • Check Version-Specific Feature Gates:
    • The file often includes comments indicating in which Kubernetes version a feature gate was introduced or graduated (e.g., from Alpha to Beta or GA). For example:
       // FeatureGateExample is an example feature gate.
       // Owner: @community
       // Alpha: v1.20
       // Beta: v1.22
       // GA: v1.24
       FeatureGateExample featuregate.Feature = "FeatureGateExample"
      
  1. Search for Feature Gate Usage:
    • To see how a specific feature gate is used in the codebase, you can search for its name in the repository. For example, search for FeatureGateExample to find where it is referenced.

Example of Feature Gate Definition:

Here’s an example of how feature gates are defined in kube_features.go:

const (
    // FeatureGateExample is an example feature gate.
    FeatureGateExample featuregate.Feature = "ExampleFeatureGate"
)

And then you will add a version feature gate entry in versioned_kube_feature.go:

	  var defaultVersionedKubernetesFeatureGates = map[featuregate.Feature]featuregate.VersionedSpecs{
	  ...
		...
			  FeatureGateExample: {
		      {Version: version.MustParse("1.26"), Default: false, PreRelease: featuregate.Alpha},
	      },
		...
		...

Additional Notes:

  • Feature gates are used to enable or disable experimental or optional features in Kubernetes.
  • The state of a feature gate (enabled or disabled) can be controlled using the --feature-gates flag when starting Kubernetes components like the kubelet, API server, or controller manager.
  • Feature gates often progress through stages: Alpha → Beta → GA (General Availability).

About backOff

Couple of things note about about backOff configuration that is set on the kubelet here…

// pkg/kubelet/kubelet.go: 937
	klet.backOff = flowcontrol.NewBackOff(base, boMax)
	klet.backOff.HasExpiredFunc = func(eventTime time.Time, lastUpdate time.Time, maxDuration time.Duration) bool {
		return eventTime.Sub(lastUpdate) > 600*time.Second
	}

… is that strangely NewBackOff do not apply any jitter, if you notice the function.

func NewBackOff(initial, max time.Duration) *Backoff {
	return NewBackOffWithJitter(initial, max, 0.0)
}

The maxJitterFactor is 0.0.

Also, if you not func that is assigned to HasExpiredFunc, it does not take maxDuration into account, but rather uses hard coded 600*time.Second, I would think that this should be maxDuration according to the comment on HasExpiredFunc.

	// HasExpiredFunc controls the logic that determines whether the backoff
	// counter should be reset, and when to GC old backoff entries. If nil, the
	// default hasExpired function will restart the backoff factor to the
	// beginning after observing time has passed at least equal to 2*maxDuration

TODO: Deepdive ProbeManager

Start from this…

  1. The klet.probeManager(pkg/kubelet/kubelet.go: 867) is initialized to an instance of prode manager interface, Manager defined at pkg/kubelet/prober/prober_manager.go: 71 which creates a probe worker for every container that specifies a probe, and it also updates each of the pod container status.

To curl the kubelet’s runtime configuration, you need to access the kubelet’s API endpoint, which typically listens on port 10250 by default. The kubelet provides a /configz endpoint that exposes its runtime configuration.

TODO: EvictionManager

As found here…

  1. At pkg/kubelet/kubelet.go: 943 creates an Manager instance(pkg/kubelet/eviction/types.go) which is implemented by the managerImpl(pkg/kubelet/eviction/eviction_manager.go: 66) by calling the eviction.NewManager. The instance of Manager is passed in both evictionManager and evictionAdmitHandler.

TODO: How does Critical Pod Admission work?

Need to work on this…

  1. At pkg/kubelet/kubelet.go: 968-969, an instance of the CriticalPodAdmissionHandler(pkg/kubelet/preemption/preemption.go: 45) is created by calling the NewCriticalPodAdmissionHandler(pkg/kubelet/preemption/preemption.go: 63) and assigned to criticalPodAdmissionHandler, if not error and then it added to the klet.admitHandlers with predicates. Its critical to understand the CriticalPodAdmissionHandler is in fact an AdmissionFailureHandler(pkg/kubelet/lifecycle/predicate.go: 87)

TODO: buildKubeletClientConfig

// cmd/kubelet/app/server.go: 930
func buildKubeletClientConfig(ctx context.Context, s *options.KubeletServer, tp oteltrace.TracerProvider, nodeName types.NodeName) (*restclient.Config, func(), error) {

TODO: Describe how volumePluginMgr works

Lookup this …

At pkg/kubelet/kubelet.go: 896

  • The klet.volumePluginMgr is initialized using the NewInitializedVolumePluginMgr function.
    • The volumePluginMgr, manages all volume plugins (e.g., CSI, FlexVolume, in-tree plugins) that the Kubelet uses to provision and manage storage for pods.
    • if err, then this function returns