summaryrefslogtreecommitdiff
path: root/libpod
diff options
context:
space:
mode:
Diffstat (limited to 'libpod')
-rw-r--r--libpod/boltdb_state.go46
-rw-r--r--libpod/boltdb_state_internal.go7
-rw-r--r--libpod/container_graph.go24
-rw-r--r--libpod/container_graph_test.go26
-rw-r--r--libpod/container_internal.go2
-rw-r--r--libpod/image/image.go29
-rw-r--r--libpod/in_memory_state.go20
-rw-r--r--libpod/networking_linux.go27
-rw-r--r--libpod/pod_api.go4
-rw-r--r--libpod/runtime.go17
-rw-r--r--libpod/runtime_ctr.go15
-rw-r--r--libpod/runtime_renumber.go17
-rw-r--r--libpod/runtime_volume.go4
-rw-r--r--libpod/runtime_volume_linux.go37
-rw-r--r--libpod/state.go10
-rw-r--r--libpod/volume.go36
16 files changed, 229 insertions, 92 deletions
diff --git a/libpod/boltdb_state.go b/libpod/boltdb_state.go
index 176781f07..1de8d80c9 100644
--- a/libpod/boltdb_state.go
+++ b/libpod/boltdb_state.go
@@ -870,7 +870,7 @@ func (s *BoltState) RewritePodConfig(pod *Pod, newCfg *PodConfig) error {
newCfgJSON, err := json.Marshal(newCfg)
if err != nil {
- return errors.Wrapf(err, "error marshalling new configuration JSON for container %s", pod.ID())
+ return errors.Wrapf(err, "error marshalling new configuration JSON for pod %s", pod.ID())
}
db, err := s.getDBCon()
@@ -900,6 +900,50 @@ func (s *BoltState) RewritePodConfig(pod *Pod, newCfg *PodConfig) error {
return err
}
+// RewriteVolumeConfig rewrites a volume's configuration.
+// WARNING: This function is DANGEROUS. Do not use without reading the full
+// comment on this function in state.go.
+func (s *BoltState) RewriteVolumeConfig(volume *Volume, newCfg *VolumeConfig) error {
+ if !s.valid {
+ return define.ErrDBClosed
+ }
+
+ if !volume.valid {
+ return define.ErrVolumeRemoved
+ }
+
+ newCfgJSON, err := json.Marshal(newCfg)
+ if err != nil {
+ return errors.Wrapf(err, "error marshalling new configuration JSON for volume %q", volume.Name())
+ }
+
+ db, err := s.getDBCon()
+ if err != nil {
+ return err
+ }
+ defer s.deferredCloseDBCon(db)
+
+ err = db.Update(func(tx *bolt.Tx) error {
+ volBkt, err := getVolBucket(tx)
+ if err != nil {
+ return err
+ }
+
+ volDB := volBkt.Bucket([]byte(volume.Name()))
+ if volDB == nil {
+ volume.valid = false
+ return errors.Wrapf(define.ErrNoSuchVolume, "no volume with name %q found in DB", volume.Name())
+ }
+
+ if err := volDB.Put(configKey, newCfgJSON); err != nil {
+ return errors.Wrapf(err, "error updating volume %q config JSON", volume.Name())
+ }
+
+ return nil
+ })
+ return err
+}
+
// Pod retrieves a pod given its full ID
func (s *BoltState) Pod(id string) (*Pod, error) {
if id == "" {
diff --git a/libpod/boltdb_state_internal.go b/libpod/boltdb_state_internal.go
index 408ef7224..6e4179835 100644
--- a/libpod/boltdb_state_internal.go
+++ b/libpod/boltdb_state_internal.go
@@ -449,6 +449,13 @@ func (s *BoltState) getVolumeFromDB(name []byte, volume *Volume, volBkt *bolt.Bu
return errors.Wrapf(err, "error unmarshalling volume %s config from DB", string(name))
}
+ // Get the lock
+ lock, err := s.runtime.lockManager.RetrieveLock(volume.config.LockID)
+ if err != nil {
+ return errors.Wrapf(err, "error retrieving lock for volume %q", string(name))
+ }
+ volume.lock = lock
+
volume.runtime = s.runtime
volume.valid = true
diff --git a/libpod/container_graph.go b/libpod/container_graph.go
index 5aa51bc2f..f6988e1ac 100644
--- a/libpod/container_graph.go
+++ b/libpod/container_graph.go
@@ -16,14 +16,30 @@ type containerNode struct {
dependedOn []*containerNode
}
-type containerGraph struct {
+// ContainerGraph is a dependency graph based on a set of containers.
+type ContainerGraph struct {
nodes map[string]*containerNode
noDepNodes []*containerNode
notDependedOnNodes map[string]*containerNode
}
-func buildContainerGraph(ctrs []*Container) (*containerGraph, error) {
- graph := new(containerGraph)
+// DependencyMap returns the dependency graph as map with the key being a
+// container and the value being the containers the key depends on.
+func (cg *ContainerGraph) DependencyMap() (dependencies map[*Container][]*Container) {
+ dependencies = make(map[*Container][]*Container)
+ for _, node := range cg.nodes {
+ dependsOn := make([]*Container, len(node.dependsOn))
+ for i, d := range node.dependsOn {
+ dependsOn[i] = d.container
+ }
+ dependencies[node.container] = dependsOn
+ }
+ return dependencies
+}
+
+// BuildContainerGraph builds a dependency graph based on the container slice.
+func BuildContainerGraph(ctrs []*Container) (*ContainerGraph, error) {
+ graph := new(ContainerGraph)
graph.nodes = make(map[string]*containerNode)
graph.notDependedOnNodes = make(map[string]*containerNode)
@@ -78,7 +94,7 @@ func buildContainerGraph(ctrs []*Container) (*containerGraph, error) {
// Detect cycles in a container graph using Tarjan's strongly connected
// components algorithm
// Return true if a cycle is found, false otherwise
-func detectCycles(graph *containerGraph) (bool, error) {
+func detectCycles(graph *ContainerGraph) (bool, error) {
type nodeInfo struct {
index int
lowLink int
diff --git a/libpod/container_graph_test.go b/libpod/container_graph_test.go
index d1a52658d..38f03c59c 100644
--- a/libpod/container_graph_test.go
+++ b/libpod/container_graph_test.go
@@ -8,7 +8,7 @@ import (
)
func TestBuildContainerGraphNoCtrsIsEmpty(t *testing.T) {
- graph, err := buildContainerGraph([]*Container{})
+ graph, err := BuildContainerGraph([]*Container{})
assert.NoError(t, err)
assert.Equal(t, 0, len(graph.nodes))
assert.Equal(t, 0, len(graph.noDepNodes))
@@ -24,7 +24,7 @@ func TestBuildContainerGraphOneCtr(t *testing.T) {
ctr1, err := getTestCtr1(manager)
assert.NoError(t, err)
- graph, err := buildContainerGraph([]*Container{ctr1})
+ graph, err := BuildContainerGraph([]*Container{ctr1})
assert.NoError(t, err)
assert.Equal(t, 1, len(graph.nodes))
assert.Equal(t, 1, len(graph.noDepNodes))
@@ -49,7 +49,7 @@ func TestBuildContainerGraphTwoCtrNoEdge(t *testing.T) {
ctr2, err := getTestCtr2(manager)
assert.NoError(t, err)
- graph, err := buildContainerGraph([]*Container{ctr1, ctr2})
+ graph, err := BuildContainerGraph([]*Container{ctr1, ctr2})
assert.NoError(t, err)
assert.Equal(t, 2, len(graph.nodes))
assert.Equal(t, 2, len(graph.noDepNodes))
@@ -76,7 +76,7 @@ func TestBuildContainerGraphTwoCtrOneEdge(t *testing.T) {
assert.NoError(t, err)
ctr2.config.UserNsCtr = ctr1.config.ID
- graph, err := buildContainerGraph([]*Container{ctr1, ctr2})
+ graph, err := BuildContainerGraph([]*Container{ctr1, ctr2})
assert.NoError(t, err)
assert.Equal(t, 2, len(graph.nodes))
assert.Equal(t, 1, len(graph.noDepNodes))
@@ -99,7 +99,7 @@ func TestBuildContainerGraphTwoCtrCycle(t *testing.T) {
ctr2.config.UserNsCtr = ctr1.config.ID
ctr1.config.NetNsCtr = ctr2.config.ID
- _, err = buildContainerGraph([]*Container{ctr1, ctr2})
+ _, err = BuildContainerGraph([]*Container{ctr1, ctr2})
assert.Error(t, err)
}
@@ -116,7 +116,7 @@ func TestBuildContainerGraphThreeCtrNoEdges(t *testing.T) {
ctr3, err := getTestCtrN("3", manager)
assert.NoError(t, err)
- graph, err := buildContainerGraph([]*Container{ctr1, ctr2, ctr3})
+ graph, err := BuildContainerGraph([]*Container{ctr1, ctr2, ctr3})
assert.NoError(t, err)
assert.Equal(t, 3, len(graph.nodes))
assert.Equal(t, 3, len(graph.noDepNodes))
@@ -150,7 +150,7 @@ func TestBuildContainerGraphThreeContainersTwoInCycle(t *testing.T) {
ctr1.config.UserNsCtr = ctr2.config.ID
ctr2.config.IPCNsCtr = ctr1.config.ID
- _, err = buildContainerGraph([]*Container{ctr1, ctr2, ctr3})
+ _, err = BuildContainerGraph([]*Container{ctr1, ctr2, ctr3})
assert.Error(t, err)
}
@@ -170,7 +170,7 @@ func TestBuildContainerGraphThreeContainersCycle(t *testing.T) {
ctr2.config.IPCNsCtr = ctr3.config.ID
ctr3.config.NetNsCtr = ctr1.config.ID
- _, err = buildContainerGraph([]*Container{ctr1, ctr2, ctr3})
+ _, err = BuildContainerGraph([]*Container{ctr1, ctr2, ctr3})
assert.Error(t, err)
}
@@ -190,7 +190,7 @@ func TestBuildContainerGraphThreeContainersNoCycle(t *testing.T) {
ctr1.config.NetNsCtr = ctr3.config.ID
ctr2.config.IPCNsCtr = ctr3.config.ID
- graph, err := buildContainerGraph([]*Container{ctr1, ctr2, ctr3})
+ graph, err := BuildContainerGraph([]*Container{ctr1, ctr2, ctr3})
assert.NoError(t, err)
assert.Equal(t, 3, len(graph.nodes))
assert.Equal(t, 1, len(graph.noDepNodes))
@@ -215,7 +215,7 @@ func TestBuildContainerGraphFourContainersNoEdges(t *testing.T) {
ctr4, err := getTestCtrN("4", manager)
assert.NoError(t, err)
- graph, err := buildContainerGraph([]*Container{ctr1, ctr2, ctr3, ctr4})
+ graph, err := BuildContainerGraph([]*Container{ctr1, ctr2, ctr3, ctr4})
assert.NoError(t, err)
assert.Equal(t, 4, len(graph.nodes))
assert.Equal(t, 4, len(graph.noDepNodes))
@@ -256,7 +256,7 @@ func TestBuildContainerGraphFourContainersTwoInCycle(t *testing.T) {
ctr1.config.IPCNsCtr = ctr2.config.ID
ctr2.config.UserNsCtr = ctr1.config.ID
- _, err = buildContainerGraph([]*Container{ctr1, ctr2, ctr3, ctr4})
+ _, err = BuildContainerGraph([]*Container{ctr1, ctr2, ctr3, ctr4})
assert.Error(t, err)
}
@@ -280,7 +280,7 @@ func TestBuildContainerGraphFourContainersAllInCycle(t *testing.T) {
ctr3.config.NetNsCtr = ctr4.config.ID
ctr4.config.UTSNsCtr = ctr1.config.ID
- _, err = buildContainerGraph([]*Container{ctr1, ctr2, ctr3, ctr4})
+ _, err = BuildContainerGraph([]*Container{ctr1, ctr2, ctr3, ctr4})
assert.Error(t, err)
}
@@ -303,7 +303,7 @@ func TestBuildContainerGraphFourContainersNoneInCycle(t *testing.T) {
ctr1.config.NetNsCtr = ctr3.config.ID
ctr2.config.UserNsCtr = ctr3.config.ID
- graph, err := buildContainerGraph([]*Container{ctr1, ctr2, ctr3, ctr4})
+ graph, err := BuildContainerGraph([]*Container{ctr1, ctr2, ctr3, ctr4})
assert.NoError(t, err)
assert.Equal(t, 4, len(graph.nodes))
assert.Equal(t, 2, len(graph.noDepNodes))
diff --git a/libpod/container_internal.go b/libpod/container_internal.go
index 313f67963..f51b53e85 100644
--- a/libpod/container_internal.go
+++ b/libpod/container_internal.go
@@ -788,7 +788,7 @@ func (c *Container) startDependencies(ctx context.Context) error {
}
// Build a dependency graph of containers
- graph, err := buildContainerGraph(depCtrs)
+ graph, err := BuildContainerGraph(depCtrs)
if err != nil {
return errors.Wrapf(err, "error generating dependency graph for container %s", c.ID())
}
diff --git a/libpod/image/image.go b/libpod/image/image.go
index cb7c390c6..0be6eeeb9 100644
--- a/libpod/image/image.go
+++ b/libpod/image/image.go
@@ -5,13 +5,13 @@ import (
"encoding/json"
"fmt"
"io"
+ "io/ioutil"
"os"
"path/filepath"
"strings"
"syscall"
"time"
- types2 "github.com/containernetworking/cni/pkg/types"
cp "github.com/containers/image/copy"
"github.com/containers/image/directory"
dockerarchive "github.com/containers/image/docker/archive"
@@ -383,11 +383,6 @@ func (i *Image) Remove(ctx context.Context, force bool) error {
return nil
}
-// Decompose an Image
-func (i *Image) Decompose() error {
- return types2.NotImplementedError
-}
-
// TODO: Rework this method to not require an assembly of the fq name with transport
/*
// GetManifest tries to GET an images manifest, returns nil on success and err on failure
@@ -555,7 +550,7 @@ func (i *Image) UntagImage(tag string) error {
// PushImageToHeuristicDestination pushes the given image to "destination", which is heuristically parsed.
// Use PushImageToReference if the destination is known precisely.
-func (i *Image) PushImageToHeuristicDestination(ctx context.Context, destination, manifestMIMEType, authFile, signaturePolicyPath string, writer io.Writer, forceCompress bool, signingOptions SigningOptions, dockerRegistryOptions *DockerRegistryOptions, additionalDockerArchiveTags []reference.NamedTagged) error {
+func (i *Image) PushImageToHeuristicDestination(ctx context.Context, destination, manifestMIMEType, authFile, digestFile, signaturePolicyPath string, writer io.Writer, forceCompress bool, signingOptions SigningOptions, dockerRegistryOptions *DockerRegistryOptions, additionalDockerArchiveTags []reference.NamedTagged) error {
if destination == "" {
return errors.Wrapf(syscall.EINVAL, "destination image name must be specified")
}
@@ -573,11 +568,11 @@ func (i *Image) PushImageToHeuristicDestination(ctx context.Context, destination
return err
}
}
- return i.PushImageToReference(ctx, dest, manifestMIMEType, authFile, signaturePolicyPath, writer, forceCompress, signingOptions, dockerRegistryOptions, additionalDockerArchiveTags)
+ return i.PushImageToReference(ctx, dest, manifestMIMEType, authFile, digestFile, signaturePolicyPath, writer, forceCompress, signingOptions, dockerRegistryOptions, additionalDockerArchiveTags)
}
// PushImageToReference pushes the given image to a location described by the given path
-func (i *Image) PushImageToReference(ctx context.Context, dest types.ImageReference, manifestMIMEType, authFile, signaturePolicyPath string, writer io.Writer, forceCompress bool, signingOptions SigningOptions, dockerRegistryOptions *DockerRegistryOptions, additionalDockerArchiveTags []reference.NamedTagged) error {
+func (i *Image) PushImageToReference(ctx context.Context, dest types.ImageReference, manifestMIMEType, authFile, digestFile, signaturePolicyPath string, writer io.Writer, forceCompress bool, signingOptions SigningOptions, dockerRegistryOptions *DockerRegistryOptions, additionalDockerArchiveTags []reference.NamedTagged) error {
sc := GetSystemContext(signaturePolicyPath, authFile, forceCompress)
sc.BlobInfoCacheDir = filepath.Join(i.imageruntime.store.GraphRoot(), "cache")
@@ -599,10 +594,22 @@ func (i *Image) PushImageToReference(ctx context.Context, dest types.ImageRefere
copyOptions := getCopyOptions(sc, writer, nil, dockerRegistryOptions, signingOptions, manifestMIMEType, additionalDockerArchiveTags)
copyOptions.DestinationCtx.SystemRegistriesConfPath = registries.SystemRegistriesConfPath() // FIXME: Set this more globally. Probably no reason not to have it in every types.SystemContext, and to compute the value just once in one place.
// Copy the image to the remote destination
- _, err = cp.Image(ctx, policyContext, dest, src, copyOptions)
+ manifestBytes, err := cp.Image(ctx, policyContext, dest, src, copyOptions)
if err != nil {
return errors.Wrapf(err, "Error copying image to the remote destination")
}
+ digest, err := manifest.Digest(manifestBytes)
+ if err != nil {
+ return errors.Wrapf(err, "error computing digest of manifest of new image %q", transports.ImageName(dest))
+ }
+
+ logrus.Debugf("Successfully pushed %s with digest %s", transports.ImageName(dest), digest.String())
+
+ if digestFile != "" {
+ if err = ioutil.WriteFile(digestFile, []byte(digest.String()), 0644); err != nil {
+ return errors.Wrapf(err, "failed to write digest to file %q", digestFile)
+ }
+ }
i.newImageEvent(events.Push)
return nil
}
@@ -1358,7 +1365,7 @@ func (i *Image) Save(ctx context.Context, source, format, output string, moreTag
return err
}
}
- if err := i.PushImageToReference(ctx, destRef, manifestType, "", "", writer, compress, SigningOptions{}, &DockerRegistryOptions{}, additionaltags); err != nil {
+ if err := i.PushImageToReference(ctx, destRef, manifestType, "", "", "", writer, compress, SigningOptions{}, &DockerRegistryOptions{}, additionaltags); err != nil {
return errors.Wrapf(err, "unable to save %q", source)
}
i.newImageEvent(events.Save)
diff --git a/libpod/in_memory_state.go b/libpod/in_memory_state.go
index 7c4abd25d..a9b735327 100644
--- a/libpod/in_memory_state.go
+++ b/libpod/in_memory_state.go
@@ -425,6 +425,26 @@ func (s *InMemoryState) RewritePodConfig(pod *Pod, newCfg *PodConfig) error {
return nil
}
+// RewriteVolumeConfig rewrites a volume's configuration.
+// This function is DANGEROUS, even with in-memory state.
+// Please read the full comment in state.go before using it.
+func (s *InMemoryState) RewriteVolumeConfig(volume *Volume, newCfg *VolumeConfig) error {
+ if !volume.valid {
+ return define.ErrVolumeRemoved
+ }
+
+ // If the volume does not exist, return error
+ stateVol, ok := s.volumes[volume.Name()]
+ if !ok {
+ volume.valid = false
+ return errors.Wrapf(define.ErrNoSuchVolume, "volume with name %q not found in state", volume.Name())
+ }
+
+ stateVol.config = newCfg
+
+ return nil
+}
+
// Volume retrieves a volume from its full name
func (s *InMemoryState) Volume(name string) (*Volume, error) {
if name == "" {
diff --git a/libpod/networking_linux.go b/libpod/networking_linux.go
index bef3f7739..fd14b2f73 100644
--- a/libpod/networking_linux.go
+++ b/libpod/networking_linux.go
@@ -17,7 +17,6 @@ import (
cnitypes "github.com/containernetworking/cni/pkg/types/current"
"github.com/containernetworking/plugins/pkg/ns"
"github.com/containers/libpod/pkg/errorhandling"
- "github.com/containers/libpod/pkg/firewall"
"github.com/containers/libpod/pkg/netns"
"github.com/containers/libpod/pkg/rootless"
"github.com/cri-o/ocicni/pkg/ocicni"
@@ -86,18 +85,6 @@ func (r *Runtime) configureNetNS(ctr *Container, ctrNS ns.NetNS) ([]*cnitypes.Re
networkStatus = append(networkStatus, resultCurrent)
}
- // Add firewall rules to ensure the container has network access.
- // Will not be necessary once CNI firewall plugin merges upstream.
- // https://github.com/containernetworking/plugins/pull/75
- for _, netStatus := range networkStatus {
- firewallConf := &firewall.FirewallNetConf{
- PrevResult: netStatus,
- }
- if err := r.firewallBackend.Add(firewallConf); err != nil {
- return nil, errors.Wrapf(err, "error adding firewall rules for container %s", ctr.ID())
- }
- }
-
return networkStatus, nil
}
@@ -390,26 +377,12 @@ func (r *Runtime) closeNetNS(ctr *Container) error {
}
// Tear down a network namespace, undoing all state associated with it.
-// The CNI firewall rules will be removed, the namespace will be unmounted,
-// and the file descriptor associated with it closed.
func (r *Runtime) teardownNetNS(ctr *Container) error {
if ctr.state.NetNS == nil {
// The container has no network namespace, we're set
return nil
}
- // Remove firewall rules we added on configuring the container.
- // Will not be necessary once CNI firewall plugin merges upstream.
- // https://github.com/containernetworking/plugins/pull/75
- for _, netStatus := range ctr.state.NetworkStatus {
- firewallConf := &firewall.FirewallNetConf{
- PrevResult: netStatus,
- }
- if err := r.firewallBackend.Del(firewallConf); err != nil {
- return errors.Wrapf(err, "error removing firewall rules for container %s", ctr.ID())
- }
- }
-
logrus.Debugf("Tearing down network namespace at %s for container %s", ctr.state.NetNS.Path(), ctr.ID())
var requestedIP net.IP
diff --git a/libpod/pod_api.go b/libpod/pod_api.go
index c7b0353bd..e2448e92a 100644
--- a/libpod/pod_api.go
+++ b/libpod/pod_api.go
@@ -37,7 +37,7 @@ func (p *Pod) Start(ctx context.Context) (map[string]error, error) {
}
// Build a dependency graph of containers in the pod
- graph, err := buildContainerGraph(allCtrs)
+ graph, err := BuildContainerGraph(allCtrs)
if err != nil {
return nil, errors.Wrapf(err, "error generating dependency graph for pod %s", p.ID())
}
@@ -289,7 +289,7 @@ func (p *Pod) Restart(ctx context.Context) (map[string]error, error) {
}
// Build a dependency graph of containers in the pod
- graph, err := buildContainerGraph(allCtrs)
+ graph, err := BuildContainerGraph(allCtrs)
if err != nil {
return nil, errors.Wrapf(err, "error generating dependency graph for pod %s", p.ID())
}
diff --git a/libpod/runtime.go b/libpod/runtime.go
index cbbf667db..28774773e 100644
--- a/libpod/runtime.go
+++ b/libpod/runtime.go
@@ -23,7 +23,6 @@ import (
"github.com/containers/libpod/libpod/events"
"github.com/containers/libpod/libpod/image"
"github.com/containers/libpod/libpod/lock"
- "github.com/containers/libpod/pkg/firewall"
sysreg "github.com/containers/libpod/pkg/registries"
"github.com/containers/libpod/pkg/rootless"
"github.com/containers/libpod/pkg/util"
@@ -77,10 +76,6 @@ var (
// place of the configuration file pointed to by ConfigPath.
OverrideConfigPath = etcDir + "/containers/libpod.conf"
- // DefaultInfraImage to use for infra container
-
- // DefaultInfraCommand to be run in an infra container
-
// DefaultSHMLockPath is the default path for SHM locks
DefaultSHMLockPath = "/libpod_lock"
// DefaultRootlessSHMLockPath is the default path for rootless SHM locks
@@ -108,7 +103,6 @@ type Runtime struct {
netPlugin ocicni.CNIPlugin
conmonPath string
imageRuntime *image.Runtime
- firewallBackend firewall.FirewallBackend
lockManager lock.Manager
configuredFrom *runtimeConfiguredFrom
@@ -1110,17 +1104,6 @@ func makeRuntime(ctx context.Context, runtime *Runtime) (err error) {
runtime.netPlugin = netPlugin
}
- // Set up a firewall backend
- backendType := ""
- if rootless.IsRootless() {
- backendType = "none"
- }
- fwBackend, err := firewall.GetBackend(backendType)
- if err != nil {
- return err
- }
- runtime.firewallBackend = fwBackend
-
// We now need to see if the system has restarted
// We check for the presence of a file in our tmp directory to verify this
// This check must be locked to prevent races
diff --git a/libpod/runtime_ctr.go b/libpod/runtime_ctr.go
index 92b2faefb..acd317d20 100644
--- a/libpod/runtime_ctr.go
+++ b/libpod/runtime_ctr.go
@@ -253,10 +253,13 @@ func (r *Runtime) setupContainer(ctx context.Context, ctr *Container) (c *Contai
// Go through named volumes and add them.
// If they don't exist they will be created using basic options.
+ // Maintain an array of them - we need to lock them later.
+ ctrNamedVolumes := make([]*Volume, 0, len(ctr.config.NamedVolumes))
for _, vol := range ctr.config.NamedVolumes {
// Check if it exists already
- _, err := r.state.Volume(vol.Name)
+ dbVol, err := r.state.Volume(vol.Name)
if err == nil {
+ ctrNamedVolumes = append(ctrNamedVolumes, dbVol)
// The volume exists, we're good
continue
} else if errors.Cause(err) != config2.ErrNoSuchVolume {
@@ -275,6 +278,8 @@ func (r *Runtime) setupContainer(ctx context.Context, ctr *Container) (c *Contai
if err := ctr.copyWithTarFromImage(vol.Dest, newVol.MountPoint()); err != nil && !os.IsNotExist(err) {
return nil, errors.Wrapf(err, "Failed to copy content into new volume mount %q", vol.Name)
}
+
+ ctrNamedVolumes = append(ctrNamedVolumes, newVol)
}
if ctr.config.LogPath == "" && ctr.config.LogDriver != JournaldLogging {
@@ -291,6 +296,14 @@ func (r *Runtime) setupContainer(ctx context.Context, ctr *Container) (c *Contai
ctr.config.Mounts = append(ctr.config.Mounts, ctr.config.ShmDir)
}
+ // Lock all named volumes we are adding ourself to, to ensure we can't
+ // use a volume being removed.
+ for _, namedVol := range ctrNamedVolumes {
+ toLock := namedVol
+ toLock.lock.Lock()
+ defer toLock.lock.Unlock()
+ }
+
// Add the container to the state
// TODO: May be worth looking into recovering from name/ID collisions here
if ctr.config.Pod != "" {
diff --git a/libpod/runtime_renumber.go b/libpod/runtime_renumber.go
index 735ffba34..9de2556b2 100644
--- a/libpod/runtime_renumber.go
+++ b/libpod/runtime_renumber.go
@@ -53,6 +53,23 @@ func (r *Runtime) renumberLocks() error {
return err
}
}
+ allVols, err := r.state.AllVolumes()
+ if err != nil {
+ return err
+ }
+ for _, vol := range allVols {
+ lock, err := r.lockManager.AllocateLock()
+ if err != nil {
+ return errors.Wrapf(err, "error allocating lock for volume %s", vol.Name())
+ }
+
+ vol.config.LockID = lock.ID()
+
+ // Write the new lock ID
+ if err := r.state.RewriteVolumeConfig(vol, vol.config); err != nil {
+ return err
+ }
+ }
r.newSystemEvent(events.Renumber)
diff --git a/libpod/runtime_volume.go b/libpod/runtime_volume.go
index d05db936b..512e778a1 100644
--- a/libpod/runtime_volume.go
+++ b/libpod/runtime_volume.go
@@ -36,6 +36,10 @@ func (r *Runtime) RemoveVolume(ctx context.Context, v *Volume, force bool) error
return nil
}
}
+
+ v.lock.Lock()
+ defer v.lock.Unlock()
+
return r.removeVolume(ctx, v, force)
}
diff --git a/libpod/runtime_volume_linux.go b/libpod/runtime_volume_linux.go
index 84703787d..70296248c 100644
--- a/libpod/runtime_volume_linux.go
+++ b/libpod/runtime_volume_linux.go
@@ -28,7 +28,7 @@ func (r *Runtime) NewVolume(ctx context.Context, options ...VolumeCreateOption)
}
// newVolume creates a new empty volume
-func (r *Runtime) newVolume(ctx context.Context, options ...VolumeCreateOption) (*Volume, error) {
+func (r *Runtime) newVolume(ctx context.Context, options ...VolumeCreateOption) (_ *Volume, Err error) {
volume, err := newVolume(r)
if err != nil {
return nil, errors.Wrapf(err, "error creating volume")
@@ -68,6 +68,21 @@ func (r *Runtime) newVolume(ctx context.Context, options ...VolumeCreateOption)
}
volume.config.MountPoint = fullVolPath
+ lock, err := r.lockManager.AllocateLock()
+ if err != nil {
+ return nil, errors.Wrapf(err, "error allocating lock for new volume")
+ }
+ volume.lock = lock
+ volume.config.LockID = volume.lock.ID()
+
+ defer func() {
+ if Err != nil {
+ if err := volume.lock.Free(); err != nil {
+ logrus.Errorf("Error freeing volume lock after failed creation: %v", err)
+ }
+ }
+ }()
+
volume.valid = true
// Add the volume to state
@@ -110,6 +125,8 @@ func (r *Runtime) removeVolume(ctx context.Context, v *Volume, force bool) error
return errors.Wrapf(err, "error removing container %s that depends on volume %s", dep, v.Name())
}
+ logrus.Debugf("Removing container %s (depends on volume %q)", ctr.ID(), v.Name())
+
// TODO: do we want to set force here when removing
// containers?
// I'm inclined to say no, in case someone accidentally
@@ -128,12 +145,24 @@ func (r *Runtime) removeVolume(ctx context.Context, v *Volume, force bool) error
return errors.Wrapf(err, "error removing volume %s", v.Name())
}
- // Delete the mountpoint path of the volume, that is delete the volume from /var/lib/containers/storage/volumes
+ var removalErr error
+
+ // Free the volume's lock
+ if err := v.lock.Free(); err != nil {
+ removalErr = errors.Wrapf(err, "error freeing lock for volume %s", v.Name())
+ }
+
+ // Delete the mountpoint path of the volume, that is delete the volume
+ // from /var/lib/containers/storage/volumes
if err := v.teardownStorage(); err != nil {
- return errors.Wrapf(err, "error cleaning up volume storage for %q", v.Name())
+ if removalErr == nil {
+ removalErr = errors.Wrapf(err, "error cleaning up volume storage for %q", v.Name())
+ } else {
+ logrus.Errorf("error cleaning up volume storage for volume %q: %v", v.Name(), err)
+ }
}
defer v.newVolumeEvent(events.Remove)
logrus.Debugf("Removed volume %s", v.Name())
- return nil
+ return removalErr
}
diff --git a/libpod/state.go b/libpod/state.go
index d0ad1a1f8..5d704e69a 100644
--- a/libpod/state.go
+++ b/libpod/state.go
@@ -115,12 +115,20 @@ type State interface {
// answer is this: use this only very sparingly, and only if you really
// know what you're doing.
RewriteContainerConfig(ctr *Container, newCfg *ContainerConfig) error
- // PLEASE READ THE ABOVE DESCRIPTION BEFORE USING.
+ // PLEASE READ THE DESCRIPTION FOR RewriteContainerConfig BEFORE USING.
// This function is identical to RewriteContainerConfig, save for the
// fact that it is used with pods instead.
// It is subject to the same conditions as RewriteContainerConfig.
// Please do not use this unless you know what you're doing.
RewritePodConfig(pod *Pod, newCfg *PodConfig) error
+ // PLEASE READ THE DESCRIPTION FOR RewriteContainerConfig BEFORE USING.
+ // This function is identical to RewriteContainerConfig, save for the
+ // fact that it is used with volumes instead.
+ // It is subject to the same conditions as RewriteContainerConfig.
+ // The exception is that volumes do not have IDs, so only volume name
+ // cannot be altered.
+ // Please do not use this unless you know what you're doing.
+ RewriteVolumeConfig(volume *Volume, newCfg *VolumeConfig) error
// Accepts full ID of pod.
// If the pod given is not in the set namespace, an error will be
diff --git a/libpod/volume.go b/libpod/volume.go
index 74126b49b..abfa7b3f4 100644
--- a/libpod/volume.go
+++ b/libpod/volume.go
@@ -2,6 +2,8 @@ package libpod
import (
"time"
+
+ "github.com/containers/libpod/libpod/lock"
)
// Volume is the type used to create named volumes
@@ -11,21 +13,35 @@ type Volume struct {
valid bool
runtime *Runtime
+ lock lock.Locker
}
// VolumeConfig holds the volume's config information
type VolumeConfig struct {
- // Name of the volume
+ // Name of the volume.
Name string `json:"name"`
-
- Labels map[string]string `json:"labels"`
- Driver string `json:"driver"`
- MountPoint string `json:"mountPoint"`
- CreatedTime time.Time `json:"createdAt,omitempty"`
- Options map[string]string `json:"options"`
- IsCtrSpecific bool `json:"ctrSpecific"`
- UID int `json:"uid"`
- GID int `json:"gid"`
+ // ID of the volume's lock.
+ LockID uint32 `json:"lockID"`
+ // Labels for the volume.
+ Labels map[string]string `json:"labels"`
+ // The volume driver. Empty string or local does not activate a volume
+ // driver, all other volumes will.
+ Driver string `json:"driver"`
+ // The location the volume is mounted at.
+ MountPoint string `json:"mountPoint"`
+ // Time the volume was created.
+ CreatedTime time.Time `json:"createdAt,omitempty"`
+ // Options to pass to the volume driver. For the local driver, this is
+ // a list of mount options. For other drivers, they are passed to the
+ // volume driver handling the volume.
+ Options map[string]string `json:"options"`
+ // Whether this volume was created for a specific container and will be
+ // removed with it.
+ IsCtrSpecific bool `json:"ctrSpecific"`
+ // UID the volume will be created as.
+ UID int `json:"uid"`
+ // GID the volume will be created as.
+ GID int `json:"gid"`
}
// Name retrieves the volume's name