summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/fs
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/sentry/fs')
-rw-r--r--pkg/sentry/fs/ext/BUILD26
-rw-r--r--pkg/sentry/fs/ext/block_map_file.go65
-rw-r--r--pkg/sentry/fs/ext/dentry.go2
-rw-r--r--pkg/sentry/fs/ext/directory.go36
-rw-r--r--pkg/sentry/fs/ext/ext.go25
-rw-r--r--pkg/sentry/fs/ext/ext_test.go2
-rw-r--r--pkg/sentry/fs/ext/extent_file.go260
-rw-r--r--pkg/sentry/fs/ext/extent_test.go161
-rw-r--r--pkg/sentry/fs/ext/filesystem.go30
-rw-r--r--pkg/sentry/fs/ext/inline_file.go66
-rw-r--r--pkg/sentry/fs/ext/inode.go158
-rw-r--r--pkg/sentry/fs/ext/named_pipe.go40
-rw-r--r--pkg/sentry/fs/ext/regular_file.go85
-rw-r--r--pkg/sentry/fs/ext/symlink.go57
-rw-r--r--pkg/sentry/fs/ext/utils.go33
-rw-r--r--pkg/sentry/fs/mounts.go2
16 files changed, 856 insertions, 192 deletions
diff --git a/pkg/sentry/fs/ext/BUILD b/pkg/sentry/fs/ext/BUILD
index 2c15875f5..8158aa522 100644
--- a/pkg/sentry/fs/ext/BUILD
+++ b/pkg/sentry/fs/ext/BUILD
@@ -1,14 +1,35 @@
package(licenses = ["notice"])
load("//tools/go_stateify:defs.bzl", "go_library", "go_test")
+load("//tools/go_generics:defs.bzl", "go_template_instance")
+
+go_template_instance(
+ name = "dentry_list",
+ out = "dentry_list.go",
+ package = "ext",
+ prefix = "dentry",
+ template = "//pkg/ilist:generic_list",
+ types = {
+ "Element": "*dentry",
+ "Linker": "*dentry",
+ },
+)
go_library(
name = "ext",
srcs = [
+ "block_map_file.go",
"dentry.go",
+ "dentry_list.go",
+ "directory.go",
"ext.go",
+ "extent_file.go",
"filesystem.go",
+ "inline_file.go",
"inode.go",
+ "named_pipe.go",
+ "regular_file.go",
+ "symlink.go",
"utils.go",
],
importpath = "gvisor.dev/gvisor/pkg/sentry/fs/ext",
@@ -16,9 +37,14 @@ go_library(
deps = [
"//pkg/abi/linux",
"//pkg/binary",
+ "//pkg/fd",
"//pkg/sentry/context",
+ "//pkg/sentry/fs",
"//pkg/sentry/fs/ext/disklayout",
"//pkg/sentry/kernel/auth",
+ "//pkg/sentry/kernel/pipe",
+ "//pkg/sentry/safemem",
+ "//pkg/sentry/usermem",
"//pkg/sentry/vfs",
"//pkg/syserror",
],
diff --git a/pkg/sentry/fs/ext/block_map_file.go b/pkg/sentry/fs/ext/block_map_file.go
new file mode 100644
index 000000000..9aabbd145
--- /dev/null
+++ b/pkg/sentry/fs/ext/block_map_file.go
@@ -0,0 +1,65 @@
+// Copyright 2019 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.
+
+package ext
+
+import (
+ "io"
+ "sync"
+
+ "gvisor.dev/gvisor/pkg/binary"
+)
+
+// blockMapFile is a type of regular file which uses direct/indirect block
+// addressing to store file data. This was deprecated in ext4.
+type blockMapFile struct {
+ regFile regularFile
+
+ // mu serializes changes to fileToPhysBlks.
+ mu sync.RWMutex
+
+ // fileToPhysBlks maps the file block numbers to the physical block numbers.
+ // the physical block number for the (i)th file block is stored in the (i)th
+ // index. This is initialized (at max) with the first 12 entries. The rest
+ // have to be read in from disk when required. Protected by mu.
+ fileToPhysBlks []uint32
+}
+
+// Compiles only if blockMapFile implements fileReader.
+var _ fileReader = (*blockMapFile)(nil)
+
+// Read implements fileReader.getFileReader.
+func (f *blockMapFile) getFileReader(dev io.ReaderAt, blkSize uint64, offset uint64) io.Reader {
+ panic("unimplemented")
+}
+
+// newBlockMapFile is the blockMapFile constructor. It initializes the file to
+// physical blocks map with (at most) the first 12 (direct) blocks.
+func newBlockMapFile(blkSize uint64, regFile regularFile) (*blockMapFile, error) {
+ file := &blockMapFile{regFile: regFile}
+ file.regFile.impl = file
+
+ toFill := uint64(12)
+ blksUsed := regFile.blksUsed(blkSize)
+ if blksUsed < toFill {
+ toFill = blksUsed
+ }
+
+ blkMap := regFile.inode.diskInode.Data()
+ file.fileToPhysBlks = make([]uint32, toFill)
+ for i := uint64(0); i < toFill; i++ {
+ binary.Unmarshal(blkMap[i*4:(i+1)*4], binary.LittleEndian, &file.fileToPhysBlks[i])
+ }
+ return file, nil
+}
diff --git a/pkg/sentry/fs/ext/dentry.go b/pkg/sentry/fs/ext/dentry.go
index 054fb42b6..19c9b3b2d 100644
--- a/pkg/sentry/fs/ext/dentry.go
+++ b/pkg/sentry/fs/ext/dentry.go
@@ -26,6 +26,8 @@ type dentry struct {
// share a single non-directory Inode (with hard links). inode is
// immutable.
inode *inode
+ // dentryEntry links Dentries into their parent directory.childList.
+ dentryEntry
}
// Compiles only if dentry implements vfs.DentryImpl.
diff --git a/pkg/sentry/fs/ext/directory.go b/pkg/sentry/fs/ext/directory.go
new file mode 100644
index 000000000..ab2b59e44
--- /dev/null
+++ b/pkg/sentry/fs/ext/directory.go
@@ -0,0 +1,36 @@
+// Copyright 2019 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.
+
+package ext
+
+// directory represents a directory inode. It holds the childList in memory.
+type directory struct {
+ inode inode
+
+ // childList is a list containing (1) child Dentries and (2) fake Dentries
+ // (with inode == nil) that represent the iteration position of
+ // directoryFDs. childList is used to support directoryFD.IterDirents()
+ // efficiently. childList is immutable.
+ childList dentryList
+
+ // TODO(b/134676337): Add directory navigators.
+}
+
+// newDirectroy is the directory constructor.
+func newDirectroy(inode inode) *directory {
+ // TODO(b/134676337): initialize childList.
+ file := &directory{inode: inode}
+ file.inode.impl = file
+ return file
+}
diff --git a/pkg/sentry/fs/ext/ext.go b/pkg/sentry/fs/ext/ext.go
index 10e235fb1..d303dd122 100644
--- a/pkg/sentry/fs/ext/ext.go
+++ b/pkg/sentry/fs/ext/ext.go
@@ -19,9 +19,9 @@ import (
"errors"
"fmt"
"io"
- "os"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/fd"
"gvisor.dev/gvisor/pkg/sentry/context"
"gvisor.dev/gvisor/pkg/sentry/fs/ext/disklayout"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
@@ -35,11 +35,11 @@ type filesystemType struct{}
// Compiles only if filesystemType implements vfs.FilesystemType.
var _ vfs.FilesystemType = (*filesystemType)(nil)
-// getDeviceFd returns the read seeker to the underlying device.
+// getDeviceFd returns an io.ReaderAt to the underlying device.
// Currently there are two ways of mounting an ext(2/3/4) fs:
// 1. Specify a mount with our internal special MountType in the OCI spec.
// 2. Expose the device to the container and mount it from application layer.
-func getDeviceFd(source string, opts vfs.NewFilesystemOptions) (io.ReadSeeker, error) {
+func getDeviceFd(source string, opts vfs.NewFilesystemOptions) (io.ReaderAt, error) {
if opts.InternalData == nil {
// User mount call.
// TODO(b/134676337): Open the device specified by `source` and return that.
@@ -47,20 +47,19 @@ func getDeviceFd(source string, opts vfs.NewFilesystemOptions) (io.ReadSeeker, e
}
// NewFilesystem call originated from within the sentry.
- fd, ok := opts.InternalData.(uintptr)
+ devFd, ok := opts.InternalData.(int)
if !ok {
- return nil, errors.New("internal data for ext fs must be a uintptr containing the file descriptor to device")
+ return nil, errors.New("internal data for ext fs must be an int containing the file descriptor to device")
}
- // We do not close this file because that would close the underlying device
- // file descriptor (which is required for reading the fs from disk).
- // TODO(b/134676337): Use pkg/fd instead.
- deviceFile := os.NewFile(fd, source)
- if deviceFile == nil {
- return nil, fmt.Errorf("ext4 device file descriptor is not valid: %d", fd)
+ if devFd < 0 {
+ return nil, fmt.Errorf("ext device file descriptor is not valid: %d", devFd)
}
- return deviceFile, nil
+ // The fd.ReadWriter returned from fd.NewReadWriter() does not take ownership
+ // of the file descriptor and hence will not close it when it is garbage
+ // collected.
+ return fd.NewReadWriter(devFd), nil
}
// NewFilesystem implements vfs.FilesystemType.NewFilesystem.
@@ -88,7 +87,7 @@ func (fstype filesystemType) NewFilesystem(ctx context.Context, creds *auth.Cred
return nil, nil, err
}
- rootInode, err := fs.getOrCreateInode(disklayout.RootDirInode)
+ rootInode, err := fs.getOrCreateInode(ctx, disklayout.RootDirInode)
if err != nil {
return nil, nil, err
}
diff --git a/pkg/sentry/fs/ext/ext_test.go b/pkg/sentry/fs/ext/ext_test.go
index ee7f7907c..18764e92a 100644
--- a/pkg/sentry/fs/ext/ext_test.go
+++ b/pkg/sentry/fs/ext/ext_test.go
@@ -69,7 +69,7 @@ func setUp(t *testing.T, imagePath string) (context.Context, *vfs.Filesystem, *v
// Mount the ext4 fs and retrieve the inode structure for the file.
mockCtx := contexttest.Context(t)
- fs, d, err := filesystemType{}.NewFilesystem(mockCtx, nil, localImagePath, vfs.NewFilesystemOptions{InternalData: f.Fd()})
+ fs, d, err := filesystemType{}.NewFilesystem(mockCtx, nil, localImagePath, vfs.NewFilesystemOptions{InternalData: int(f.Fd())})
if err != nil {
f.Close()
return nil, nil, nil, nil, err
diff --git a/pkg/sentry/fs/ext/extent_file.go b/pkg/sentry/fs/ext/extent_file.go
new file mode 100644
index 000000000..aa4102dbb
--- /dev/null
+++ b/pkg/sentry/fs/ext/extent_file.go
@@ -0,0 +1,260 @@
+// Copyright 2019 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.
+
+package ext
+
+import (
+ "io"
+ "sort"
+
+ "gvisor.dev/gvisor/pkg/binary"
+ "gvisor.dev/gvisor/pkg/sentry/fs/ext/disklayout"
+ "gvisor.dev/gvisor/pkg/syserror"
+)
+
+// extentFile is a type of regular file which uses extents to store file data.
+type extentFile struct {
+ regFile regularFile
+
+ // root is the root extent node. This lives in the 60 byte diskInode.Data().
+ // Immutable.
+ root disklayout.ExtentNode
+}
+
+// Compiles only if extentFile implements fileReader.
+var _ fileReader = (*extentFile)(nil)
+
+// Read implements fileReader.getFileReader.
+func (f *extentFile) getFileReader(dev io.ReaderAt, blkSize uint64, offset uint64) io.Reader {
+ return &extentReader{
+ dev: dev,
+ file: f,
+ fileOff: offset,
+ blkSize: blkSize,
+ }
+}
+
+// newExtentFile is the extent file constructor. It reads the entire extent
+// tree into memory.
+// TODO(b/134676337): Build extent tree on demand to reduce memory usage.
+func newExtentFile(dev io.ReaderAt, blkSize uint64, regFile regularFile) (*extentFile, error) {
+ file := &extentFile{regFile: regFile}
+ file.regFile.impl = file
+ err := file.buildExtTree(dev, blkSize)
+ if err != nil {
+ return nil, err
+ }
+ return file, nil
+}
+
+// buildExtTree builds the extent tree by reading it from disk by doing
+// running a simple DFS. It first reads the root node from the inode struct in
+// memory. Then it recursively builds the rest of the tree by reading it off
+// disk.
+//
+// Precondition: inode flag InExtents must be set.
+func (f *extentFile) buildExtTree(dev io.ReaderAt, blkSize uint64) error {
+ rootNodeData := f.regFile.inode.diskInode.Data()
+
+ binary.Unmarshal(rootNodeData[:disklayout.ExtentStructsSize], binary.LittleEndian, &f.root.Header)
+
+ // Root node can not have more than 4 entries: 60 bytes = 1 header + 4 entries.
+ if f.root.Header.NumEntries > 4 {
+ // read(2) specifies that EINVAL should be returned if the file is unsuitable
+ // for reading.
+ return syserror.EINVAL
+ }
+
+ f.root.Entries = make([]disklayout.ExtentEntryPair, f.root.Header.NumEntries)
+ for i, off := uint16(0), disklayout.ExtentStructsSize; i < f.root.Header.NumEntries; i, off = i+1, off+disklayout.ExtentStructsSize {
+ var curEntry disklayout.ExtentEntry
+ if f.root.Header.Height == 0 {
+ // Leaf node.
+ curEntry = &disklayout.Extent{}
+ } else {
+ // Internal node.
+ curEntry = &disklayout.ExtentIdx{}
+ }
+ binary.Unmarshal(rootNodeData[off:off+disklayout.ExtentStructsSize], binary.LittleEndian, curEntry)
+ f.root.Entries[i].Entry = curEntry
+ }
+
+ // If this node is internal, perform DFS.
+ if f.root.Header.Height > 0 {
+ for i := uint16(0); i < f.root.Header.NumEntries; i++ {
+ var err error
+ if f.root.Entries[i].Node, err = buildExtTreeFromDisk(dev, f.root.Entries[i].Entry, blkSize); err != nil {
+ return err
+ }
+ }
+ }
+
+ return nil
+}
+
+// buildExtTreeFromDisk reads the extent tree nodes from disk and recursively
+// builds the tree. Performs a simple DFS. It returns the ExtentNode pointed to
+// by the ExtentEntry.
+func buildExtTreeFromDisk(dev io.ReaderAt, entry disklayout.ExtentEntry, blkSize uint64) (*disklayout.ExtentNode, error) {
+ var header disklayout.ExtentHeader
+ off := entry.PhysicalBlock() * blkSize
+ err := readFromDisk(dev, int64(off), &header)
+ if err != nil {
+ return nil, err
+ }
+
+ entries := make([]disklayout.ExtentEntryPair, header.NumEntries)
+ for i, off := uint16(0), off+disklayout.ExtentStructsSize; i < header.NumEntries; i, off = i+1, off+disklayout.ExtentStructsSize {
+ var curEntry disklayout.ExtentEntry
+ if header.Height == 0 {
+ // Leaf node.
+ curEntry = &disklayout.Extent{}
+ } else {
+ // Internal node.
+ curEntry = &disklayout.ExtentIdx{}
+ }
+
+ err := readFromDisk(dev, int64(off), curEntry)
+ if err != nil {
+ return nil, err
+ }
+ entries[i].Entry = curEntry
+ }
+
+ // If this node is internal, perform DFS.
+ if header.Height > 0 {
+ for i := uint16(0); i < header.NumEntries; i++ {
+ var err error
+ entries[i].Node, err = buildExtTreeFromDisk(dev, entries[i].Entry, blkSize)
+ if err != nil {
+ return nil, err
+ }
+ }
+ }
+
+ return &disklayout.ExtentNode{header, entries}, nil
+}
+
+// extentReader implements io.Reader which can traverse the extent tree and
+// read file data. This is not thread safe.
+type extentReader struct {
+ dev io.ReaderAt
+ file *extentFile
+ fileOff uint64 // Represents the current file offset being read from.
+ blkSize uint64
+}
+
+// Compiles only if inlineReader implements io.Reader.
+var _ io.Reader = (*extentReader)(nil)
+
+// Read implements io.Reader.Read.
+func (r *extentReader) Read(dst []byte) (int, error) {
+ if len(dst) == 0 {
+ return 0, nil
+ }
+
+ if r.fileOff >= r.file.regFile.inode.diskInode.Size() {
+ return 0, io.EOF
+ }
+
+ return r.read(&r.file.root, dst)
+}
+
+// read is a helper which traverses the extent tree and reads data.
+func (r *extentReader) read(node *disklayout.ExtentNode, dst []byte) (int, error) {
+ // Perform a binary search for the node covering bytes starting at r.fileOff.
+ // A highly fragmented filesystem can have upto 340 entries and so linear
+ // search should be avoided. Finds the first entry which does not cover the
+ // file block we want and subtracts 1 to get the desired index.
+ fileBlk := r.fileBlock()
+ n := len(node.Entries)
+ found := sort.Search(n, func(i int) bool {
+ return node.Entries[i].Entry.FileBlock() > fileBlk
+ }) - 1
+
+ // We should be in this recursive step only if the data we want exists under
+ // the current node.
+ if found < 0 {
+ panic("searching for a file block in an extent entry which does not cover it")
+ }
+
+ read := 0
+ toRead := len(dst)
+ var curR int
+ var err error
+ for i := found; i < n && read < toRead; i++ {
+ if node.Header.Height == 0 {
+ curR, err = r.readFromExtent(node.Entries[i].Entry.(*disklayout.Extent), dst[read:])
+ } else {
+ curR, err = r.read(node.Entries[i].Node, dst[read:])
+ }
+
+ read += curR
+ if err != nil {
+ return read, err
+ }
+ }
+
+ return read, nil
+}
+
+// readFromExtent reads file data from the extent. It takes advantage of the
+// sequential nature of extents and reads file data from multiple blocks in one
+// call. Also updates the file offset.
+//
+// A non-nil error indicates that this is a partial read and there is probably
+// more to read from this extent. The caller should propagate the error upward
+// and not move to the next extent in the tree.
+//
+// A subsequent call to extentReader.Read should continue reading from where we
+// left off as expected.
+func (r *extentReader) readFromExtent(ex *disklayout.Extent, dst []byte) (int, error) {
+ curFileBlk := r.fileBlock()
+ exFirstFileBlk := ex.FileBlock()
+ exLastFileBlk := exFirstFileBlk + uint32(ex.Length) // This is exclusive.
+
+ // We should be in this recursive step only if the data we want exists under
+ // the current extent.
+ if curFileBlk < exFirstFileBlk || exLastFileBlk <= curFileBlk {
+ panic("searching for a file block in an extent which does not cover it")
+ }
+
+ curPhyBlk := uint64(curFileBlk-exFirstFileBlk) + ex.PhysicalBlock()
+ readStart := curPhyBlk*r.blkSize + r.fileBlockOff()
+
+ endPhyBlk := ex.PhysicalBlock() + uint64(ex.Length)
+ extentEnd := endPhyBlk * r.blkSize // This is exclusive.
+
+ toRead := int(extentEnd - readStart)
+ if len(dst) < toRead {
+ toRead = len(dst)
+ }
+
+ n, _ := r.dev.ReadAt(dst[:toRead], int64(readStart))
+ r.fileOff += uint64(n)
+ if n < toRead {
+ return n, syserror.EIO
+ }
+ return n, nil
+}
+
+// fileBlock returns the file block number we are currently reading.
+func (r *extentReader) fileBlock() uint32 {
+ return uint32(r.fileOff / r.blkSize)
+}
+
+// fileBlockOff returns the current offset within the current file block.
+func (r *extentReader) fileBlockOff() uint64 {
+ return r.fileOff % r.blkSize
+}
diff --git a/pkg/sentry/fs/ext/extent_test.go b/pkg/sentry/fs/ext/extent_test.go
index b3f342c8e..dff401114 100644
--- a/pkg/sentry/fs/ext/extent_test.go
+++ b/pkg/sentry/fs/ext/extent_test.go
@@ -16,6 +16,8 @@ package ext
import (
"bytes"
+ "io"
+ "math/rand"
"testing"
"github.com/google/go-cmp/cmp"
@@ -24,9 +26,14 @@ import (
"gvisor.dev/gvisor/pkg/sentry/fs/ext/disklayout"
)
-// TestExtentTree tests the extent tree building logic.
+const (
+ // mockExtentBlkSize is the mock block size used for testing.
+ // No block has more than 1 header + 4 entries.
+ mockExtentBlkSize = uint64(64)
+)
+
+// The tree described below looks like:
//
-// Test tree:
// 0.{Head}[Idx][Idx]
// / \
// / \
@@ -44,12 +51,8 @@ import (
//
// Please note that ext4 might not construct extent trees looking like this.
// This is purely for testing the tree traversal logic.
-func TestExtentTree(t *testing.T) {
- blkSize := uint64(64) // No block has more than 1 header + 4 entries.
- mockDisk := make([]byte, blkSize*10)
- mockInode := &inode{diskInode: &disklayout.InodeNew{}}
-
- node3 := &disklayout.ExtentNode{
+var (
+ node3 = &disklayout.ExtentNode{
Header: disklayout.ExtentHeader{
Magic: disklayout.ExtentMagic,
NumEntries: 1,
@@ -68,7 +71,7 @@ func TestExtentTree(t *testing.T) {
},
}
- node2 := &disklayout.ExtentNode{
+ node2 = &disklayout.ExtentNode{
Header: disklayout.ExtentHeader{
Magic: disklayout.ExtentMagic,
NumEntries: 1,
@@ -86,7 +89,7 @@ func TestExtentTree(t *testing.T) {
},
}
- node1 := &disklayout.ExtentNode{
+ node1 = &disklayout.ExtentNode{
Header: disklayout.ExtentHeader{
Magic: disklayout.ExtentMagic,
NumEntries: 2,
@@ -113,7 +116,7 @@ func TestExtentTree(t *testing.T) {
},
}
- node0 := &disklayout.ExtentNode{
+ node0 = &disklayout.ExtentNode{
Header: disklayout.ExtentHeader{
Magic: disklayout.ExtentMagic,
NumEntries: 2,
@@ -137,22 +140,95 @@ func TestExtentTree(t *testing.T) {
},
},
}
+)
- writeTree(mockInode, mockDisk, node0, blkSize)
+// TestExtentReader tests extentReader functionality. We should be able to use
+// the file reader like any other io.Reader.
+func TestExtentReader(t *testing.T) {
+ type extentReaderTest struct {
+ name string
+ from func(uint64) uint64
+ to func(uint64) uint64
+ }
- r := bytes.NewReader(mockDisk)
- if err := mockInode.buildExtTree(r, blkSize); err != nil {
- t.Fatalf("inode.buildExtTree failed: %v", err)
+ tests := []extentReaderTest{
+ {
+ name: "read first half",
+ from: beginning,
+ to: middle,
+ },
+ {
+ name: "read entire file",
+ from: beginning,
+ to: end,
+ },
+ {
+ name: "read second half",
+ from: middle,
+ to: end,
+ },
}
+ dev, mockExtentFile, want := extentTreeSetUp(t, node0)
+ size := mockExtentFile.regFile.inode.diskInode.Size()
+
+ for _, test := range tests {
+ from := test.from(size)
+ to := test.to(size)
+ fileReader := mockExtentFile.getFileReader(dev, mockExtentBlkSize, from)
+
+ got := make([]byte, to-from)
+ if _, err := io.ReadFull(fileReader, got); err != nil {
+ t.Errorf("file read failed: %v", err)
+ }
+
+ if diff := cmp.Diff(got, want[from:to]); diff != "" {
+ t.Errorf("file data mismatch (-want +got):\n%s", diff)
+ }
+ }
+}
+
+// TestBuildExtentTree tests the extent tree building logic.
+func TestBuildExtentTree(t *testing.T) {
+ _, mockExtentFile, _ := extentTreeSetUp(t, node0)
+
opt := cmpopts.IgnoreUnexported(disklayout.ExtentIdx{}, disklayout.ExtentHeader{})
- if diff := cmp.Diff(mockInode.root, node0, opt); diff != "" {
+ if diff := cmp.Diff(&mockExtentFile.root, node0, opt); diff != "" {
t.Errorf("extent tree mismatch (-want +got):\n%s", diff)
}
}
-// writeTree writes the tree represented by `root` to the inode and disk passed.
-func writeTree(in *inode, disk []byte, root *disklayout.ExtentNode, blkSize uint64) {
+// extentTreeSetUp writes the passed extent tree to a mock disk as an extent
+// tree. It also constucts a mock extent file with the same tree built in it.
+// It also writes random data file data and returns it.
+func extentTreeSetUp(t *testing.T, root *disklayout.ExtentNode) (io.ReaderAt, *extentFile, []byte) {
+ t.Helper()
+
+ mockDisk := make([]byte, mockExtentBlkSize*10)
+ mockExtentFile := &extentFile{
+ regFile: regularFile{
+ inode: inode{
+ diskInode: &disklayout.InodeNew{
+ InodeOld: disklayout.InodeOld{
+ SizeLo: uint32(mockExtentBlkSize) * getNumPhyBlks(root),
+ },
+ },
+ },
+ },
+ }
+
+ fileData := writeTree(&mockExtentFile.regFile.inode, mockDisk, node0, mockExtentBlkSize)
+
+ r := bytes.NewReader(mockDisk)
+ if err := mockExtentFile.buildExtTree(r, mockExtentBlkSize); err != nil {
+ t.Fatalf("inode.buildExtTree failed: %v", err)
+ }
+ return r, mockExtentFile, fileData
+}
+
+// writeTree writes the tree represented by `root` to the inode and disk. It
+// also writes random file data on disk.
+func writeTree(in *inode, disk []byte, root *disklayout.ExtentNode, mockExtentBlkSize uint64) []byte {
rootData := binary.Marshal(nil, binary.LittleEndian, root.Header)
for _, ep := range root.Entries {
rootData = binary.Marshal(rootData, binary.LittleEndian, ep.Entry)
@@ -160,26 +236,57 @@ func writeTree(in *inode, disk []byte, root *disklayout.ExtentNode, blkSize uint
copy(in.diskInode.Data(), rootData)
- if root.Header.Height > 0 {
- for _, ep := range root.Entries {
- writeTreeToDisk(disk, ep, blkSize)
+ var fileData []byte
+ for _, ep := range root.Entries {
+ if root.Header.Height == 0 {
+ fileData = append(fileData, writeRandomFileData(disk, ep.Entry.(*disklayout.Extent))...)
+ } else {
+ fileData = append(fileData, writeTreeToDisk(disk, ep)...)
}
}
+ return fileData
}
// writeTreeToDisk is the recursive step for writeTree which writes the tree
-// on the disk only.
-func writeTreeToDisk(disk []byte, curNode disklayout.ExtentEntryPair, blkSize uint64) {
+// on the disk only. Also writes random file data on disk.
+func writeTreeToDisk(disk []byte, curNode disklayout.ExtentEntryPair) []byte {
nodeData := binary.Marshal(nil, binary.LittleEndian, curNode.Node.Header)
for _, ep := range curNode.Node.Entries {
nodeData = binary.Marshal(nodeData, binary.LittleEndian, ep.Entry)
}
- copy(disk[curNode.Entry.PhysicalBlock()*blkSize:], nodeData)
+ copy(disk[curNode.Entry.PhysicalBlock()*mockExtentBlkSize:], nodeData)
+
+ var fileData []byte
+ for _, ep := range curNode.Node.Entries {
+ if curNode.Node.Header.Height == 0 {
+ fileData = append(fileData, writeRandomFileData(disk, ep.Entry.(*disklayout.Extent))...)
+ } else {
+ fileData = append(fileData, writeTreeToDisk(disk, ep)...)
+ }
+ }
+ return fileData
+}
+
+// writeRandomFileData writes random bytes to the blocks on disk that the
+// passed extent points to.
+func writeRandomFileData(disk []byte, ex *disklayout.Extent) []byte {
+ phyExStartBlk := ex.PhysicalBlock()
+ phyExStartOff := phyExStartBlk * mockExtentBlkSize
+ phyExEndOff := phyExStartOff + uint64(ex.Length)*mockExtentBlkSize
+ rand.Read(disk[phyExStartOff:phyExEndOff])
+ return disk[phyExStartOff:phyExEndOff]
+}
- if curNode.Node.Header.Height > 0 {
- for _, ep := range curNode.Node.Entries {
- writeTreeToDisk(disk, ep, blkSize)
+// getNumPhyBlks returns the number of physical blocks covered under the node.
+func getNumPhyBlks(node *disklayout.ExtentNode) uint32 {
+ var res uint32
+ for _, ep := range node.Entries {
+ if node.Header.Height == 0 {
+ res += uint32(ep.Entry.(*disklayout.Extent).Length)
+ } else {
+ res += getNumPhyBlks(ep.Node)
}
}
+ return res
}
diff --git a/pkg/sentry/fs/ext/filesystem.go b/pkg/sentry/fs/ext/filesystem.go
index 7150e75a5..12aeb5dac 100644
--- a/pkg/sentry/fs/ext/filesystem.go
+++ b/pkg/sentry/fs/ext/filesystem.go
@@ -31,22 +31,16 @@ type filesystem struct {
vfsfs vfs.Filesystem
- // mu serializes changes to the Dentry tree and the usage of the read seeker.
- mu sync.Mutex
+ // mu serializes changes to the Dentry tree.
+ mu sync.RWMutex
- // dev is the ReadSeeker for the underlying fs device. It is protected by mu.
- //
- // The ext filesystems aim to maximize locality, i.e. place all the data
- // blocks of a file close together. On a spinning disk, locality reduces the
- // amount of movement of the head hence speeding up IO operations. On an SSD
- // there are no moving parts but locality increases the size of each transer
- // request. Hence, having mutual exclusion on the read seeker while reading a
- // file *should* help in achieving the intended performance gains.
- //
- // Note: This synchronization was not coupled with the ReadSeeker itself
- // because we want to synchronize across read/seek operations for the
- // performance gains mentioned above. Helps enforcing one-file-at-a-time IO.
- dev io.ReadSeeker
+ // dev is the io.ReaderAt for the underlying fs device. It does not require
+ // protection because io.ReaderAt permits concurrent read calls to it. It
+ // translates to the pread syscall which passes on the read request directly
+ // to the device driver. Device drivers are intelligent in serving multiple
+ // concurrent read requests in the optimal order (taking locality into
+ // consideration).
+ dev io.ReaderAt
// inodeCache maps absolute inode numbers to the corresponding Inode struct.
// Inodes should be removed from this once their reference count hits 0.
@@ -69,13 +63,13 @@ var _ vfs.FilesystemImpl = (*filesystem)(nil)
// getOrCreateInode gets the inode corresponding to the inode number passed in.
// It creates a new one with the given inode number if one does not exist.
//
-// Preconditions: must be holding fs.mu.
-func (fs *filesystem) getOrCreateInode(inodeNum uint32) (*inode, error) {
+// Precondition: must be holding fs.mu.
+func (fs *filesystem) getOrCreateInode(ctx context.Context, inodeNum uint32) (*inode, error) {
if in, ok := fs.inodeCache[inodeNum]; ok {
return in, nil
}
- in, err := newInode(fs.dev, fs.sb, fs.bgs, inodeNum)
+ in, err := newInode(ctx, fs.dev, fs.sb, fs.bgs, inodeNum)
if err != nil {
return nil, err
}
diff --git a/pkg/sentry/fs/ext/inline_file.go b/pkg/sentry/fs/ext/inline_file.go
new file mode 100644
index 000000000..b9adfe548
--- /dev/null
+++ b/pkg/sentry/fs/ext/inline_file.go
@@ -0,0 +1,66 @@
+// Copyright 2019 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.
+
+package ext
+
+import (
+ "io"
+)
+
+// inlineFile is a type of regular file. All the data here is stored in the
+// inode.Data() array.
+type inlineFile struct {
+ regFile regularFile
+}
+
+// Compiles only if inlineFile implements fileReader.
+var _ fileReader = (*inlineFile)(nil)
+
+// getFileReader implements fileReader.getFileReader.
+func (f *inlineFile) getFileReader(_ io.ReaderAt, _ uint64, offset uint64) io.Reader {
+ diskInode := f.regFile.inode.diskInode
+ return &inlineReader{offset: offset, data: diskInode.Data()[:diskInode.Size()]}
+}
+
+// newInlineFile is the inlineFile constructor.
+func newInlineFile(regFile regularFile) *inlineFile {
+ file := &inlineFile{regFile: regFile}
+ file.regFile.impl = file
+ return file
+}
+
+// inlineReader implements io.Reader which can read the underlying data. This
+// is not thread safe.
+type inlineReader struct {
+ offset uint64
+ data []byte
+}
+
+// Compiles only if inlineReader implements io.Reader.
+var _ io.Reader = (*inlineReader)(nil)
+
+// Read implements io.Reader.Read.
+func (r *inlineReader) Read(dst []byte) (int, error) {
+ if len(dst) == 0 {
+ return 0, nil
+ }
+
+ if int(r.offset) >= len(r.data) {
+ return 0, io.EOF
+ }
+
+ n := copy(dst, r.data[r.offset:])
+ r.offset += uint64(n)
+ return n, nil
+}
diff --git a/pkg/sentry/fs/ext/inode.go b/pkg/sentry/fs/ext/inode.go
index df1ea0bda..00e022953 100644
--- a/pkg/sentry/fs/ext/inode.go
+++ b/pkg/sentry/fs/ext/inode.go
@@ -18,12 +18,26 @@ import (
"io"
"sync/atomic"
- "gvisor.dev/gvisor/pkg/binary"
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/sentry/context"
"gvisor.dev/gvisor/pkg/sentry/fs/ext/disklayout"
"gvisor.dev/gvisor/pkg/syserror"
)
// inode represents an ext inode.
+//
+// inode uses the same inheritance pattern that pkg/sentry/vfs structures use.
+// This has been done to increase memory locality.
+//
+// Implementations:
+// inode --
+// |-- pipe
+// |-- dir
+// |-- symlink
+// |-- regular--
+// |-- extent file
+// |-- block map file
+// |-- inline file
type inode struct {
// refs is a reference count. refs is accessed using atomic memory operations.
refs int64
@@ -35,9 +49,9 @@ type inode struct {
// diskInode gives us access to the inode struct on disk. Immutable.
diskInode disklayout.Inode
- // root is the root extent node. This lives in the 60 byte diskInode.Blocks().
- // Immutable. Nil if the inode does not use extents.
- root *disklayout.ExtentNode
+ // This is immutable. The first field of the implementations must have inode
+ // as the first field to ensure temporality.
+ impl interface{}
}
// incRef increments the inode ref count.
@@ -61,7 +75,7 @@ func (in *inode) tryIncRef() bool {
// decRef decrements the inode ref count and releases the inode resources if
// the ref count hits 0.
//
-// Preconditions: Must have locked fs.mu.
+// Precondition: Must have locked fs.mu.
func (in *inode) decRef(fs *filesystem) {
if refs := atomic.AddInt64(&in.refs, -1); refs == 0 {
delete(fs.inodeCache, in.inodeNum)
@@ -72,19 +86,17 @@ func (in *inode) decRef(fs *filesystem) {
// newInode is the inode constructor. Reads the inode off disk. Identifies
// inodes based on the absolute inode number on disk.
-//
-// Preconditions: Must hold the mutex of the filesystem containing dev.
-func newInode(dev io.ReadSeeker, sb disklayout.SuperBlock, bgs []disklayout.BlockGroup, inodeNum uint32) (*inode, error) {
+func newInode(ctx context.Context, dev io.ReaderAt, sb disklayout.SuperBlock, bgs []disklayout.BlockGroup, inodeNum uint32) (*inode, error) {
if inodeNum == 0 {
panic("inode number 0 on ext filesystems is not possible")
}
- in := &inode{refs: 1, inodeNum: inodeNum}
inodeRecordSize := sb.InodeSize()
+ var diskInode disklayout.Inode
if inodeRecordSize == disklayout.OldInodeSize {
- in.diskInode = &disklayout.InodeOld{}
+ diskInode = &disklayout.InodeOld{}
} else {
- in.diskInode = &disklayout.InodeNew{}
+ diskInode = &disklayout.InodeNew{}
}
// Calculate where the inode is actually placed.
@@ -93,16 +105,38 @@ func newInode(dev io.ReadSeeker, sb disklayout.SuperBlock, bgs []disklayout.Bloc
inodeTableOff := bgs[getBGNum(inodeNum, inodesPerGrp)].InodeTable() * blkSize
inodeOff := inodeTableOff + uint64(uint32(inodeRecordSize)*getBGOff(inodeNum, inodesPerGrp))
- // Read it from disk and figure out which type of inode this is.
- if err := readFromDisk(dev, int64(inodeOff), in.diskInode); err != nil {
+ if err := readFromDisk(dev, int64(inodeOff), diskInode); err != nil {
return nil, err
}
- if in.diskInode.Flags().Extents {
- in.buildExtTree(dev, blkSize)
+ // Build the inode based on its type.
+ inode := inode{
+ refs: 1,
+ inodeNum: inodeNum,
+ diskInode: diskInode,
}
- return in, nil
+ switch diskInode.Mode().FileType() {
+ case linux.ModeSymlink:
+ f, err := newSymlink(dev, blkSize, inode)
+ if err != nil {
+ return nil, err
+ }
+ return &f.inode, nil
+ case linux.ModeRegular:
+ f, err := newRegularFile(dev, blkSize, inode)
+ if err != nil {
+ return nil, err
+ }
+ return &f.inode, nil
+ case linux.ModeDirectory:
+ return &newDirectroy(inode).inode, nil
+ case linux.ModeNamedPipe:
+ return &newNamedPipe(ctx, inode).inode, nil
+ default:
+ // TODO(b/134676337): Return appropriate errors for sockets and devices.
+ return nil, syserror.EINVAL
+ }
}
// getBGNum returns the block group number that a given inode belongs to.
@@ -115,95 +149,3 @@ func getBGNum(inodeNum uint32, inodesPerGrp uint32) uint32 {
func getBGOff(inodeNum uint32, inodesPerGrp uint32) uint32 {
return (inodeNum - 1) % inodesPerGrp
}
-
-// buildExtTree builds the extent tree by reading it from disk by doing
-// running a simple DFS. It first reads the root node from the inode struct in
-// memory. Then it recursively builds the rest of the tree by reading it off
-// disk.
-//
-// Preconditions:
-// - Must hold the mutex of the filesystem containing dev.
-// - Inode flag InExtents must be set.
-func (in *inode) buildExtTree(dev io.ReadSeeker, blkSize uint64) error {
- rootNodeData := in.diskInode.Data()
-
- var rootHeader disklayout.ExtentHeader
- binary.Unmarshal(rootNodeData[:disklayout.ExtentStructsSize], binary.LittleEndian, &rootHeader)
-
- // Root node can not have more than 4 entries: 60 bytes = 1 header + 4 entries.
- if rootHeader.NumEntries > 4 {
- // read(2) specifies that EINVAL should be returned if the file is unsuitable
- // for reading.
- return syserror.EINVAL
- }
-
- rootEntries := make([]disklayout.ExtentEntryPair, rootHeader.NumEntries)
- for i, off := uint16(0), disklayout.ExtentStructsSize; i < rootHeader.NumEntries; i, off = i+1, off+disklayout.ExtentStructsSize {
- var curEntry disklayout.ExtentEntry
- if rootHeader.Height == 0 {
- // Leaf node.
- curEntry = &disklayout.Extent{}
- } else {
- // Internal node.
- curEntry = &disklayout.ExtentIdx{}
- }
- binary.Unmarshal(rootNodeData[off:off+disklayout.ExtentStructsSize], binary.LittleEndian, curEntry)
- rootEntries[i].Entry = curEntry
- }
-
- // If this node is internal, perform DFS.
- if rootHeader.Height > 0 {
- for i := uint16(0); i < rootHeader.NumEntries; i++ {
- var err error
- if rootEntries[i].Node, err = buildExtTreeFromDisk(dev, rootEntries[i].Entry, blkSize); err != nil {
- return err
- }
- }
- }
-
- in.root = &disklayout.ExtentNode{rootHeader, rootEntries}
- return nil
-}
-
-// buildExtTreeFromDisk reads the extent tree nodes from disk and recursively
-// builds the tree. Performs a simple DFS. It returns the ExtentNode pointed to
-// by the ExtentEntry.
-//
-// Preconditions: Must hold the mutex of the filesystem containing dev.
-func buildExtTreeFromDisk(dev io.ReadSeeker, entry disklayout.ExtentEntry, blkSize uint64) (*disklayout.ExtentNode, error) {
- var header disklayout.ExtentHeader
- off := entry.PhysicalBlock() * blkSize
- if err := readFromDisk(dev, int64(off), &header); err != nil {
- return nil, err
- }
-
- entries := make([]disklayout.ExtentEntryPair, header.NumEntries)
- for i, off := uint16(0), off+disklayout.ExtentStructsSize; i < header.NumEntries; i, off = i+1, off+disklayout.ExtentStructsSize {
- var curEntry disklayout.ExtentEntry
- if header.Height == 0 {
- // Leaf node.
- curEntry = &disklayout.Extent{}
- } else {
- // Internal node.
- curEntry = &disklayout.ExtentIdx{}
- }
-
- if err := readFromDisk(dev, int64(off), curEntry); err != nil {
- return nil, err
- }
- entries[i].Entry = curEntry
- }
-
- // If this node is internal, perform DFS.
- if header.Height > 0 {
- for i := uint16(0); i < header.NumEntries; i++ {
- var err error
- entries[i].Node, err = buildExtTreeFromDisk(dev, entries[i].Entry, blkSize)
- if err != nil {
- return nil, err
- }
- }
- }
-
- return &disklayout.ExtentNode{header, entries}, nil
-}
diff --git a/pkg/sentry/fs/ext/named_pipe.go b/pkg/sentry/fs/ext/named_pipe.go
new file mode 100644
index 000000000..0f3af1b53
--- /dev/null
+++ b/pkg/sentry/fs/ext/named_pipe.go
@@ -0,0 +1,40 @@
+// Copyright 2019 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.
+
+package ext
+
+import (
+ "gvisor.dev/gvisor/pkg/sentry/context"
+ "gvisor.dev/gvisor/pkg/sentry/fs"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/pipe"
+ "gvisor.dev/gvisor/pkg/sentry/usermem"
+)
+
+// namedPipe represents a named pipe inode. It is currently just a wrapper
+// around pkg/sentry/kernel/pipe.
+type namedPipe struct {
+ inode inode
+
+ p *pipe.Pipe
+ inodeOps fs.InodeOperations
+}
+
+// newNamedPipe is the namedPipe constructor.
+func newNamedPipe(ctx context.Context, inode inode) *namedPipe {
+ file := &namedPipe{inode: inode}
+ file.inode.impl = file
+ file.p = pipe.NewPipe(ctx, true /* isNamed */, pipe.DefaultPipeSize, usermem.PageSize)
+ file.inodeOps = pipe.NewInodeOperations(ctx, fs.FilePermsFromMode(file.inode.diskInode.Mode()), file.p)
+ return file
+}
diff --git a/pkg/sentry/fs/ext/regular_file.go b/pkg/sentry/fs/ext/regular_file.go
new file mode 100644
index 000000000..b48f61795
--- /dev/null
+++ b/pkg/sentry/fs/ext/regular_file.go
@@ -0,0 +1,85 @@
+// Copyright 2019 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.
+
+package ext
+
+import (
+ "io"
+)
+
+// fileReader is used to abstact away the complexity of how the file data is
+// stored under the hood. Provides a method to get a file reader which can be
+// used to read file data without worrying about how it is organized on disk.
+type fileReader interface {
+
+ // getFileReader returns a Reader implementation which can be used to read a
+ // file. It abstracts away the complexity of how the file is actually
+ // organized on disk. The reader is initialized with the passed offset.
+ //
+ // This reader is not meant to be retained across Read operations as it needs
+ // to be reinitialized with the correct offset for every Read.
+ getFileReader(dev io.ReaderAt, blkSize uint64, offset uint64) io.Reader
+}
+
+// regularFile represents a regular file's inode. This too follows the
+// inheritance pattern prevelant in the vfs layer described in
+// pkg/sentry/vfs/README.md.
+type regularFile struct {
+ inode inode
+
+ // This is immutable. The first field of fileReader implementations must be
+ // regularFile to ensure temporality.
+ impl fileReader
+}
+
+// newRegularFile is the regularFile constructor. It figures out what kind of
+// file this is and initializes the fileReader.
+func newRegularFile(dev io.ReaderAt, blkSize uint64, inode inode) (*regularFile, error) {
+ regFile := regularFile{
+ inode: inode,
+ }
+
+ inodeFlags := inode.diskInode.Flags()
+
+ if inodeFlags.Extents {
+ file, err := newExtentFile(dev, blkSize, regFile)
+ if err != nil {
+ return nil, err
+ }
+
+ file.regFile.inode.impl = &file.regFile
+ return &file.regFile, nil
+ }
+
+ if inodeFlags.Inline {
+ if inode.diskInode.Size() > 60 {
+ panic("ext fs: inline file larger than 60 bytes")
+ }
+
+ file := newInlineFile(regFile)
+ file.regFile.inode.impl = &file.regFile
+ return &file.regFile, nil
+ }
+
+ file, err := newBlockMapFile(blkSize, regFile)
+ if err != nil {
+ return nil, err
+ }
+ file.regFile.inode.impl = &file.regFile
+ return &file.regFile, nil
+}
+
+func (f *regularFile) blksUsed(blkSize uint64) uint64 {
+ return (f.inode.diskInode.Size() + blkSize - 1) / blkSize
+}
diff --git a/pkg/sentry/fs/ext/symlink.go b/pkg/sentry/fs/ext/symlink.go
new file mode 100644
index 000000000..6a55c1a7b
--- /dev/null
+++ b/pkg/sentry/fs/ext/symlink.go
@@ -0,0 +1,57 @@
+// Copyright 2019 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.
+
+package ext
+
+import (
+ "io"
+
+ "gvisor.dev/gvisor/pkg/syserror"
+)
+
+// symlink represents a symlink inode.
+type symlink struct {
+ inode inode
+ target string // immutable
+}
+
+// newSymlink is the symlink constructor. It reads out the symlink target from
+// the inode (however it might have been stored).
+func newSymlink(dev io.ReaderAt, blkSize uint64, inode inode) (*symlink, error) {
+ var file *symlink
+ var link []byte
+
+ // If the symlink target is lesser than 60 bytes, its stores in inode.Data().
+ // Otherwise either extents or block maps will be used to store the link.
+ size := inode.diskInode.Size()
+ if size < 60 {
+ link = inode.diskInode.Data()[:size]
+ } else {
+ // Create a regular file out of this inode and read out the target.
+ regFile, err := newRegularFile(dev, blkSize, inode)
+ if err != nil {
+ return nil, err
+ }
+
+ link = make([]byte, size)
+ reader := regFile.impl.getFileReader(dev, blkSize, 0)
+ if _, err := io.ReadFull(reader, link); err != nil {
+ return nil, syserror.EIO
+ }
+ }
+
+ file = &symlink{inode: inode, target: string(link)}
+ file.inode.impl = file
+ return file, nil
+}
diff --git a/pkg/sentry/fs/ext/utils.go b/pkg/sentry/fs/ext/utils.go
index 3472c5fa8..3d89d664d 100644
--- a/pkg/sentry/fs/ext/utils.go
+++ b/pkg/sentry/fs/ext/utils.go
@@ -15,38 +15,30 @@
package ext
import (
- "encoding/binary"
"io"
+ "gvisor.dev/gvisor/pkg/binary"
"gvisor.dev/gvisor/pkg/sentry/fs/ext/disklayout"
"gvisor.dev/gvisor/pkg/syserror"
)
// readFromDisk performs a binary read from disk into the given struct from
// the absolute offset provided.
-//
-// All disk reads should use this helper so we avoid reading from stale
-// previously used offsets. This function forces the offset parameter.
-//
-// Precondition: Must hold the mutex of the filesystem containing dev.
-func readFromDisk(dev io.ReadSeeker, abOff int64, v interface{}) error {
- if _, err := dev.Seek(abOff, io.SeekStart); err != nil {
- return syserror.EIO
- }
-
- if err := binary.Read(dev, binary.LittleEndian, v); err != nil {
+func readFromDisk(dev io.ReaderAt, abOff int64, v interface{}) error {
+ n := binary.Size(v)
+ buf := make([]byte, n)
+ if read, _ := dev.ReadAt(buf, abOff); read < int(n) {
return syserror.EIO
}
+ binary.Unmarshal(buf, binary.LittleEndian, v)
return nil
}
// readSuperBlock reads the SuperBlock from block group 0 in the underlying
// device. There are three versions of the superblock. This function identifies
// and returns the correct version.
-//
-// Precondition: Must hold the mutex of the filesystem containing dev.
-func readSuperBlock(dev io.ReadSeeker) (disklayout.SuperBlock, error) {
+func readSuperBlock(dev io.ReaderAt) (disklayout.SuperBlock, error) {
var sb disklayout.SuperBlock = &disklayout.SuperBlockOld{}
if err := readFromDisk(dev, disklayout.SbOffset, sb); err != nil {
return nil, err
@@ -76,19 +68,12 @@ func blockGroupsCount(sb disklayout.SuperBlock) uint64 {
blocksPerGroup := uint64(sb.BlocksPerGroup())
// Round up the result. float64 can compromise precision so do it manually.
- bgCount := blocksCount / blocksPerGroup
- if blocksCount%blocksPerGroup != 0 {
- bgCount++
- }
-
- return bgCount
+ return (blocksCount + blocksPerGroup - 1) / blocksPerGroup
}
// readBlockGroups reads the block group descriptor table from block group 0 in
// the underlying device.
-//
-// Precondition: Must hold the mutex of the filesystem containing dev.
-func readBlockGroups(dev io.ReadSeeker, sb disklayout.SuperBlock) ([]disklayout.BlockGroup, error) {
+func readBlockGroups(dev io.ReaderAt, sb disklayout.SuperBlock) ([]disklayout.BlockGroup, error) {
bgCount := blockGroupsCount(sb)
bgdSize := uint64(sb.BgDescSize())
is64Bit := sb.IncompatibleFeatures().Is64Bit
diff --git a/pkg/sentry/fs/mounts.go b/pkg/sentry/fs/mounts.go
index 693ffc760..728575864 100644
--- a/pkg/sentry/fs/mounts.go
+++ b/pkg/sentry/fs/mounts.go
@@ -678,7 +678,7 @@ func (mns *MountNamespace) ResolveExecutablePath(ctx context.Context, wd, name s
return "", syserror.ENOENT
}
-// GetPath returns the PATH as a slice of strings given the environemnt
+// GetPath returns the PATH as a slice of strings given the environment
// variables.
func GetPath(env []string) []string {
const prefix = "PATH="