From 29872351f9da489ffc057b91084256fc3f90a699 Mon Sep 17 00:00:00 2001 From: Matthew Heon Date: Thu, 3 Aug 2017 09:56:07 -0400 Subject: [PATCH] Move container creation logic into libkpod Retain the Kubernetes CRI interface for now, pending further refactoring. Signed-off-by: Matthew Heon --- libkpod/container_server.go | 37 +- libkpod/create.go | 823 ++++++++++++++++++++++++++++++++++++ libkpod/naming.go | 41 ++ server/container_create.go | 813 +---------------------------------- server/naming.go | 40 +- server/sandbox_run.go | 5 +- server/server.go | 31 -- 7 files changed, 912 insertions(+), 878 deletions(-) create mode 100644 libkpod/create.go create mode 100644 libkpod/naming.go diff --git a/libkpod/container_server.go b/libkpod/container_server.go index c3535e9b..36f77060 100644 --- a/libkpod/container_server.go +++ b/libkpod/container_server.go @@ -3,6 +3,7 @@ package libkpod import ( "encoding/json" "fmt" + "io/ioutil" "os" "path/filepath" "sync" @@ -17,6 +18,8 @@ import ( "github.com/kubernetes-incubator/cri-o/pkg/annotations" "github.com/kubernetes-incubator/cri-o/pkg/registrar" "github.com/kubernetes-incubator/cri-o/pkg/storage" + "github.com/kubernetes-incubator/cri-o/server/apparmor" + "github.com/kubernetes-incubator/cri-o/server/seccomp" rspec "github.com/opencontainers/runtime-spec/specs-go" "github.com/opencontainers/selinux/go-selinux/label" "github.com/pkg/errors" @@ -40,6 +43,12 @@ type ContainerServer struct { stateLock sync.Locker state *containerServerState config *Config + + seccompEnabled bool + seccompProfile seccomp.Seccomp + + appArmorEnabled bool + appArmorProfile string } // Runtime returns the oci runtime for the ContainerServer @@ -92,6 +101,11 @@ func (c *ContainerServer) StorageRuntimeServer() storage.RuntimeServer { return c.storageRuntimeServer } +// SeccompEnabled returns whether seccomp is enabled for this configuration +func (c *ContainerServer) SeccompEnabled() bool { + return c.seccompEnabled +} + // New creates a new ContainerServer with options provided func New(config *Config) (*ContainerServer, error) { store, err := cstorage.GetStore(cstorage.StoreOptions{ @@ -130,6 +144,23 @@ func New(config *Config) (*ContainerServer, error) { lock = new(sync.Mutex) } + var seccompConfig seccomp.Seccomp + if seccomp.IsEnabled() { + seccompProfile, fileErr := ioutil.ReadFile(config.SeccompProfile) + if fileErr != nil { + return nil, fmt.Errorf("opening seccomp profile (%s) failed: %v", config.SeccompProfile, fileErr) + } + if jsonErr := json.Unmarshal(seccompProfile, &seccompConfig); jsonErr != nil { + return nil, fmt.Errorf("decoding seccomp profile failed: %v", jsonErr) + } + } + + if apparmor.IsEnabled() && config.ApparmorProfile == apparmor.DefaultApparmorProfile { + if apparmorErr := apparmor.EnsureDefaultApparmorProfile(); apparmorErr != nil { + return nil, fmt.Errorf("ensuring the default apparmor profile is installed failed: %v", apparmorErr) + } + } + return &ContainerServer{ runtime: runtime, store: store, @@ -145,7 +176,11 @@ func New(config *Config) (*ContainerServer, error) { containers: oci.NewMemoryStore(), sandboxes: make(map[string]*sandbox.Sandbox), }, - config: config, + config: config, + seccompEnabled: seccomp.IsEnabled(), + seccompProfile: seccompConfig, + appArmorEnabled: apparmor.IsEnabled(), + appArmorProfile: config.ApparmorProfile, }, nil } diff --git a/libkpod/create.go b/libkpod/create.go new file mode 100644 index 00000000..41165aef --- /dev/null +++ b/libkpod/create.go @@ -0,0 +1,823 @@ +package libkpod + +import ( + "encoding/json" + "errors" + "fmt" + "io" + "os" + "path/filepath" + "strconv" + "strings" + "time" + + "github.com/sirupsen/logrus" + "github.com/docker/docker/pkg/stringid" + "github.com/docker/docker/pkg/symlink" + "github.com/kubernetes-incubator/cri-o/libkpod/sandbox" + "github.com/kubernetes-incubator/cri-o/oci" + "github.com/kubernetes-incubator/cri-o/pkg/annotations" + "github.com/kubernetes-incubator/cri-o/pkg/storage" + "github.com/kubernetes-incubator/cri-o/server/apparmor" + "github.com/kubernetes-incubator/cri-o/server/seccomp" + "github.com/opencontainers/image-spec/specs-go/v1" + "github.com/opencontainers/runc/libcontainer/cgroups" + "github.com/opencontainers/runc/libcontainer/devices" + "github.com/opencontainers/runc/libcontainer/user" + rspec "github.com/opencontainers/runtime-spec/specs-go" + "github.com/opencontainers/runtime-tools/generate" + "github.com/opencontainers/selinux/go-selinux/label" + "golang.org/x/net/context" + "golang.org/x/sys/unix" + pb "k8s.io/kubernetes/pkg/kubelet/api/v1alpha1/runtime" +) + +const ( + seccompUnconfined = "unconfined" + seccompRuntimeDefault = "runtime/default" + seccompLocalhostPrefix = "localhost/" +) + +func addOCIBindMounts(sb *sandbox.Sandbox, containerConfig *pb.ContainerConfig, specgen *generate.Generator) error { + mounts := containerConfig.GetMounts() + for _, mount := range mounts { + dest := mount.ContainerPath + if dest == "" { + return fmt.Errorf("Mount.ContainerPath is empty") + } + + src := mount.HostPath + if src == "" { + return fmt.Errorf("Mount.HostPath is empty") + } + + if _, err := os.Stat(src); err != nil && os.IsNotExist(err) { + if err1 := os.MkdirAll(src, 0644); err1 != nil { + return fmt.Errorf("Failed to mkdir %s: %s", src, err) + } + } + + options := []string{"rw"} + if mount.Readonly { + options = []string{"ro"} + } + + if mount.SelinuxRelabel { + // Need a way in kubernetes to determine if the volume is shared or private + if err := label.Relabel(src, sb.MountLabel(), true); err != nil && err != unix.ENOTSUP { + return fmt.Errorf("relabel failed %s: %v", src, err) + } + } + + specgen.AddBindMount(src, dest, options) + } + + return nil +} + +func addImageVolumes(rootfs string, imageVolumes ImageVolumesType, containerInfo *storage.ContainerInfo, specgen *generate.Generator, mountLabel string) error { + for dest := range containerInfo.Config.Config.Volumes { + fp, err := symlink.FollowSymlinkInScope(filepath.Join(rootfs, dest), rootfs) + if err != nil { + return err + } + switch imageVolumes { + case ImageVolumesMkdir: + if err1 := os.MkdirAll(fp, 0644); err1 != nil { + return err1 + } + case ImageVolumesBind: + volumeDirName := stringid.GenerateNonCryptoID() + src := filepath.Join(containerInfo.RunDir, "mounts", volumeDirName) + if err1 := os.MkdirAll(src, 0644); err1 != nil { + return err1 + } + // Label the source with the sandbox selinux mount label + if mountLabel != "" { + if err1 := label.Relabel(src, mountLabel, true); err1 != nil && err1 != unix.ENOTSUP { + return fmt.Errorf("relabel failed %s: %v", src, err1) + } + } + + logrus.Debugf("Adding bind mounted volume: %s to %s", src, dest) + specgen.AddBindMount(src, dest, []string{"rw"}) + case ImageVolumesIgnore: + logrus.Debugf("Ignoring volume %v", dest) + default: + logrus.Fatalf("Unrecognized image volumes setting") + } + } + return nil +} + +func addDevices(sb *sandbox.Sandbox, containerConfig *pb.ContainerConfig, specgen *generate.Generator) error { + sp := specgen.Spec() + for _, device := range containerConfig.GetDevices() { + dev, err := devices.DeviceFromPath(device.HostPath, device.Permissions) + if err != nil { + return fmt.Errorf("failed to add device: %v", err) + } + rd := rspec.LinuxDevice{ + Path: device.ContainerPath, + Type: string(dev.Type), + Major: dev.Major, + Minor: dev.Minor, + UID: &dev.Uid, + GID: &dev.Gid, + } + specgen.AddDevice(rd) + sp.Linux.Resources.Devices = append(sp.Linux.Resources.Devices, rspec.LinuxDeviceCgroup{ + Allow: true, + Type: string(dev.Type), + Major: &dev.Major, + Minor: &dev.Minor, + Access: dev.Permissions, + }) + } + return nil +} + +// buildOCIProcessArgs build an OCI compatible process arguments slice. +func buildOCIProcessArgs(containerKubeConfig *pb.ContainerConfig, imageOCIConfig *v1.Image) ([]string, error) { + //# Start the nginx container using the default command, but use custom + //arguments (arg1 .. argN) for that command. + //kubectl run nginx --image=nginx -- ... + + //# Start the nginx container using a different command and custom arguments. + //kubectl run nginx --image=nginx --command -- ... + + kubeCommands := containerKubeConfig.Command + kubeArgs := containerKubeConfig.Args + + // merge image config and kube config + // same as docker does today... + if imageOCIConfig != nil { + if len(kubeCommands) == 0 { + if len(kubeArgs) == 0 { + kubeArgs = imageOCIConfig.Config.Cmd + } + if kubeCommands == nil { + kubeCommands = imageOCIConfig.Config.Entrypoint + } + } + } + + if len(kubeCommands) == 0 && len(kubeArgs) == 0 { + return nil, fmt.Errorf("no command specified") + } + + // create entrypoint and args + var entrypoint string + var args []string + if len(kubeCommands) != 0 { + entrypoint = kubeCommands[0] + args = append(kubeCommands[1:], kubeArgs...) + } else { + entrypoint = kubeArgs[0] + args = kubeArgs[1:] + } + + processArgs := append([]string{entrypoint}, args...) + + logrus.Debugf("OCI process args %v", processArgs) + + return processArgs, nil +} + +// setupContainerUser sets the UID, GID and supplemental groups in OCI runtime config +func setupContainerUser(specgen *generate.Generator, rootfs string, sc *pb.LinuxContainerSecurityContext, imageConfig *v1.Image) error { + if sc != nil { + containerUser := "" + // Case 1: run as user is set by kubelet + if sc.RunAsUser != nil { + containerUser = strconv.FormatInt(sc.GetRunAsUser().Value, 10) + } else { + // Case 2: run as username is set by kubelet + userName := sc.RunAsUsername + if userName != "" { + containerUser = userName + } else { + // Case 3: get user from image config + if imageConfig != nil { + imageUser := imageConfig.Config.User + if imageUser != "" { + containerUser = imageUser + } + } + } + } + + logrus.Debugf("CONTAINER USER: %+v", containerUser) + + // Add uid, gid and groups from user + uid, gid, addGroups, err1 := getUserInfo(rootfs, containerUser) + if err1 != nil { + return err1 + } + + logrus.Debugf("UID: %v, GID: %v, Groups: %+v", uid, gid, addGroups) + specgen.SetProcessUID(uid) + specgen.SetProcessGID(gid) + for _, group := range addGroups { + specgen.AddProcessAdditionalGid(group) + } + + // Add groups from CRI + groups := sc.SupplementalGroups + for _, group := range groups { + specgen.AddProcessAdditionalGid(uint32(group)) + } + } + return nil +} + +func hostNetwork(containerConfig *pb.ContainerConfig) bool { + securityContext := containerConfig.GetLinux().GetSecurityContext() + if securityContext == nil || securityContext.GetNamespaceOptions() == nil { + return false + } + + return securityContext.GetNamespaceOptions().HostNetwork +} + +// EnsureSaneLogPath is a hack to fix https://issues.k8s.io/44043 which causes +// logPath to be a broken symlink to some magical Docker path. Ideally we +// wouldn't have to deal with this, but until that issue is fixed we have to +// remove the path if it's a broken symlink. +func EnsureSaneLogPath(logPath string) error { + // If the path exists but the resolved path does not, then we have a broken + // symlink and we need to remove it. + fi, err := os.Lstat(logPath) + if err != nil || fi.Mode()&os.ModeSymlink == 0 { + // Non-existant files and non-symlinks aren't our problem. + return nil + } + + _, err = os.Stat(logPath) + if os.IsNotExist(err) { + err = os.RemoveAll(logPath) + if err != nil { + return fmt.Errorf("EnsureSaneLogPath remove bad logPath: %s", err) + } + } + return nil +} + +// ContainerCreate creates a new container in specified sandbox +func (c *ContainerServer) ContainerCreate(ctx context.Context, req *pb.CreateContainerRequest) (ctr *oci.Container, err error) { + logrus.Debugf("CreateContainerRequest %+v", req) + + c.updateLock.RLock() + defer c.updateLock.RUnlock() + + sbID := req.PodSandboxId + if sbID == "" { + return nil, fmt.Errorf("PodSandboxId should not be empty") + } + + sandboxID, err := c.podIDIndex.Get(sbID) + if err != nil { + return nil, fmt.Errorf("PodSandbox with ID starting with %s not found: %v", sbID, err) + } + + sb := c.GetSandbox(sandboxID) + if sb == nil { + return nil, fmt.Errorf("specified sandbox not found: %s", sandboxID) + } + + // The config of the container + containerConfig := req.GetConfig() + if containerConfig == nil { + return nil, fmt.Errorf("CreateContainerRequest.ContainerConfig is nil") + } + + name := containerConfig.GetMetadata().Name + if name == "" { + return nil, fmt.Errorf("CreateContainerRequest.ContainerConfig.Name is empty") + } + + containerID, containerName, err := c.generateContainerIDandName(sb.Metadata(), containerConfig) + if err != nil { + return nil, err + } + + defer func() { + if err != nil { + c.ReleaseContainerName(containerName) + } + }() + + container, err := c.createSandboxContainer(ctx, containerID, containerName, sb, req.GetSandboxConfig(), containerConfig) + if err != nil { + return nil, err + } + defer func() { + if err != nil { + err2 := c.storageRuntimeServer.DeleteContainer(containerID) + if err2 != nil { + logrus.Warnf("Failed to cleanup container directory: %v", err2) + } + } + }() + + if err = c.runtime.CreateContainer(container, sb.CgroupParent()); err != nil { + return nil, err + } + + if err = c.runtime.UpdateStatus(container); err != nil { + return nil, err + } + + c.AddContainer(container) + + if err = c.ctrIDIndex.Add(containerID); err != nil { + c.RemoveContainer(container) + return nil, err + } + + c.ContainerStateToDisk(container) + + return container, nil +} + +func (c *ContainerServer) createSandboxContainer(ctx context.Context, containerID string, containerName string, sb *sandbox.Sandbox, SandboxConfig *pb.PodSandboxConfig, containerConfig *pb.ContainerConfig) (*oci.Container, error) { + if sb == nil { + return nil, errors.New("createSandboxContainer needs a sandbox") + } + + // TODO: simplify this function (cyclomatic complexity here is high) + // TODO: factor generating/updating the spec into something other projects can vendor + + // creates a spec Generator with the default spec. + specgen := generate.New() + specgen.HostSpecific = true + specgen.ClearProcessRlimits() + + if err := addOCIBindMounts(sb, containerConfig, &specgen); err != nil { + return nil, err + } + + // Add cgroup mount so container process can introspect its own limits + specgen.AddCgroupsMount("ro") + + if err := addDevices(sb, containerConfig, &specgen); err != nil { + return nil, err + } + + labels := containerConfig.GetLabels() + + metadata := containerConfig.GetMetadata() + + kubeAnnotations := containerConfig.GetAnnotations() + if kubeAnnotations != nil { + for k, v := range kubeAnnotations { + specgen.AddAnnotation(k, v) + } + } + + // set this container's apparmor profile if it is set by sandbox + if c.appArmorEnabled { + appArmorProfileName := c.getAppArmorProfileName(sb.Annotations(), metadata.Name) + if appArmorProfileName != "" { + // reload default apparmor profile if it is unloaded. + if c.appArmorProfile == apparmor.DefaultApparmorProfile { + if err := apparmor.EnsureDefaultApparmorProfile(); err != nil { + return nil, err + } + } + + specgen.SetProcessApparmorProfile(appArmorProfileName) + } + } + var readOnlyRootfs bool + if containerConfig.GetLinux().GetSecurityContext() != nil { + if containerConfig.GetLinux().GetSecurityContext().Privileged { + specgen.SetupPrivileged(true) + } + + if containerConfig.GetLinux().GetSecurityContext().ReadonlyRootfs { + readOnlyRootfs = true + specgen.SetRootReadonly(true) + } + } + + logPath := containerConfig.LogPath + if logPath == "" { + // TODO: Should we use sandboxConfig.GetLogDirectory() here? + logPath = filepath.Join(sb.LogDir(), containerID+".log") + } + if !filepath.IsAbs(logPath) { + // XXX: It's not really clear what this should be versus the sbox logDirectory. + logrus.Warnf("requested logPath for ctr id %s is a relative path: %s", containerID, logPath) + logPath = filepath.Join(sb.LogDir(), logPath) + } + + // Handle https://issues.k8s.io/44043 + if err := EnsureSaneLogPath(logPath); err != nil { + return nil, err + } + + logrus.WithFields(logrus.Fields{ + "sbox.logdir": sb.LogDir(), + "ctr.logfile": containerConfig.LogPath, + "log_path": logPath, + }).Debugf("setting container's log_path") + + specgen.SetProcessTerminal(containerConfig.Tty) + + linux := containerConfig.GetLinux() + if linux != nil { + resources := linux.GetResources() + if resources != nil { + cpuPeriod := resources.CpuPeriod + if cpuPeriod != 0 { + specgen.SetLinuxResourcesCPUPeriod(uint64(cpuPeriod)) + } + + cpuQuota := resources.CpuQuota + if cpuQuota != 0 { + specgen.SetLinuxResourcesCPUQuota(cpuQuota) + } + + cpuShares := resources.CpuShares + if cpuShares != 0 { + specgen.SetLinuxResourcesCPUShares(uint64(cpuShares)) + } + + memoryLimit := resources.MemoryLimitInBytes + if memoryLimit != 0 { + specgen.SetLinuxResourcesMemoryLimit(memoryLimit) + } + + oomScoreAdj := resources.OomScoreAdj + specgen.SetProcessOOMScoreAdj(int(oomScoreAdj)) + } + + if sb.CgroupParent() != "" { + if c.config.CgroupManager == "systemd" { + cgPath := sb.CgroupParent() + ":" + "crio" + ":" + containerID + specgen.SetLinuxCgroupsPath(cgPath) + } else { + specgen.SetLinuxCgroupsPath(sb.CgroupParent() + "/" + containerID) + } + } + + capabilities := linux.GetSecurityContext().GetCapabilities() + toCAPPrefixed := func(cap string) string { + if !strings.HasPrefix(strings.ToLower(cap), "cap_") { + return "CAP_" + cap + } + return cap + } + if capabilities != nil { + addCaps := capabilities.AddCapabilities + if addCaps != nil { + for _, cap := range addCaps { + if err := specgen.AddProcessCapability(toCAPPrefixed(cap)); err != nil { + return nil, err + } + } + } + + dropCaps := capabilities.DropCapabilities + if dropCaps != nil { + for _, cap := range dropCaps { + if err := specgen.DropProcessCapability(toCAPPrefixed(cap)); err != nil { + logrus.Debugf("failed to drop cap %s: %v", toCAPPrefixed(cap), err) + } + } + } + } + + specgen.SetProcessSelinuxLabel(sb.ProcessLabel()) + specgen.SetLinuxMountLabel(sb.MountLabel()) + + if containerConfig.GetLinux().GetSecurityContext() != nil && + !containerConfig.GetLinux().GetSecurityContext().Privileged { + for _, mp := range []string{ + "/proc/kcore", + "/proc/latency_stats", + "/proc/timer_list", + "/proc/timer_stats", + "/proc/sched_debug", + "/sys/firmware", + } { + specgen.AddLinuxMaskedPaths(mp) + } + + for _, rp := range []string{ + "/proc/asound", + "/proc/bus", + "/proc/fs", + "/proc/irq", + "/proc/sys", + "/proc/sysrq-trigger", + } { + specgen.AddLinuxReadonlyPaths(rp) + } + } + } + // Join the namespace paths for the pod sandbox container. + podInfraState := c.runtime.ContainerStatus(sb.InfraContainer()) + + logrus.Debugf("pod container state %+v", podInfraState) + + ipcNsPath := fmt.Sprintf("/proc/%d/ns/ipc", podInfraState.Pid) + if err := specgen.AddOrReplaceLinuxNamespace("ipc", ipcNsPath); err != nil { + return nil, err + } + + netNsPath := sb.NetNsPath() + if netNsPath == "" { + // The sandbox does not have a permanent namespace, + // it's on the host one. + netNsPath = fmt.Sprintf("/proc/%d/ns/net", podInfraState.Pid) + } + + if err := specgen.AddOrReplaceLinuxNamespace("network", netNsPath); err != nil { + return nil, err + } + + imageSpec := containerConfig.GetImage() + if imageSpec == nil { + return nil, fmt.Errorf("CreateContainerRequest.ContainerConfig.Image is nil") + } + + image := imageSpec.Image + if image == "" { + return nil, fmt.Errorf("CreateContainerRequest.ContainerConfig.Image.Image is empty") + } + images, err := c.storageImageServer.ResolveNames(image) + if err != nil { + // This means we got an image ID + if strings.Contains(err.Error(), "cannot specify 64-byte hexadecimal strings") { + images = append(images, image) + } else { + return nil, err + } + } + image = images[0] + + // bind mount the pod shm + specgen.AddBindMount(sb.ShmPath(), "/dev/shm", []string{"rw"}) + + options := []string{"rw"} + if readOnlyRootfs { + options = []string{"ro"} + } + if sb.ResolvPath() != "" { + // bind mount the pod resolver file + specgen.AddBindMount(sb.ResolvPath(), "/etc/resolv.conf", options) + } + + // Bind mount /etc/hosts for host networking containers + if hostNetwork(containerConfig) { + specgen.AddBindMount("/etc/hosts", "/etc/hosts", options) + } + + if sb.Hostname() != "" { + specgen.SetHostname(sb.Hostname()) + } + + specgen.AddAnnotation(annotations.Name, containerName) + specgen.AddAnnotation(annotations.ContainerID, containerID) + specgen.AddAnnotation(annotations.SandboxID, sb.ID()) + specgen.AddAnnotation(annotations.SandboxName, sb.InfraContainer().Name()) + specgen.AddAnnotation(annotations.ContainerType, annotations.ContainerTypeContainer) + specgen.AddAnnotation(annotations.LogPath, logPath) + specgen.AddAnnotation(annotations.TTY, fmt.Sprintf("%v", containerConfig.Tty)) + specgen.AddAnnotation(annotations.Stdin, fmt.Sprintf("%v", containerConfig.Stdin)) + specgen.AddAnnotation(annotations.StdinOnce, fmt.Sprintf("%v", containerConfig.StdinOnce)) + specgen.AddAnnotation(annotations.Image, image) + + created := time.Now() + specgen.AddAnnotation(annotations.Created, created.Format(time.RFC3339Nano)) + + metadataJSON, err := json.Marshal(metadata) + if err != nil { + return nil, err + } + specgen.AddAnnotation(annotations.Metadata, string(metadataJSON)) + + labelsJSON, err := json.Marshal(labels) + if err != nil { + return nil, err + } + specgen.AddAnnotation(annotations.Labels, string(labelsJSON)) + + kubeAnnotationsJSON, err := json.Marshal(kubeAnnotations) + if err != nil { + return nil, err + } + specgen.AddAnnotation(annotations.Annotations, string(kubeAnnotationsJSON)) + + if err = c.setupSeccomp(&specgen, containerName, sb.Annotations()); err != nil { + return nil, err + } + + metaname := metadata.Name + attempt := metadata.Attempt + containerInfo, err := c.storageRuntimeServer.CreateContainer(c.imageContext, + sb.Name(), sb.ID(), + image, image, + containerName, containerID, + metaname, + attempt, + sb.MountLabel(), + nil) + if err != nil { + return nil, err + } + + mountPoint, err := c.storageRuntimeServer.StartContainer(containerID) + if err != nil { + return nil, fmt.Errorf("failed to mount container %s(%s): %v", containerName, containerID, err) + } + + containerImageConfig := containerInfo.Config + if containerImageConfig == nil { + return nil, fmt.Errorf("empty image config for %s", image) + } + + if containerImageConfig.Config.StopSignal != "" { + // this key is defined in image-spec conversion document at https://github.com/opencontainers/image-spec/pull/492/files#diff-8aafbe2c3690162540381b8cdb157112R57 + specgen.AddAnnotation("org.opencontainers.image.stopSignal", containerImageConfig.Config.StopSignal) + } + + // Add image volumes + if err := addImageVolumes(mountPoint, c.config.ImageVolumes, &containerInfo, &specgen, sb.MountLabel()); err != nil { + return nil, err + } + + processArgs, err := buildOCIProcessArgs(containerConfig, containerImageConfig) + if err != nil { + return nil, err + } + specgen.SetProcessArgs(processArgs) + + // Add environment variables from CRI and image config + envs := containerConfig.GetEnvs() + if envs != nil { + for _, item := range envs { + key := item.Key + value := item.Value + if key == "" { + continue + } + specgen.AddProcessEnv(key, value) + } + } + if containerImageConfig != nil { + for _, item := range containerImageConfig.Config.Env { + parts := strings.SplitN(item, "=", 2) + if len(parts) != 2 { + return nil, fmt.Errorf("invalid env from image: %s", item) + } + + if parts[0] == "" { + continue + } + specgen.AddProcessEnv(parts[0], parts[1]) + } + } + + // Set working directory + // Pick it up from image config first and override if specified in CRI + containerCwd := "/" + if containerImageConfig != nil { + imageCwd := containerImageConfig.Config.WorkingDir + if imageCwd != "" { + containerCwd = imageCwd + } + } + runtimeCwd := containerConfig.WorkingDir + if runtimeCwd != "" { + containerCwd = runtimeCwd + } + specgen.SetProcessCwd(containerCwd) + + // Setup user and groups + if linux != nil { + if err = setupContainerUser(&specgen, mountPoint, linux.GetSecurityContext(), containerImageConfig); err != nil { + return nil, err + } + } + + // Set up pids limit if pids cgroup is mounted + _, err = cgroups.FindCgroupMountpoint("pids") + if err == nil { + specgen.SetLinuxResourcesPidsLimit(c.config.PidsLimit) + } + + // by default, the root path is an empty string. set it now. + specgen.SetRootPath(mountPoint) + + saveOptions := generate.ExportOptions{} + if err = specgen.SaveToFile(filepath.Join(containerInfo.Dir, "config.json"), saveOptions); err != nil { + return nil, err + } + if err = specgen.SaveToFile(filepath.Join(containerInfo.RunDir, "config.json"), saveOptions); err != nil { + return nil, err + } + + container, err := oci.NewContainer(containerID, containerName, containerInfo.RunDir, logPath, sb.NetNs(), labels, kubeAnnotations, image, metadata, sb.ID(), containerConfig.Tty, containerConfig.Stdin, containerConfig.StdinOnce, sb.Privileged(), sb.Trusted(), containerInfo.Dir, created, containerImageConfig.Config.StopSignal) + if err != nil { + return nil, err + } + + return container, nil +} + +func (c *ContainerServer) setupSeccomp(specgen *generate.Generator, cname string, sbAnnotations map[string]string) error { + profile, ok := sbAnnotations["security.alpha.kubernetes.io/seccomp/container/"+cname] + if !ok { + profile, ok = sbAnnotations["security.alpha.kubernetes.io/seccomp/pod"] + if !ok { + // running w/o seccomp, aka unconfined + profile = seccompUnconfined + } + } + if !c.seccompEnabled { + if profile != seccompUnconfined { + return fmt.Errorf("seccomp is not enabled in your kernel, cannot run with a profile") + } + logrus.Warn("seccomp is not enabled in your kernel, running container without profile") + } + if profile == seccompUnconfined { + // running w/o seccomp, aka unconfined + specgen.Spec().Linux.Seccomp = nil + return nil + } + if profile == seccompRuntimeDefault { + return seccomp.LoadProfileFromStruct(c.seccompProfile, specgen) + } + if !strings.HasPrefix(profile, seccompLocalhostPrefix) { + return fmt.Errorf("unknown seccomp profile option: %q", profile) + } + //file, err := ioutil.ReadFile(filepath.Join(s.seccompProfileRoot, strings.TrimPrefix(profile, seccompLocalhostPrefix))) + //if err != nil { + //return err + //} + // TODO(runcom): setup from provided node's seccomp profile + // can't do this yet, see https://issues.k8s.io/36997 + return nil +} + +// getAppArmorProfileName gets the profile name for the given container. +func (c *ContainerServer) getAppArmorProfileName(annotations map[string]string, ctrName string) string { + profile := apparmor.GetProfileNameFromPodAnnotations(annotations, ctrName) + + if profile == "" { + return "" + } + + if profile == apparmor.ProfileRuntimeDefault { + // If the value is runtime/default, then return default profile. + return c.appArmorProfile + } + + return strings.TrimPrefix(profile, apparmor.ProfileNamePrefix) +} + +// openContainerFile opens a file inside a container rootfs safely +func openContainerFile(rootfs string, path string) (io.ReadCloser, error) { + fp, err := symlink.FollowSymlinkInScope(filepath.Join(rootfs, path), rootfs) + if err != nil { + return nil, err + } + return os.Open(fp) +} + +// getUserInfo returns UID, GID and additional groups for specified user +// by looking them up in /etc/passwd and /etc/group +func getUserInfo(rootfs string, userName string) (uint32, uint32, []uint32, error) { + // We don't care if we can't open the file because + // not all images will have these files + passwdFile, err := openContainerFile(rootfs, "/etc/passwd") + if err != nil { + logrus.Warnf("Failed to open /etc/passwd: %v", err) + } else { + defer passwdFile.Close() + } + + groupFile, err := openContainerFile(rootfs, "/etc/group") + if err != nil { + logrus.Warnf("Failed to open /etc/group: %v", err) + } else { + defer groupFile.Close() + } + + execUser, err := user.GetExecUser(userName, nil, passwdFile, groupFile) + if err != nil { + return 0, 0, nil, err + } + + uid := uint32(execUser.Uid) + gid := uint32(execUser.Gid) + var additionalGids []uint32 + for _, g := range execUser.Sgids { + additionalGids = append(additionalGids, uint32(g)) + } + + return uid, gid, additionalGids, nil +} diff --git a/libkpod/naming.go b/libkpod/naming.go new file mode 100644 index 00000000..2e73f2ad --- /dev/null +++ b/libkpod/naming.go @@ -0,0 +1,41 @@ +package libkpod + +import ( + "fmt" + "strings" + + "github.com/docker/docker/pkg/stringid" + pb "k8s.io/kubernetes/pkg/kubelet/api/v1alpha1/runtime" +) + +const ( + // KubePrefix is the name prefix for kubernetes pods + KubePrefix = "k8s" + // InfraName is used to identify infra containers + InfraName = "infra" + // NameDelimiter is the delimeter between parts of a name + NameDelimiter = "_" +) + +func makeContainerName(sandboxMetadata *pb.PodSandboxMetadata, containerConfig *pb.ContainerConfig) string { + return strings.Join([]string{ + KubePrefix, + containerConfig.Metadata.Name, + sandboxMetadata.Name, + sandboxMetadata.Namespace, + sandboxMetadata.Uid, + fmt.Sprintf("%d", containerConfig.Metadata.Attempt), + }, NameDelimiter) +} + +func (c *ContainerServer) generateContainerIDandName(sandboxMetadata *pb.PodSandboxMetadata, containerConfig *pb.ContainerConfig) (string, string, error) { + var ( + err error + id = stringid.GenerateNonCryptoID() + ) + name, err := c.ReserveContainerName(id, makeContainerName(sandboxMetadata, containerConfig)) + if err != nil { + return "", "", err + } + return id, name, err +} diff --git a/server/container_create.go b/server/container_create.go index 9e49d718..466aba6d 100644 --- a/server/container_create.go +++ b/server/container_create.go @@ -1,829 +1,22 @@ package server import ( - "encoding/json" - "errors" - "fmt" - "io" - "os" - "path/filepath" - "strconv" - "strings" - "time" - - "github.com/docker/docker/pkg/stringid" - "github.com/docker/docker/pkg/symlink" - "github.com/kubernetes-incubator/cri-o/libkpod" - "github.com/kubernetes-incubator/cri-o/libkpod/sandbox" - "github.com/kubernetes-incubator/cri-o/oci" - "github.com/kubernetes-incubator/cri-o/pkg/annotations" - "github.com/kubernetes-incubator/cri-o/pkg/storage" - "github.com/kubernetes-incubator/cri-o/server/apparmor" - "github.com/kubernetes-incubator/cri-o/server/seccomp" - "github.com/opencontainers/image-spec/specs-go/v1" - "github.com/opencontainers/runc/libcontainer/cgroups" - "github.com/opencontainers/runc/libcontainer/devices" - "github.com/opencontainers/runc/libcontainer/user" - rspec "github.com/opencontainers/runtime-spec/specs-go" - "github.com/opencontainers/runtime-tools/generate" - "github.com/opencontainers/selinux/go-selinux/label" "github.com/sirupsen/logrus" "golang.org/x/net/context" - "golang.org/x/sys/unix" pb "k8s.io/kubernetes/pkg/kubelet/api/v1alpha1/runtime" ) -const ( - seccompUnconfined = "unconfined" - seccompRuntimeDefault = "runtime/default" - seccompLocalhostPrefix = "localhost/" -) - -func addOCIBindMounts(sb *sandbox.Sandbox, containerConfig *pb.ContainerConfig, specgen *generate.Generator) error { - mounts := containerConfig.GetMounts() - for _, mount := range mounts { - dest := mount.ContainerPath - if dest == "" { - return fmt.Errorf("Mount.ContainerPath is empty") - } - - src := mount.HostPath - if src == "" { - return fmt.Errorf("Mount.HostPath is empty") - } - - if _, err := os.Stat(src); err != nil && os.IsNotExist(err) { - if err1 := os.MkdirAll(src, 0644); err1 != nil { - return fmt.Errorf("Failed to mkdir %s: %s", src, err) - } - } - - options := []string{"rw"} - if mount.Readonly { - options = []string{"ro"} - } - - if mount.SelinuxRelabel { - // Need a way in kubernetes to determine if the volume is shared or private - if err := label.Relabel(src, sb.MountLabel(), true); err != nil && err != unix.ENOTSUP { - return fmt.Errorf("relabel failed %s: %v", src, err) - } - } - - specgen.AddBindMount(src, dest, options) - } - - return nil -} - -func addImageVolumes(rootfs string, s *Server, containerInfo *storage.ContainerInfo, specgen *generate.Generator, mountLabel string) error { - for dest := range containerInfo.Config.Config.Volumes { - fp, err := symlink.FollowSymlinkInScope(filepath.Join(rootfs, dest), rootfs) - if err != nil { - return err - } - switch s.config.ImageVolumes { - case libkpod.ImageVolumesMkdir: - if err1 := os.MkdirAll(fp, 0644); err1 != nil { - return err1 - } - case libkpod.ImageVolumesBind: - volumeDirName := stringid.GenerateNonCryptoID() - src := filepath.Join(containerInfo.RunDir, "mounts", volumeDirName) - if err1 := os.MkdirAll(src, 0644); err1 != nil { - return err1 - } - // Label the source with the sandbox selinux mount label - if mountLabel != "" { - if err1 := label.Relabel(src, mountLabel, true); err1 != nil && err1 != unix.ENOTSUP { - return fmt.Errorf("relabel failed %s: %v", src, err1) - } - } - - logrus.Debugf("Adding bind mounted volume: %s to %s", src, dest) - specgen.AddBindMount(src, dest, []string{"rw"}) - case libkpod.ImageVolumesIgnore: - logrus.Debugf("Ignoring volume %v", dest) - default: - logrus.Fatalf("Unrecognized image volumes setting") - } - } - return nil -} - -func addDevices(sb *sandbox.Sandbox, containerConfig *pb.ContainerConfig, specgen *generate.Generator) error { - sp := specgen.Spec() - for _, device := range containerConfig.GetDevices() { - dev, err := devices.DeviceFromPath(device.HostPath, device.Permissions) - if err != nil { - return fmt.Errorf("failed to add device: %v", err) - } - rd := rspec.LinuxDevice{ - Path: device.ContainerPath, - Type: string(dev.Type), - Major: dev.Major, - Minor: dev.Minor, - UID: &dev.Uid, - GID: &dev.Gid, - } - specgen.AddDevice(rd) - sp.Linux.Resources.Devices = append(sp.Linux.Resources.Devices, rspec.LinuxDeviceCgroup{ - Allow: true, - Type: string(dev.Type), - Major: &dev.Major, - Minor: &dev.Minor, - Access: dev.Permissions, - }) - } - return nil -} - -// buildOCIProcessArgs build an OCI compatible process arguments slice. -func buildOCIProcessArgs(containerKubeConfig *pb.ContainerConfig, imageOCIConfig *v1.Image) ([]string, error) { - //# Start the nginx container using the default command, but use custom - //arguments (arg1 .. argN) for that command. - //kubectl run nginx --image=nginx -- ... - - //# Start the nginx container using a different command and custom arguments. - //kubectl run nginx --image=nginx --command -- ... - - kubeCommands := containerKubeConfig.Command - kubeArgs := containerKubeConfig.Args - - // merge image config and kube config - // same as docker does today... - if imageOCIConfig != nil { - if len(kubeCommands) == 0 { - if len(kubeArgs) == 0 { - kubeArgs = imageOCIConfig.Config.Cmd - } - if kubeCommands == nil { - kubeCommands = imageOCIConfig.Config.Entrypoint - } - } - } - - if len(kubeCommands) == 0 && len(kubeArgs) == 0 { - return nil, fmt.Errorf("no command specified") - } - - // create entrypoint and args - var entrypoint string - var args []string - if len(kubeCommands) != 0 { - entrypoint = kubeCommands[0] - args = append(kubeCommands[1:], kubeArgs...) - } else { - entrypoint = kubeArgs[0] - args = kubeArgs[1:] - } - - processArgs := append([]string{entrypoint}, args...) - - logrus.Debugf("OCI process args %v", processArgs) - - return processArgs, nil -} - -// setupContainerUser sets the UID, GID and supplemental groups in OCI runtime config -func setupContainerUser(specgen *generate.Generator, rootfs string, sc *pb.LinuxContainerSecurityContext, imageConfig *v1.Image) error { - if sc != nil { - containerUser := "" - // Case 1: run as user is set by kubelet - if sc.RunAsUser != nil { - containerUser = strconv.FormatInt(sc.GetRunAsUser().Value, 10) - } else { - // Case 2: run as username is set by kubelet - userName := sc.RunAsUsername - if userName != "" { - containerUser = userName - } else { - // Case 3: get user from image config - if imageConfig != nil { - imageUser := imageConfig.Config.User - if imageUser != "" { - containerUser = imageUser - } - } - } - } - - logrus.Debugf("CONTAINER USER: %+v", containerUser) - - // Add uid, gid and groups from user - uid, gid, addGroups, err1 := getUserInfo(rootfs, containerUser) - if err1 != nil { - return err1 - } - - logrus.Debugf("UID: %v, GID: %v, Groups: %+v", uid, gid, addGroups) - specgen.SetProcessUID(uid) - specgen.SetProcessGID(gid) - for _, group := range addGroups { - specgen.AddProcessAdditionalGid(group) - } - - // Add groups from CRI - groups := sc.SupplementalGroups - for _, group := range groups { - specgen.AddProcessAdditionalGid(uint32(group)) - } - } - return nil -} - -func hostNetwork(containerConfig *pb.ContainerConfig) bool { - securityContext := containerConfig.GetLinux().GetSecurityContext() - if securityContext == nil || securityContext.GetNamespaceOptions() == nil { - return false - } - - return securityContext.GetNamespaceOptions().HostNetwork -} - -// ensureSaneLogPath is a hack to fix https://issues.k8s.io/44043 which causes -// logPath to be a broken symlink to some magical Docker path. Ideally we -// wouldn't have to deal with this, but until that issue is fixed we have to -// remove the path if it's a broken symlink. -func ensureSaneLogPath(logPath string) error { - // If the path exists but the resolved path does not, then we have a broken - // symlink and we need to remove it. - fi, err := os.Lstat(logPath) - if err != nil || fi.Mode()&os.ModeSymlink == 0 { - // Non-existant files and non-symlinks aren't our problem. - return nil - } - - _, err = os.Stat(logPath) - if os.IsNotExist(err) { - err = os.RemoveAll(logPath) - if err != nil { - return fmt.Errorf("ensureSaneLogPath remove bad logPath: %s", err) - } - } - return nil -} - -// CreateContainer creates a new container in specified PodSandbox +// CreateContainer creates a new container in the specified PodSandbox func (s *Server) CreateContainer(ctx context.Context, req *pb.CreateContainerRequest) (res *pb.CreateContainerResponse, err error) { - logrus.Debugf("CreateContainerRequest %+v", req) - - s.updateLock.RLock() - defer s.updateLock.RUnlock() - - sbID := req.PodSandboxId - if sbID == "" { - return nil, fmt.Errorf("PodSandboxId should not be empty") - } - - sandboxID, err := s.PodIDIndex().Get(sbID) - if err != nil { - return nil, fmt.Errorf("PodSandbox with ID starting with %s not found: %v", sbID, err) - } - - sb := s.getSandbox(sandboxID) - if sb == nil { - return nil, fmt.Errorf("specified sandbox not found: %s", sandboxID) - } - - // The config of the container - containerConfig := req.GetConfig() - if containerConfig == nil { - return nil, fmt.Errorf("CreateContainerRequest.ContainerConfig is nil") - } - - name := containerConfig.GetMetadata().Name - if name == "" { - return nil, fmt.Errorf("CreateContainerRequest.ContainerConfig.Name is empty") - } - - containerID, containerName, err := s.generateContainerIDandName(sb.Metadata(), containerConfig) + ctr, err := s.ContainerCreate(ctx, req) if err != nil { return nil, err } - defer func() { - if err != nil { - s.ReleaseContainerName(containerName) - } - }() - - container, err := s.createSandboxContainer(ctx, containerID, containerName, sb, req.GetSandboxConfig(), containerConfig) - if err != nil { - return nil, err - } - defer func() { - if err != nil { - err2 := s.StorageRuntimeServer().DeleteContainer(containerID) - if err2 != nil { - logrus.Warnf("Failed to cleanup container directory: %v", err2) - } - } - }() - - if err = s.Runtime().CreateContainer(container, sb.CgroupParent()); err != nil { - return nil, err - } - - if err = s.Runtime().UpdateStatus(container); err != nil { - return nil, err - } - - s.addContainer(container) - - if err = s.CtrIDIndex().Add(containerID); err != nil { - s.removeContainer(container) - return nil, err - } - - s.ContainerStateToDisk(container) - resp := &pb.CreateContainerResponse{ - ContainerId: containerID, + ContainerId: ctr.ID(), } logrus.Debugf("CreateContainerResponse: %+v", resp) return resp, nil } - -func (s *Server) createSandboxContainer(ctx context.Context, containerID string, containerName string, sb *sandbox.Sandbox, SandboxConfig *pb.PodSandboxConfig, containerConfig *pb.ContainerConfig) (*oci.Container, error) { - if sb == nil { - return nil, errors.New("createSandboxContainer needs a sandbox") - } - - // TODO: simplify this function (cyclomatic complexity here is high) - // TODO: factor generating/updating the spec into something other projects can vendor - - // creates a spec Generator with the default spec. - specgen := generate.New() - specgen.HostSpecific = true - specgen.ClearProcessRlimits() - - if err := addOCIBindMounts(sb, containerConfig, &specgen); err != nil { - return nil, err - } - - // Add cgroup mount so container process can introspect its own limits - specgen.AddCgroupsMount("ro") - - if err := addDevices(sb, containerConfig, &specgen); err != nil { - return nil, err - } - - labels := containerConfig.GetLabels() - - metadata := containerConfig.GetMetadata() - - kubeAnnotations := containerConfig.GetAnnotations() - if kubeAnnotations != nil { - for k, v := range kubeAnnotations { - specgen.AddAnnotation(k, v) - } - } - - // set this container's apparmor profile if it is set by sandbox - if s.appArmorEnabled { - appArmorProfileName := s.getAppArmorProfileName(sb.Annotations(), metadata.Name) - if appArmorProfileName != "" { - // reload default apparmor profile if it is unloaded. - if s.appArmorProfile == apparmor.DefaultApparmorProfile { - if err := apparmor.EnsureDefaultApparmorProfile(); err != nil { - return nil, err - } - } - - specgen.SetProcessApparmorProfile(appArmorProfileName) - } - } - var readOnlyRootfs bool - if containerConfig.GetLinux().GetSecurityContext() != nil { - if containerConfig.GetLinux().GetSecurityContext().Privileged { - specgen.SetupPrivileged(true) - } - - if containerConfig.GetLinux().GetSecurityContext().ReadonlyRootfs { - readOnlyRootfs = true - specgen.SetRootReadonly(true) - } - } - - logPath := containerConfig.LogPath - if logPath == "" { - // TODO: Should we use sandboxConfig.GetLogDirectory() here? - logPath = filepath.Join(sb.LogDir(), containerID+".log") - } - if !filepath.IsAbs(logPath) { - // XXX: It's not really clear what this should be versus the sbox logDirectory. - logrus.Warnf("requested logPath for ctr id %s is a relative path: %s", containerID, logPath) - logPath = filepath.Join(sb.LogDir(), logPath) - } - - // Handle https://issues.k8s.io/44043 - if err := ensureSaneLogPath(logPath); err != nil { - return nil, err - } - - logrus.WithFields(logrus.Fields{ - "sbox.logdir": sb.LogDir(), - "ctr.logfile": containerConfig.LogPath, - "log_path": logPath, - }).Debugf("setting container's log_path") - - specgen.SetProcessTerminal(containerConfig.Tty) - - linux := containerConfig.GetLinux() - if linux != nil { - resources := linux.GetResources() - if resources != nil { - cpuPeriod := resources.CpuPeriod - if cpuPeriod != 0 { - specgen.SetLinuxResourcesCPUPeriod(uint64(cpuPeriod)) - } - - cpuQuota := resources.CpuQuota - if cpuQuota != 0 { - specgen.SetLinuxResourcesCPUQuota(cpuQuota) - } - - cpuShares := resources.CpuShares - if cpuShares != 0 { - specgen.SetLinuxResourcesCPUShares(uint64(cpuShares)) - } - - memoryLimit := resources.MemoryLimitInBytes - if memoryLimit != 0 { - specgen.SetLinuxResourcesMemoryLimit(memoryLimit) - } - - oomScoreAdj := resources.OomScoreAdj - specgen.SetProcessOOMScoreAdj(int(oomScoreAdj)) - } - - if sb.CgroupParent() != "" { - if s.config.CgroupManager == "systemd" { - cgPath := sb.CgroupParent() + ":" + "crio" + ":" + containerID - specgen.SetLinuxCgroupsPath(cgPath) - } else { - specgen.SetLinuxCgroupsPath(sb.CgroupParent() + "/" + containerID) - } - } - - capabilities := linux.GetSecurityContext().GetCapabilities() - toCAPPrefixed := func(cap string) string { - if !strings.HasPrefix(strings.ToLower(cap), "cap_") { - return "CAP_" + cap - } - return cap - } - if capabilities != nil { - addCaps := capabilities.AddCapabilities - if addCaps != nil { - for _, cap := range addCaps { - if err := specgen.AddProcessCapability(toCAPPrefixed(cap)); err != nil { - return nil, err - } - } - } - - dropCaps := capabilities.DropCapabilities - if dropCaps != nil { - for _, cap := range dropCaps { - if err := specgen.DropProcessCapability(toCAPPrefixed(cap)); err != nil { - logrus.Debugf("failed to drop cap %s: %v", toCAPPrefixed(cap), err) - } - } - } - } - - specgen.SetProcessSelinuxLabel(sb.ProcessLabel()) - specgen.SetLinuxMountLabel(sb.MountLabel()) - - if containerConfig.GetLinux().GetSecurityContext() != nil && - !containerConfig.GetLinux().GetSecurityContext().Privileged { - for _, mp := range []string{ - "/proc/kcore", - "/proc/latency_stats", - "/proc/timer_list", - "/proc/timer_stats", - "/proc/sched_debug", - "/sys/firmware", - } { - specgen.AddLinuxMaskedPaths(mp) - } - - for _, rp := range []string{ - "/proc/asound", - "/proc/bus", - "/proc/fs", - "/proc/irq", - "/proc/sys", - "/proc/sysrq-trigger", - } { - specgen.AddLinuxReadonlyPaths(rp) - } - } - } - // Join the namespace paths for the pod sandbox container. - podInfraState := s.Runtime().ContainerStatus(sb.InfraContainer()) - - logrus.Debugf("pod container state %+v", podInfraState) - - ipcNsPath := fmt.Sprintf("/proc/%d/ns/ipc", podInfraState.Pid) - if err := specgen.AddOrReplaceLinuxNamespace("ipc", ipcNsPath); err != nil { - return nil, err - } - - netNsPath := sb.NetNsPath() - if netNsPath == "" { - // The sandbox does not have a permanent namespace, - // it's on the host one. - netNsPath = fmt.Sprintf("/proc/%d/ns/net", podInfraState.Pid) - } - - if err := specgen.AddOrReplaceLinuxNamespace("network", netNsPath); err != nil { - return nil, err - } - - imageSpec := containerConfig.GetImage() - if imageSpec == nil { - return nil, fmt.Errorf("CreateContainerRequest.ContainerConfig.Image is nil") - } - - image := imageSpec.Image - if image == "" { - return nil, fmt.Errorf("CreateContainerRequest.ContainerConfig.Image.Image is empty") - } - images, err := s.StorageImageServer().ResolveNames(image) - if err != nil { - // This means we got an image ID - if strings.Contains(err.Error(), "cannot specify 64-byte hexadecimal strings") { - images = append(images, image) - } else { - return nil, err - } - } - image = images[0] - - // bind mount the pod shm - specgen.AddBindMount(sb.ShmPath(), "/dev/shm", []string{"rw"}) - - options := []string{"rw"} - if readOnlyRootfs { - options = []string{"ro"} - } - if sb.ResolvPath() != "" { - // bind mount the pod resolver file - specgen.AddBindMount(sb.ResolvPath(), "/etc/resolv.conf", options) - } - - // Bind mount /etc/hosts for host networking containers - if hostNetwork(containerConfig) { - specgen.AddBindMount("/etc/hosts", "/etc/hosts", options) - } - - if sb.Hostname() != "" { - specgen.SetHostname(sb.Hostname()) - } - - specgen.AddAnnotation(annotations.Name, containerName) - specgen.AddAnnotation(annotations.ContainerID, containerID) - specgen.AddAnnotation(annotations.SandboxID, sb.ID()) - specgen.AddAnnotation(annotations.SandboxName, sb.InfraContainer().Name()) - specgen.AddAnnotation(annotations.ContainerType, annotations.ContainerTypeContainer) - specgen.AddAnnotation(annotations.LogPath, logPath) - specgen.AddAnnotation(annotations.TTY, fmt.Sprintf("%v", containerConfig.Tty)) - specgen.AddAnnotation(annotations.Stdin, fmt.Sprintf("%v", containerConfig.Stdin)) - specgen.AddAnnotation(annotations.StdinOnce, fmt.Sprintf("%v", containerConfig.StdinOnce)) - specgen.AddAnnotation(annotations.Image, image) - - created := time.Now() - specgen.AddAnnotation(annotations.Created, created.Format(time.RFC3339Nano)) - - metadataJSON, err := json.Marshal(metadata) - if err != nil { - return nil, err - } - specgen.AddAnnotation(annotations.Metadata, string(metadataJSON)) - - labelsJSON, err := json.Marshal(labels) - if err != nil { - return nil, err - } - specgen.AddAnnotation(annotations.Labels, string(labelsJSON)) - - kubeAnnotationsJSON, err := json.Marshal(kubeAnnotations) - if err != nil { - return nil, err - } - specgen.AddAnnotation(annotations.Annotations, string(kubeAnnotationsJSON)) - - if err = s.setupSeccomp(&specgen, containerName, sb.Annotations()); err != nil { - return nil, err - } - - metaname := metadata.Name - attempt := metadata.Attempt - containerInfo, err := s.StorageRuntimeServer().CreateContainer(s.ImageContext(), - sb.Name(), sb.ID(), - image, image, - containerName, containerID, - metaname, - attempt, - sb.MountLabel(), - nil) - if err != nil { - return nil, err - } - - mountPoint, err := s.StorageRuntimeServer().StartContainer(containerID) - if err != nil { - return nil, fmt.Errorf("failed to mount container %s(%s): %v", containerName, containerID, err) - } - - containerImageConfig := containerInfo.Config - if containerImageConfig == nil { - return nil, fmt.Errorf("empty image config for %s", image) - } - - if containerImageConfig.Config.StopSignal != "" { - // this key is defined in image-spec conversion document at https://github.com/opencontainers/image-spec/pull/492/files#diff-8aafbe2c3690162540381b8cdb157112R57 - specgen.AddAnnotation("org.opencontainers.image.stopSignal", containerImageConfig.Config.StopSignal) - } - - // Add image volumes - if err := addImageVolumes(mountPoint, s, &containerInfo, &specgen, sb.MountLabel()); err != nil { - return nil, err - } - - processArgs, err := buildOCIProcessArgs(containerConfig, containerImageConfig) - if err != nil { - return nil, err - } - specgen.SetProcessArgs(processArgs) - - // Add environment variables from CRI and image config - envs := containerConfig.GetEnvs() - if envs != nil { - for _, item := range envs { - key := item.Key - value := item.Value - if key == "" { - continue - } - specgen.AddProcessEnv(key, value) - } - } - if containerImageConfig != nil { - for _, item := range containerImageConfig.Config.Env { - parts := strings.SplitN(item, "=", 2) - if len(parts) != 2 { - return nil, fmt.Errorf("invalid env from image: %s", item) - } - - if parts[0] == "" { - continue - } - specgen.AddProcessEnv(parts[0], parts[1]) - } - } - - // Set working directory - // Pick it up from image config first and override if specified in CRI - containerCwd := "/" - if containerImageConfig != nil { - imageCwd := containerImageConfig.Config.WorkingDir - if imageCwd != "" { - containerCwd = imageCwd - } - } - runtimeCwd := containerConfig.WorkingDir - if runtimeCwd != "" { - containerCwd = runtimeCwd - } - specgen.SetProcessCwd(containerCwd) - - // Setup user and groups - if linux != nil { - if err = setupContainerUser(&specgen, mountPoint, linux.GetSecurityContext(), containerImageConfig); err != nil { - return nil, err - } - } - - // Set up pids limit if pids cgroup is mounted - _, err = cgroups.FindCgroupMountpoint("pids") - if err == nil { - specgen.SetLinuxResourcesPidsLimit(s.config.PidsLimit) - } - - // by default, the root path is an empty string. set it now. - specgen.SetRootPath(mountPoint) - - saveOptions := generate.ExportOptions{} - if err = specgen.SaveToFile(filepath.Join(containerInfo.Dir, "config.json"), saveOptions); err != nil { - return nil, err - } - if err = specgen.SaveToFile(filepath.Join(containerInfo.RunDir, "config.json"), saveOptions); err != nil { - return nil, err - } - - container, err := oci.NewContainer(containerID, containerName, containerInfo.RunDir, logPath, sb.NetNs(), labels, kubeAnnotations, image, metadata, sb.ID(), containerConfig.Tty, containerConfig.Stdin, containerConfig.StdinOnce, sb.Privileged(), sb.Trusted(), containerInfo.Dir, created, containerImageConfig.Config.StopSignal) - if err != nil { - return nil, err - } - - return container, nil -} - -func (s *Server) setupSeccomp(specgen *generate.Generator, cname string, sbAnnotations map[string]string) error { - profile, ok := sbAnnotations["security.alpha.kubernetes.io/seccomp/container/"+cname] - if !ok { - profile, ok = sbAnnotations["security.alpha.kubernetes.io/seccomp/pod"] - if !ok { - // running w/o seccomp, aka unconfined - profile = seccompUnconfined - } - } - if !s.seccompEnabled { - if profile != seccompUnconfined { - return fmt.Errorf("seccomp is not enabled in your kernel, cannot run with a profile") - } - logrus.Warn("seccomp is not enabled in your kernel, running container without profile") - } - if profile == seccompUnconfined { - // running w/o seccomp, aka unconfined - specgen.Spec().Linux.Seccomp = nil - return nil - } - if profile == seccompRuntimeDefault { - return seccomp.LoadProfileFromStruct(s.seccompProfile, specgen) - } - if !strings.HasPrefix(profile, seccompLocalhostPrefix) { - return fmt.Errorf("unknown seccomp profile option: %q", profile) - } - //file, err := ioutil.ReadFile(filepath.Join(s.seccompProfileRoot, strings.TrimPrefix(profile, seccompLocalhostPrefix))) - //if err != nil { - //return err - //} - // TODO(runcom): setup from provided node's seccomp profile - // can't do this yet, see https://issues.k8s.io/36997 - return nil -} - -// getAppArmorProfileName gets the profile name for the given container. -func (s *Server) getAppArmorProfileName(annotations map[string]string, ctrName string) string { - profile := apparmor.GetProfileNameFromPodAnnotations(annotations, ctrName) - - if profile == "" { - return "" - } - - if profile == apparmor.ProfileRuntimeDefault { - // If the value is runtime/default, then return default profile. - return s.appArmorProfile - } - - return strings.TrimPrefix(profile, apparmor.ProfileNamePrefix) -} - -// openContainerFile opens a file inside a container rootfs safely -func openContainerFile(rootfs string, path string) (io.ReadCloser, error) { - fp, err := symlink.FollowSymlinkInScope(filepath.Join(rootfs, path), rootfs) - if err != nil { - return nil, err - } - return os.Open(fp) -} - -// getUserInfo returns UID, GID and additional groups for specified user -// by looking them up in /etc/passwd and /etc/group -func getUserInfo(rootfs string, userName string) (uint32, uint32, []uint32, error) { - // We don't care if we can't open the file because - // not all images will have these files - passwdFile, err := openContainerFile(rootfs, "/etc/passwd") - if err != nil { - logrus.Warnf("Failed to open /etc/passwd: %v", err) - } else { - defer passwdFile.Close() - } - - groupFile, err := openContainerFile(rootfs, "/etc/group") - if err != nil { - logrus.Warnf("Failed to open /etc/group: %v", err) - } else { - defer groupFile.Close() - } - - execUser, err := user.GetExecUser(userName, nil, passwdFile, groupFile) - if err != nil { - return 0, 0, nil, err - } - - uid := uint32(execUser.Uid) - gid := uint32(execUser.Gid) - var additionalGids []uint32 - for _, g := range execUser.Sgids { - additionalGids = append(additionalGids, uint32(g)) - } - - return uid, gid, additionalGids, nil -} diff --git a/server/naming.go b/server/naming.go index 2d1e2d5b..6c92de07 100644 --- a/server/naming.go +++ b/server/naming.go @@ -5,45 +5,29 @@ import ( "strings" "github.com/docker/docker/pkg/stringid" + "github.com/kubernetes-incubator/cri-o/libkpod" pb "k8s.io/kubernetes/pkg/kubelet/api/v1alpha1/runtime" ) -const ( - kubePrefix = "k8s" - infraName = "infra" - nameDelimiter = "_" -) - func makeSandboxName(sandboxConfig *pb.PodSandboxConfig) string { return strings.Join([]string{ - kubePrefix, + libkpod.KubePrefix, sandboxConfig.Metadata.Name, sandboxConfig.Metadata.Namespace, sandboxConfig.Metadata.Uid, fmt.Sprintf("%d", sandboxConfig.Metadata.Attempt), - }, nameDelimiter) + }, libkpod.NameDelimiter) } func makeSandboxContainerName(sandboxConfig *pb.PodSandboxConfig) string { return strings.Join([]string{ - kubePrefix, - infraName, + libkpod.KubePrefix, + libkpod.InfraName, sandboxConfig.Metadata.Name, sandboxConfig.Metadata.Namespace, sandboxConfig.Metadata.Uid, fmt.Sprintf("%d", sandboxConfig.Metadata.Attempt), - }, nameDelimiter) -} - -func makeContainerName(sandboxMetadata *pb.PodSandboxMetadata, containerConfig *pb.ContainerConfig) string { - return strings.Join([]string{ - kubePrefix, - containerConfig.Metadata.Name, - sandboxMetadata.Name, - sandboxMetadata.Namespace, - sandboxMetadata.Uid, - fmt.Sprintf("%d", containerConfig.Metadata.Attempt), - }, nameDelimiter) + }, libkpod.NameDelimiter) } func (s *Server) generatePodIDandName(sandboxConfig *pb.PodSandboxConfig) (string, string, error) { @@ -72,15 +56,3 @@ func (s *Server) generateContainerIDandNameForSandbox(sandboxConfig *pb.PodSandb } return id, name, err } - -func (s *Server) generateContainerIDandName(sandboxMetadata *pb.PodSandboxMetadata, containerConfig *pb.ContainerConfig) (string, string, error) { - var ( - err error - id = stringid.GenerateNonCryptoID() - ) - name, err := s.ReserveContainerName(id, makeContainerName(sandboxMetadata, containerConfig)) - if err != nil { - return "", "", err - } - return id, name, err -} diff --git a/server/sandbox_run.go b/server/sandbox_run.go index 8ee1e054..0b1e42da 100644 --- a/server/sandbox_run.go +++ b/server/sandbox_run.go @@ -12,6 +12,7 @@ import ( "time" "github.com/containers/storage" + "github.com/kubernetes-incubator/cri-o/libkpod" "github.com/kubernetes-incubator/cri-o/libkpod/sandbox" "github.com/kubernetes-incubator/cri-o/oci" "github.com/kubernetes-incubator/cri-o/pkg/annotations" @@ -301,7 +302,7 @@ func (s *Server) RunPodSandbox(ctx context.Context, req *pb.RunPodSandboxRequest logPath := filepath.Join(logDir, id+".log") // Handle https://issues.k8s.io/44043 - if err := ensureSaneLogPath(logPath); err != nil { + if err := libkpod.EnsureSaneLogPath(logPath); err != nil { return nil, err } @@ -443,7 +444,7 @@ func (s *Server) RunPodSandbox(ctx context.Context, req *pb.RunPodSandboxRequest } } - if !s.seccompEnabled { + if !s.SeccompEnabled() { g.Spec().Linux.Seccomp = nil } diff --git a/server/server.go b/server/server.go index c5e26c4f..3de28752 100644 --- a/server/server.go +++ b/server/server.go @@ -1,9 +1,7 @@ package server import ( - "encoding/json" "fmt" - "io/ioutil" "net" "os" "sync" @@ -13,8 +11,6 @@ import ( "github.com/kubernetes-incubator/cri-o/oci" "github.com/kubernetes-incubator/cri-o/pkg/ocicni" "github.com/kubernetes-incubator/cri-o/pkg/storage" - "github.com/kubernetes-incubator/cri-o/server/apparmor" - "github.com/kubernetes-incubator/cri-o/server/seccomp" "github.com/pkg/errors" "github.com/sirupsen/logrus" knet "k8s.io/apimachinery/pkg/util/net" @@ -52,12 +48,6 @@ type Server struct { netPlugin ocicni.CNIPlugin hostportManager hostport.HostPortManager - seccompEnabled bool - seccompProfile seccomp.Seccomp - - appArmorEnabled bool - appArmorProfile string - stream streamService } @@ -165,27 +155,6 @@ func New(config *Config) (*Server, error) { netPlugin: netPlugin, hostportManager: hostportManager, config: *config, - seccompEnabled: seccomp.IsEnabled(), - appArmorEnabled: apparmor.IsEnabled(), - appArmorProfile: config.ApparmorProfile, - } - - if s.seccompEnabled { - seccompProfile, fileErr := ioutil.ReadFile(config.SeccompProfile) - if fileErr != nil { - return nil, fmt.Errorf("opening seccomp profile (%s) failed: %v", config.SeccompProfile, fileErr) - } - var seccompConfig seccomp.Seccomp - if jsonErr := json.Unmarshal(seccompProfile, &seccompConfig); jsonErr != nil { - return nil, fmt.Errorf("decoding seccomp profile failed: %v", jsonErr) - } - s.seccompProfile = seccompConfig - } - - if s.appArmorEnabled && s.appArmorProfile == apparmor.DefaultApparmorProfile { - if apparmorErr := apparmor.EnsureDefaultApparmorProfile(); apparmorErr != nil { - return nil, fmt.Errorf("ensuring the default apparmor profile is installed failed: %v", apparmorErr) - } } s.restore()