summaryrefslogtreecommitdiffhomepage
path: root/pkg/p9
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/p9')
-rw-r--r--pkg/p9/BUILD2
-rw-r--r--pkg/p9/client.go4
-rw-r--r--pkg/p9/client_file.go4
-rw-r--r--pkg/p9/client_test.go2
-rw-r--r--pkg/p9/file.go2
-rw-r--r--pkg/p9/handlers.go12
-rw-r--r--pkg/p9/local_server/BUILD14
-rw-r--r--pkg/p9/local_server/local_server.go353
-rw-r--r--pkg/p9/messages.go2
-rw-r--r--pkg/p9/p9.go2
-rw-r--r--pkg/p9/p9test/BUILD4
-rw-r--r--pkg/p9/p9test/client_test.go66
-rw-r--r--pkg/p9/p9test/p9test.go4
-rw-r--r--pkg/p9/path_tree.go208
-rw-r--r--pkg/p9/server.go63
-rw-r--r--pkg/p9/transport.go6
-rw-r--r--pkg/p9/transport_test.go49
17 files changed, 323 insertions, 474 deletions
diff --git a/pkg/p9/BUILD b/pkg/p9/BUILD
index 36b2ec5f6..c6737bf97 100644
--- a/pkg/p9/BUILD
+++ b/pkg/p9/BUILD
@@ -21,7 +21,7 @@ go_library(
"transport.go",
"version.go",
],
- importpath = "gvisor.googlesource.com/gvisor/pkg/p9",
+ importpath = "gvisor.dev/gvisor/pkg/p9",
deps = [
"//pkg/fd",
"//pkg/log",
diff --git a/pkg/p9/client.go b/pkg/p9/client.go
index 56587e2cf..7dc20aeef 100644
--- a/pkg/p9/client.go
+++ b/pkg/p9/client.go
@@ -20,8 +20,8 @@ import (
"sync"
"syscall"
- "gvisor.googlesource.com/gvisor/pkg/log"
- "gvisor.googlesource.com/gvisor/pkg/unet"
+ "gvisor.dev/gvisor/pkg/log"
+ "gvisor.dev/gvisor/pkg/unet"
)
// ErrOutOfTags indicates no tags are available.
diff --git a/pkg/p9/client_file.go b/pkg/p9/client_file.go
index 258080f67..a6cc0617e 100644
--- a/pkg/p9/client_file.go
+++ b/pkg/p9/client_file.go
@@ -21,8 +21,8 @@ import (
"sync/atomic"
"syscall"
- "gvisor.googlesource.com/gvisor/pkg/fd"
- "gvisor.googlesource.com/gvisor/pkg/log"
+ "gvisor.dev/gvisor/pkg/fd"
+ "gvisor.dev/gvisor/pkg/log"
)
// Attach attaches to a server.
diff --git a/pkg/p9/client_test.go b/pkg/p9/client_test.go
index fc49729d8..87b2dd61e 100644
--- a/pkg/p9/client_test.go
+++ b/pkg/p9/client_test.go
@@ -18,7 +18,7 @@ import (
"syscall"
"testing"
- "gvisor.googlesource.com/gvisor/pkg/unet"
+ "gvisor.dev/gvisor/pkg/unet"
)
// TestVersion tests the version negotiation.
diff --git a/pkg/p9/file.go b/pkg/p9/file.go
index a456e8b3d..907445e15 100644
--- a/pkg/p9/file.go
+++ b/pkg/p9/file.go
@@ -17,7 +17,7 @@ package p9
import (
"syscall"
- "gvisor.googlesource.com/gvisor/pkg/fd"
+ "gvisor.dev/gvisor/pkg/fd"
)
// Attacher is provided by the server.
diff --git a/pkg/p9/handlers.go b/pkg/p9/handlers.go
index f32368763..999b4f684 100644
--- a/pkg/p9/handlers.go
+++ b/pkg/p9/handlers.go
@@ -23,8 +23,8 @@ import (
"sync/atomic"
"syscall"
- "gvisor.googlesource.com/gvisor/pkg/fd"
- "gvisor.googlesource.com/gvisor/pkg/log"
+ "gvisor.dev/gvisor/pkg/fd"
+ "gvisor.dev/gvisor/pkg/log"
)
// ExtractErrno extracts a syscall.Errno from a error, best effort.
@@ -224,7 +224,7 @@ func (t *Tattach) handle(cs *connState) message {
file: sf,
refs: 1,
mode: attr.Mode.FileType(),
- pathNode: &cs.server.pathTree,
+ pathNode: cs.server.pathTree,
}
defer root.DecRef()
@@ -545,15 +545,15 @@ func (t *Tunlinkat) handle(cs *connState) message {
// Before we do the unlink itself, we need to ensure that there
// are no operations in flight on associated path node. The
// child's path node lock must be held to ensure that the
- // unlink at marking the child deleted below is atomic with
+ // unlinkat marking the child deleted below is atomic with
// respect to any other read or write operations.
//
// This is one case where we have a lock ordering issue, but
// since we always acquire deeper in the hierarchy, we know
// that we are free of lock cycles.
childPathNode := ref.pathNode.pathNodeFor(t.Name)
- childPathNode.mu.Lock()
- defer childPathNode.mu.Unlock()
+ childPathNode.opMu.Lock()
+ defer childPathNode.opMu.Unlock()
// Do the unlink.
err = ref.file.UnlinkAt(t.Name, t.Flags)
diff --git a/pkg/p9/local_server/BUILD b/pkg/p9/local_server/BUILD
deleted file mode 100644
index aa6db186c..000000000
--- a/pkg/p9/local_server/BUILD
+++ /dev/null
@@ -1,14 +0,0 @@
-load("@io_bazel_rules_go//go:def.bzl", "go_binary")
-
-package(licenses = ["notice"])
-
-go_binary(
- name = "local_server",
- srcs = ["local_server.go"],
- deps = [
- "//pkg/fd",
- "//pkg/log",
- "//pkg/p9",
- "//pkg/unet",
- ],
-)
diff --git a/pkg/p9/local_server/local_server.go b/pkg/p9/local_server/local_server.go
deleted file mode 100644
index 9546b3de5..000000000
--- a/pkg/p9/local_server/local_server.go
+++ /dev/null
@@ -1,353 +0,0 @@
-// Copyright 2018 The gVisor Authors.
-//
-// Licensed under the Apache License, Version 2.0 (the "License");
-// you may not use this file except in compliance with the License.
-// You may obtain a copy of the License at
-//
-// http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-// Binary local_server provides a local 9P2000.L server for the p9 package.
-//
-// To use, first start the server:
-// local_server /tmp/my_bind_addr
-//
-// Then, connect using the Linux 9P filesystem:
-// mount -t 9p -o trans=unix /tmp/my_bind_addr /mnt
-//
-// This package also serves as an examplar.
-package main
-
-import (
- "os"
- "path"
- "syscall"
-
- "gvisor.googlesource.com/gvisor/pkg/fd"
- "gvisor.googlesource.com/gvisor/pkg/log"
- "gvisor.googlesource.com/gvisor/pkg/p9"
- "gvisor.googlesource.com/gvisor/pkg/unet"
-)
-
-// local wraps a local file.
-type local struct {
- p9.DefaultWalkGetAttr
-
- path string
- file *os.File
-}
-
-// info constructs a QID for this file.
-func (l *local) info() (p9.QID, os.FileInfo, error) {
- var (
- qid p9.QID
- fi os.FileInfo
- err error
- )
-
- // Stat the file.
- if l.file != nil {
- fi, err = l.file.Stat()
- } else {
- fi, err = os.Lstat(l.path)
- }
- if err != nil {
- log.Warningf("error stating %#v: %v", l, err)
- return qid, nil, err
- }
-
- // Construct the QID type.
- qid.Type = p9.ModeFromOS(fi.Mode()).QIDType()
-
- // Save the path from the Ino.
- qid.Path = fi.Sys().(*syscall.Stat_t).Ino
- return qid, fi, nil
-}
-
-// Attach implements p9.Attacher.Attach.
-func (l *local) Attach() (p9.File, error) {
- return &local{path: "/"}, nil
-}
-
-// Walk implements p9.File.Walk.
-func (l *local) Walk(names []string) ([]p9.QID, p9.File, error) {
- var qids []p9.QID
- last := &local{path: l.path}
- for _, name := range names {
- c := &local{path: path.Join(last.path, name)}
- qid, _, err := c.info()
- if err != nil {
- return nil, nil, err
- }
- qids = append(qids, qid)
- last = c
- }
- return qids, last, nil
-}
-
-// StatFS implements p9.File.StatFS.
-//
-// Not implemented.
-func (l *local) StatFS() (p9.FSStat, error) {
- return p9.FSStat{}, syscall.ENOSYS
-}
-
-// FSync implements p9.File.FSync.
-func (l *local) FSync() error {
- return l.file.Sync()
-}
-
-// GetAttr implements p9.File.GetAttr.
-//
-// Not fully implemented.
-func (l *local) GetAttr(req p9.AttrMask) (p9.QID, p9.AttrMask, p9.Attr, error) {
- qid, fi, err := l.info()
- if err != nil {
- return qid, p9.AttrMask{}, p9.Attr{}, err
- }
-
- stat := fi.Sys().(*syscall.Stat_t)
- attr := p9.Attr{
- Mode: p9.FileMode(stat.Mode),
- UID: p9.UID(stat.Uid),
- GID: p9.GID(stat.Gid),
- NLink: stat.Nlink,
- RDev: stat.Rdev,
- Size: uint64(stat.Size),
- BlockSize: uint64(stat.Blksize),
- Blocks: uint64(stat.Blocks),
- ATimeSeconds: uint64(stat.Atim.Sec),
- ATimeNanoSeconds: uint64(stat.Atim.Nsec),
- MTimeSeconds: uint64(stat.Mtim.Sec),
- MTimeNanoSeconds: uint64(stat.Mtim.Nsec),
- CTimeSeconds: uint64(stat.Ctim.Sec),
- CTimeNanoSeconds: uint64(stat.Ctim.Nsec),
- }
- valid := p9.AttrMask{
- Mode: true,
- UID: true,
- GID: true,
- NLink: true,
- RDev: true,
- Size: true,
- Blocks: true,
- ATime: true,
- MTime: true,
- CTime: true,
- }
-
- return qid, valid, attr, nil
-}
-
-// SetAttr implements p9.File.SetAttr.
-//
-// Not implemented.
-func (l *local) SetAttr(valid p9.SetAttrMask, attr p9.SetAttr) error {
- return syscall.ENOSYS
-}
-
-// Remove implements p9.File.Remove.
-//
-// Not implemented.
-func (l *local) Remove() error {
- return syscall.ENOSYS
-}
-
-// Rename implements p9.File.Rename.
-//
-// Not implemented.
-func (l *local) Rename(directory p9.File, name string) error {
- return syscall.ENOSYS
-}
-
-// Close implements p9.File.Close.
-func (l *local) Close() error {
- if l.file != nil {
- return l.file.Close()
- }
- return nil
-}
-
-// Open implements p9.File.Open.
-func (l *local) Open(mode p9.OpenFlags) (*fd.FD, p9.QID, uint32, error) {
- qid, _, err := l.info()
- if err != nil {
- return nil, qid, 0, err
- }
-
- // Do the actual open.
- f, err := os.OpenFile(l.path, int(mode), 0)
- if err != nil {
- return nil, qid, 0, err
- }
- l.file = f
-
- // Note: we don't send the local file for this server.
- return nil, qid, 4096, nil
-}
-
-// Read implements p9.File.Read.
-func (l *local) ReadAt(p []byte, offset uint64) (int, error) {
- return l.file.ReadAt(p, int64(offset))
-}
-
-// Write implements p9.File.Write.
-func (l *local) WriteAt(p []byte, offset uint64) (int, error) {
- return l.file.WriteAt(p, int64(offset))
-}
-
-// Create implements p9.File.Create.
-func (l *local) Create(name string, mode p9.OpenFlags, permissions p9.FileMode, _ p9.UID, _ p9.GID) (*fd.FD, p9.File, p9.QID, uint32, error) {
- f, err := os.OpenFile(l.path, int(mode)|syscall.O_CREAT|syscall.O_EXCL, os.FileMode(permissions))
- if err != nil {
- return nil, nil, p9.QID{}, 0, err
- }
-
- l2 := &local{path: path.Join(l.path, name), file: f}
- qid, _, err := l2.info()
- if err != nil {
- l2.Close()
- return nil, nil, p9.QID{}, 0, err
- }
-
- return nil, l2, qid, 4096, nil
-}
-
-// Mkdir implements p9.File.Mkdir.
-//
-// Not properly implemented.
-func (l *local) Mkdir(name string, permissions p9.FileMode, _ p9.UID, _ p9.GID) (p9.QID, error) {
- if err := os.Mkdir(path.Join(l.path, name), os.FileMode(permissions)); err != nil {
- return p9.QID{}, err
- }
-
- // Blank QID.
- return p9.QID{}, nil
-}
-
-// Symlink implements p9.File.Symlink.
-//
-// Not properly implemented.
-func (l *local) Symlink(oldname string, newname string, _ p9.UID, _ p9.GID) (p9.QID, error) {
- if err := os.Symlink(oldname, path.Join(l.path, newname)); err != nil {
- return p9.QID{}, err
- }
-
- // Blank QID.
- return p9.QID{}, nil
-}
-
-// Link implements p9.File.Link.
-//
-// Not properly implemented.
-func (l *local) Link(target p9.File, newname string) error {
- return os.Link(target.(*local).path, path.Join(l.path, newname))
-}
-
-// Mknod implements p9.File.Mknod.
-//
-// Not implemented.
-func (l *local) Mknod(name string, mode p9.FileMode, major uint32, minor uint32, _ p9.UID, _ p9.GID) (p9.QID, error) {
- return p9.QID{}, syscall.ENOSYS
-}
-
-// RenameAt implements p9.File.RenameAt.
-//
-// Not implemented.
-func (l *local) RenameAt(oldname string, newdir p9.File, newname string) error {
- return syscall.ENOSYS
-}
-
-// UnlinkAt implements p9.File.UnlinkAt.
-//
-// Not implemented.
-func (l *local) UnlinkAt(name string, flags uint32) error {
- return syscall.ENOSYS
-}
-
-// Readdir implements p9.File.Readdir.
-func (l *local) Readdir(offset uint64, count uint32) ([]p9.Dirent, error) {
- // We only do *all* dirents in single shot.
- const maxDirentBuffer = 1024 * 1024
- buf := make([]byte, maxDirentBuffer)
- n, err := syscall.ReadDirent(int(l.file.Fd()), buf)
- if err != nil {
- // Return zero entries.
- return nil, nil
- }
-
- // Parse the entries; note that we read up to offset+count here.
- _, newCount, newNames := syscall.ParseDirent(buf[:n], int(offset)+int(count), nil)
- var dirents []p9.Dirent
- for i := int(offset); i >= 0 && i < newCount; i++ {
- entry := local{path: path.Join(l.path, newNames[i])}
- qid, _, err := entry.info()
- if err != nil {
- continue
- }
- dirents = append(dirents, p9.Dirent{
- QID: qid,
- Type: qid.Type,
- Name: newNames[i],
- Offset: uint64(i + 1),
- })
- }
-
- return dirents, nil
-}
-
-// Readlink implements p9.File.Readlink.
-//
-// Not properly implemented.
-func (l *local) Readlink() (string, error) {
- return os.Readlink(l.path)
-}
-
-// Flush implements p9.File.Flush.
-func (l *local) Flush() error {
- return nil
-}
-
-// Connect implements p9.File.Connect.
-func (l *local) Connect(p9.ConnectFlags) (*fd.FD, error) {
- return nil, syscall.ECONNREFUSED
-}
-
-// Renamed implements p9.File.Renamed.
-func (l *local) Renamed(parent p9.File, newName string) {
- l.path = path.Join(parent.(*local).path, newName)
-}
-
-// Allocate implements p9.File.Allocate.
-func (l *local) Allocate(mode p9.AllocateMode, offset, length uint64) error {
- return syscall.Fallocate(int(l.file.Fd()), mode.ToLinux(), int64(offset), int64(length))
-}
-
-func main() {
- log.SetLevel(log.Debug)
-
- if len(os.Args) != 2 {
- log.Warningf("usage: %s <bind-addr>", os.Args[0])
- os.Exit(1)
- }
-
- // Bind and listen on the socket.
- serverSocket, err := unet.BindAndListen(os.Args[1], false)
- if err != nil {
- log.Warningf("err binding: %v", err)
- os.Exit(1)
- }
-
- // Run the server.
- s := p9.NewServer(&local{})
- s.Serve(serverSocket)
-}
-
-var (
- _ p9.File = &local{}
-)
diff --git a/pkg/p9/messages.go b/pkg/p9/messages.go
index 75d6bc832..fd9eb1c5d 100644
--- a/pkg/p9/messages.go
+++ b/pkg/p9/messages.go
@@ -18,7 +18,7 @@ import (
"fmt"
"math"
- "gvisor.googlesource.com/gvisor/pkg/fd"
+ "gvisor.dev/gvisor/pkg/fd"
)
// ErrInvalidMsgType is returned when an unsupported message type is found.
diff --git a/pkg/p9/p9.go b/pkg/p9/p9.go
index 4039862e6..e12831dbd 100644
--- a/pkg/p9/p9.go
+++ b/pkg/p9/p9.go
@@ -92,7 +92,7 @@ func (o OpenFlags) String() string {
}
}
-// Tag is a messsage tag.
+// Tag is a message tag.
type Tag uint16
// FID is a file identifier.
diff --git a/pkg/p9/p9test/BUILD b/pkg/p9/p9test/BUILD
index cf22edde8..6e939a49a 100644
--- a/pkg/p9/p9test/BUILD
+++ b/pkg/p9/p9test/BUILD
@@ -8,7 +8,7 @@ alias(
actual = "@com_github_golang_mock//mockgen:mockgen",
)
-MOCK_SRC_PACKAGE = "gvisor.googlesource.com/gvisor/pkg/p9"
+MOCK_SRC_PACKAGE = "gvisor.dev/gvisor/pkg/p9"
# mockgen_reflect is a source file that contains mock generation code that
# imports the p9 package and generates a specification via reflection. The
@@ -64,7 +64,7 @@ go_library(
"mocks.go",
"p9test.go",
],
- importpath = "gvisor.googlesource.com/gvisor/pkg/p9/p9test",
+ importpath = "gvisor.dev/gvisor/pkg/p9/p9test",
visibility = ["//:sandbox"],
deps = [
"//pkg/fd",
diff --git a/pkg/p9/p9test/client_test.go b/pkg/p9/p9test/client_test.go
index e00dd03ab..fe649c2e8 100644
--- a/pkg/p9/p9test/client_test.go
+++ b/pkg/p9/p9test/client_test.go
@@ -28,8 +28,8 @@ import (
"time"
"github.com/golang/mock/gomock"
- "gvisor.googlesource.com/gvisor/pkg/fd"
- "gvisor.googlesource.com/gvisor/pkg/p9"
+ "gvisor.dev/gvisor/pkg/fd"
+ "gvisor.dev/gvisor/pkg/p9"
)
func TestPanic(t *testing.T) {
@@ -269,14 +269,14 @@ type fileGenerator func(*Harness, string, p9.File) (*Mock, *Mock, p9.File)
func walkHelper(h *Harness, name string, dir p9.File) (parentBackend *Mock, walkedBackend *Mock, walked p9.File) {
_, parent, err := dir.Walk(nil)
if err != nil {
- h.t.Fatalf("got walk err %v, want nil", err)
+ h.t.Fatalf("Walk(nil) got err %v, want nil", err)
}
defer parent.Close()
parentBackend = h.Pop(parent)
_, walked, err = parent.Walk([]string{name})
if err != nil {
- h.t.Fatalf("got walk err %v, want nil", err)
+ h.t.Fatalf("Walk(%s) got err %v, want nil", name, err)
}
walkedBackend = h.Pop(walked)
@@ -389,7 +389,7 @@ func checkDeleted(h *Harness, file p9.File) {
_, newFile, err := file.Walk(nil)
if err == syscall.EBUSY {
// We can't walk from here because this reference is open
- // aleady. Okay, we will also have unopened cases through
+ // already. Okay, we will also have unopened cases through
// TestUnlink, just skip the remove operation for now.
return
} else if err != nil {
@@ -854,6 +854,62 @@ func TestRenameAtInvalid(t *testing.T) {
}
}
+// TestRenameSecondOrder tests that indirect rename targets continue to receive
+// Renamed calls after a rename of its renamed parent. i.e.,
+//
+// 1. Create /one/file
+// 2. Create /directory
+// 3. Rename /one -> /directory/one
+// 4. Rename /directory -> /three/foo
+// 5. file from (1) should still receive Renamed.
+//
+// This is a regression test for b/135219260.
+func TestRenameSecondOrder(t *testing.T) {
+ h, c := NewHarness(t)
+ defer h.Finish()
+
+ rootBackend, root := newRoot(h, c)
+ defer root.Close()
+
+ // Walk to /one.
+ _, oneBackend, oneFile := walkHelper(h, "one", root)
+ defer oneFile.Close()
+
+ // Walk to and generate /one/file.
+ //
+ // walkHelper re-walks to oneFile, so we need the second backend,
+ // which will also receive Renamed calls.
+ oneSecondBackend, fileBackend, fileFile := walkHelper(h, "file", oneFile)
+ defer fileFile.Close()
+
+ // Walk to and generate /directory.
+ _, directoryBackend, directoryFile := walkHelper(h, "directory", root)
+ defer directoryFile.Close()
+
+ // Rename /one to /directory/one.
+ rootBackend.EXPECT().RenameAt("one", directoryBackend, "one").Return(nil)
+ expectRenamed(oneBackend, []string{}, directoryBackend, "one")
+ expectRenamed(oneSecondBackend, []string{}, directoryBackend, "one")
+ expectRenamed(fileBackend, []string{}, oneBackend, "file")
+ if err := renameAt(h, root, directoryFile, "one", "one", false); err != nil {
+ h.t.Fatalf("got rename err %v, want nil", err)
+ }
+
+ // Walk to /three.
+ _, threeBackend, threeFile := walkHelper(h, "three", root)
+ defer threeFile.Close()
+
+ // Rename /directory to /three/foo.
+ rootBackend.EXPECT().RenameAt("directory", threeBackend, "foo").Return(nil)
+ expectRenamed(directoryBackend, []string{}, threeBackend, "foo")
+ expectRenamed(oneBackend, []string{}, directoryBackend, "one")
+ expectRenamed(oneSecondBackend, []string{}, directoryBackend, "one")
+ expectRenamed(fileBackend, []string{}, oneBackend, "file")
+ if err := renameAt(h, root, threeFile, "directory", "foo", false); err != nil {
+ h.t.Fatalf("got rename err %v, want nil", err)
+ }
+}
+
func TestReadlink(t *testing.T) {
for name := range newTypeMap(nil) {
t.Run(name, func(t *testing.T) {
diff --git a/pkg/p9/p9test/p9test.go b/pkg/p9/p9test/p9test.go
index 1c8eff200..95846e5f7 100644
--- a/pkg/p9/p9test/p9test.go
+++ b/pkg/p9/p9test/p9test.go
@@ -23,8 +23,8 @@ import (
"testing"
"github.com/golang/mock/gomock"
- "gvisor.googlesource.com/gvisor/pkg/p9"
- "gvisor.googlesource.com/gvisor/pkg/unet"
+ "gvisor.dev/gvisor/pkg/p9"
+ "gvisor.dev/gvisor/pkg/unet"
)
// Harness is an attacher mock.
diff --git a/pkg/p9/path_tree.go b/pkg/p9/path_tree.go
index f37ad4ab2..865459411 100644
--- a/pkg/p9/path_tree.go
+++ b/pkg/p9/path_tree.go
@@ -23,87 +23,199 @@ import (
//
// These are shared by all fidRefs that point to the same path.
//
-// These are not synchronized because we allow certain operations (file walk)
-// to proceed without having to acquire a write lock. The lock in this
-// structure exists to synchronize high-level, semantic operations, such as the
-// simultaneous creation and deletion of a file.
+// Lock ordering:
+// opMu
+// childMu
//
-// (+) below is the path component string.
+// Two different pathNodes may only be locked if Server.renameMu is held for
+// write, in which case they can be acquired in any order.
type pathNode struct {
- mu sync.RWMutex // See above.
- fidRefs sync.Map // => map[*fidRef]string(+)
- children sync.Map // => map[string(+)]*pathNode
- count int64
+ // opMu synchronizes high-level, sematic operations, such as the
+ // simultaneous creation and deletion of a file.
+ //
+ // opMu does not directly protect any fields in pathNode.
+ opMu sync.RWMutex
+
+ // childMu protects the fields below.
+ childMu sync.RWMutex
+
+ // childNodes maps child path component names to their pathNode.
+ childNodes map[string]*pathNode
+
+ // childRefs maps child path component names to all of the their
+ // references.
+ childRefs map[string]map[*fidRef]struct{}
+
+ // childRefNames maps child references back to their path component
+ // name.
+ childRefNames map[*fidRef]string
+}
+
+func newPathNode() *pathNode {
+ return &pathNode{
+ childNodes: make(map[string]*pathNode),
+ childRefs: make(map[string]map[*fidRef]struct{}),
+ childRefNames: make(map[*fidRef]string),
+ }
+}
+
+// forEachChildRef calls fn for each child reference.
+func (p *pathNode) forEachChildRef(fn func(ref *fidRef, name string)) {
+ p.childMu.RLock()
+ defer p.childMu.RUnlock()
+
+ for name, m := range p.childRefs {
+ for ref := range m {
+ fn(ref, name)
+ }
+ }
+}
+
+// forEachChildNode calls fn for each child pathNode.
+func (p *pathNode) forEachChildNode(fn func(pn *pathNode)) {
+ p.childMu.RLock()
+ defer p.childMu.RUnlock()
+
+ for _, pn := range p.childNodes {
+ fn(pn)
+ }
}
// pathNodeFor returns the path node for the given name, or a new one.
-//
-// Precondition: mu must be held in a readable fashion.
func (p *pathNode) pathNodeFor(name string) *pathNode {
- // Load the existing path node.
- if pn, ok := p.children.Load(name); ok {
- return pn.(*pathNode)
+ p.childMu.RLock()
+ // Fast path, node already exists.
+ if pn, ok := p.childNodes[name]; ok {
+ p.childMu.RUnlock()
+ return pn
+ }
+ p.childMu.RUnlock()
+
+ // Slow path, create a new pathNode for shared use.
+ p.childMu.Lock()
+
+ // Re-check after re-lock.
+ if pn, ok := p.childNodes[name]; ok {
+ p.childMu.Unlock()
+ return pn
}
- // Create a new pathNode for shared use.
- pn, _ := p.children.LoadOrStore(name, new(pathNode))
- return pn.(*pathNode)
+ pn := newPathNode()
+ p.childNodes[name] = pn
+ p.childMu.Unlock()
+ return pn
}
// nameFor returns the name for the given fidRef.
//
-// Precondition: mu must be held in a readable fashion.
+// Precondition: addChild is called for ref before nameFor.
func (p *pathNode) nameFor(ref *fidRef) string {
- if s, ok := p.fidRefs.Load(ref); ok {
- return s.(string)
+ p.childMu.RLock()
+ n, ok := p.childRefNames[ref]
+ p.childMu.RUnlock()
+
+ if !ok {
+ // This should not happen, don't proceed.
+ panic(fmt.Sprintf("expected name for %+v, none found", ref))
}
- // This should not happen, don't proceed.
- panic(fmt.Sprintf("expected name for %+v, none found", ref))
+ return n
}
-// addChild adds a child to the given pathNode.
-//
-// This applies only to an individual fidRef.
+// addChildLocked adds a child reference to p.
//
-// Precondition: mu must be held in a writable fashion.
-func (p *pathNode) addChild(ref *fidRef, name string) {
- if s, ok := p.fidRefs.Load(ref); ok {
+// Precondition: As addChild, plus childMu is locked for write.
+func (p *pathNode) addChildLocked(ref *fidRef, name string) {
+ if n, ok := p.childRefNames[ref]; ok {
// This should not happen, don't proceed.
- panic(fmt.Sprintf("unexpected fidRef %+v with path %q, wanted %q", ref, s, name))
+ panic(fmt.Sprintf("unexpected fidRef %+v with path %q, wanted %q", ref, n, name))
}
- p.fidRefs.Store(ref, name)
+ p.childRefNames[ref] = name
+
+ m, ok := p.childRefs[name]
+ if !ok {
+ m = make(map[*fidRef]struct{})
+ p.childRefs[name] = m
+ }
+
+ m[ref] = struct{}{}
}
-// removeChild removes the given child.
+// addChild adds a child reference to p.
//
-// This applies only to an individual fidRef.
+// Precondition: ref may only be added once at a time.
+func (p *pathNode) addChild(ref *fidRef, name string) {
+ p.childMu.Lock()
+ p.addChildLocked(ref, name)
+ p.childMu.Unlock()
+}
+
+// removeChild removes the given child.
//
-// Precondition: mu must be held in a writable fashion.
+// This applies only to an individual fidRef, which is not required to exist.
func (p *pathNode) removeChild(ref *fidRef) {
- p.fidRefs.Delete(ref)
+ p.childMu.Lock()
+
+ // This ref may not exist anymore. This can occur, e.g., in unlink,
+ // where a removeWithName removes the ref, and then a DecRef on the ref
+ // attempts to remove again.
+ if name, ok := p.childRefNames[ref]; ok {
+ m, ok := p.childRefs[name]
+ if !ok {
+ // This should not happen, don't proceed.
+ p.childMu.Unlock()
+ panic(fmt.Sprintf("name %s missing from childfidRefs", name))
+ }
+
+ delete(m, ref)
+ if len(m) == 0 {
+ delete(p.childRefs, name)
+ }
+ }
+
+ delete(p.childRefNames, ref)
+
+ p.childMu.Unlock()
}
-// removeWithName removes all references with the given name.
+// addPathNodeFor adds an existing pathNode as the node for name.
//
-// The original pathNode is returned by this function, and removed from this
-// pathNode. Any operations on the removed tree must use this value.
+// Preconditions: newName does not exist.
+func (p *pathNode) addPathNodeFor(name string, pn *pathNode) {
+ p.childMu.Lock()
+
+ if opn, ok := p.childNodes[name]; ok {
+ p.childMu.Unlock()
+ panic(fmt.Sprintf("unexpected pathNode %+v with path %q", opn, name))
+ }
+
+ p.childNodes[name] = pn
+ p.childMu.Unlock()
+}
+
+// removeWithName removes all references with the given name.
//
-// The provided function is executed after removal.
+// The provided function is executed after reference removal. The only method
+// it may (transitively) call on this pathNode is addChildLocked.
//
-// Precondition: mu must be held in a writable fashion.
+// If a child pathNode for name exists, it is removed from this pathNode and
+// returned by this function. Any operations on the removed tree must use this
+// value.
func (p *pathNode) removeWithName(name string, fn func(ref *fidRef)) *pathNode {
- p.fidRefs.Range(func(key, value interface{}) bool {
- if value.(string) == name {
- p.fidRefs.Delete(key)
- fn(key.(*fidRef))
+ p.childMu.Lock()
+ defer p.childMu.Unlock()
+
+ if m, ok := p.childRefs[name]; ok {
+ for ref := range m {
+ delete(m, ref)
+ delete(p.childRefNames, ref)
+ fn(ref)
}
- return true
- })
+ }
- // Return the original path node.
- origPathNode := p.pathNodeFor(name)
- p.children.Delete(name)
+ // Return the original path node, if it exists.
+ origPathNode := p.childNodes[name]
+ delete(p.childNodes, name)
return origPathNode
}
diff --git a/pkg/p9/server.go b/pkg/p9/server.go
index f377a6557..b294efbb0 100644
--- a/pkg/p9/server.go
+++ b/pkg/p9/server.go
@@ -21,8 +21,8 @@ import (
"sync/atomic"
"syscall"
- "gvisor.googlesource.com/gvisor/pkg/log"
- "gvisor.googlesource.com/gvisor/pkg/unet"
+ "gvisor.dev/gvisor/pkg/log"
+ "gvisor.dev/gvisor/pkg/unet"
)
// Server is a 9p2000.L server.
@@ -35,7 +35,7 @@ type Server struct {
// These may be across different connections, but rename operations
// must be serialized globally for safely. There is a single pathTree
// for the entire server, and not per connection.
- pathTree pathNode
+ pathTree *pathNode
// renameMu is a global lock protecting rename operations. With this
// lock, we can be certain that any given rename operation can safely
@@ -49,6 +49,7 @@ type Server struct {
func NewServer(attacher Attacher) *Server {
return &Server{
attacher: attacher,
+ pathTree: newPathNode(),
}
}
@@ -201,19 +202,16 @@ func (f *fidRef) maybeParent() *fidRef {
// notifyDelete marks all fidRefs as deleted.
//
-// Precondition: the write lock must be held on the given pathNode.
+// Precondition: this must be called via safelyWrite or safelyGlobal.
func notifyDelete(pn *pathNode) {
// Call on all local references.
- pn.fidRefs.Range(func(key, _ interface{}) bool {
- ref := key.(*fidRef)
+ pn.forEachChildRef(func(ref *fidRef, _ string) {
atomic.StoreUint32(&ref.deleted, 1)
- return true
})
// Call on all subtrees.
- pn.children.Range(func(_, value interface{}) bool {
- notifyDelete(value.(*pathNode))
- return true
+ pn.forEachChildNode(func(pn *pathNode) {
+ notifyDelete(pn)
})
}
@@ -225,28 +223,26 @@ func (f *fidRef) markChildDeleted(name string) {
atomic.StoreUint32(&ref.deleted, 1)
})
- // Mark everything below as deleted.
- notifyDelete(origPathNode)
+ if origPathNode != nil {
+ // Mark all children as deleted.
+ notifyDelete(origPathNode)
+ }
}
// notifyNameChange calls the relevant Renamed method on all nodes in the path,
// recursively. Note that this applies only for subtrees, as these
// notifications do not apply to the actual file whose name has changed.
//
-// Precondition: the write lock must be held on the given pathNode.
+// Precondition: this must be called via safelyGlobal.
func notifyNameChange(pn *pathNode) {
// Call on all local references.
- pn.fidRefs.Range(func(key, value interface{}) bool {
- ref := key.(*fidRef)
- name := value.(string)
+ pn.forEachChildRef(func(ref *fidRef, name string) {
ref.file.Renamed(ref.parent.file, name)
- return true
})
// Call on all subtrees.
- pn.children.Range(func(_, value interface{}) bool {
- notifyNameChange(value.(*pathNode))
- return true
+ pn.forEachChildNode(func(pn *pathNode) {
+ notifyNameChange(pn)
})
}
@@ -256,18 +252,25 @@ func notifyNameChange(pn *pathNode) {
func (f *fidRef) renameChildTo(oldName string, target *fidRef, newName string) {
target.markChildDeleted(newName)
origPathNode := f.pathNode.removeWithName(oldName, func(ref *fidRef) {
+ // N.B. DecRef can take f.pathNode's parent's childMu. This is
+ // allowed because renameMu is held for write via safelyGlobal.
ref.parent.DecRef() // Drop original reference.
ref.parent = target // Change parent.
ref.parent.IncRef() // Acquire new one.
- target.pathNode.addChild(ref, newName)
+ if f.pathNode == target.pathNode {
+ target.pathNode.addChildLocked(ref, newName)
+ } else {
+ target.pathNode.addChild(ref, newName)
+ }
ref.file.Renamed(target.file, newName)
})
- // Replace the previous (now deleted) path node.
- f.pathNode.children.Store(newName, origPathNode)
-
- // Call Renamed on everything above.
- notifyNameChange(origPathNode)
+ if origPathNode != nil {
+ // Replace the previous (now deleted) path node.
+ target.pathNode.addPathNodeFor(newName, origPathNode)
+ // Call Renamed on all children.
+ notifyNameChange(origPathNode)
+ }
}
// safelyRead executes the given operation with the local path node locked.
@@ -275,8 +278,8 @@ func (f *fidRef) renameChildTo(oldName string, target *fidRef, newName string) {
func (f *fidRef) safelyRead(fn func() error) (err error) {
f.server.renameMu.RLock()
defer f.server.renameMu.RUnlock()
- f.pathNode.mu.RLock()
- defer f.pathNode.mu.RUnlock()
+ f.pathNode.opMu.RLock()
+ defer f.pathNode.opMu.RUnlock()
return fn()
}
@@ -285,8 +288,8 @@ func (f *fidRef) safelyRead(fn func() error) (err error) {
func (f *fidRef) safelyWrite(fn func() error) (err error) {
f.server.renameMu.RLock()
defer f.server.renameMu.RUnlock()
- f.pathNode.mu.Lock()
- defer f.pathNode.mu.Unlock()
+ f.pathNode.opMu.Lock()
+ defer f.pathNode.opMu.Unlock()
return fn()
}
diff --git a/pkg/p9/transport.go b/pkg/p9/transport.go
index ef59077ff..5648df589 100644
--- a/pkg/p9/transport.go
+++ b/pkg/p9/transport.go
@@ -22,9 +22,9 @@ import (
"sync"
"syscall"
- "gvisor.googlesource.com/gvisor/pkg/fd"
- "gvisor.googlesource.com/gvisor/pkg/log"
- "gvisor.googlesource.com/gvisor/pkg/unet"
+ "gvisor.dev/gvisor/pkg/fd"
+ "gvisor.dev/gvisor/pkg/log"
+ "gvisor.dev/gvisor/pkg/unet"
)
// ErrSocket is returned in cases of a socket issue.
diff --git a/pkg/p9/transport_test.go b/pkg/p9/transport_test.go
index 0f88ff249..cdb3bc841 100644
--- a/pkg/p9/transport_test.go
+++ b/pkg/p9/transport_test.go
@@ -19,8 +19,8 @@ import (
"os"
"testing"
- "gvisor.googlesource.com/gvisor/pkg/fd"
- "gvisor.googlesource.com/gvisor/pkg/unet"
+ "gvisor.dev/gvisor/pkg/fd"
+ "gvisor.dev/gvisor/pkg/unet"
)
const (
@@ -179,6 +179,51 @@ func TestSendClosed(t *testing.T) {
}
}
+func BenchmarkSendRecv(b *testing.B) {
+ server, client, err := unet.SocketPair(false)
+ if err != nil {
+ b.Fatalf("socketpair got err %v expected nil", err)
+ }
+ defer server.Close()
+ defer client.Close()
+
+ // Exchange Rflush messages since these contain no data and therefore incur
+ // no additional marshaling overhead.
+ go func() {
+ for i := 0; i < b.N; i++ {
+ tag, m, err := recv(server, maximumLength, msgRegistry.get)
+ if err != nil {
+ b.Fatalf("recv got err %v expected nil", err)
+ }
+ if tag != Tag(1) {
+ b.Fatalf("got tag %v expected 1", tag)
+ }
+ if _, ok := m.(*Rflush); !ok {
+ b.Fatalf("got message %T expected *Rflush", m)
+ }
+ if err := send(server, Tag(2), &Rflush{}); err != nil {
+ b.Fatalf("send got err %v expected nil", err)
+ }
+ }
+ }()
+ b.ResetTimer()
+ for i := 0; i < b.N; i++ {
+ if err := send(client, Tag(1), &Rflush{}); err != nil {
+ b.Fatalf("send got err %v expected nil", err)
+ }
+ tag, m, err := recv(client, maximumLength, msgRegistry.get)
+ if err != nil {
+ b.Fatalf("recv got err %v expected nil", err)
+ }
+ if tag != Tag(2) {
+ b.Fatalf("got tag %v expected 2", tag)
+ }
+ if _, ok := m.(*Rflush); !ok {
+ b.Fatalf("got message %v expected *Rflush", m)
+ }
+ }
+}
+
func init() {
msgRegistry.register(MsgTypeBadDecode, func() message { return &badDecode{} })
}