summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/vfs/epoll.go
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/sentry/vfs/epoll.go')
-rw-r--r--pkg/sentry/vfs/epoll.go377
1 files changed, 377 insertions, 0 deletions
diff --git a/pkg/sentry/vfs/epoll.go b/pkg/sentry/vfs/epoll.go
new file mode 100644
index 000000000..7c83f9a5a
--- /dev/null
+++ b/pkg/sentry/vfs/epoll.go
@@ -0,0 +1,377 @@
+// Copyright 2020 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 vfs
+
+import (
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sync"
+ "gvisor.dev/gvisor/pkg/syserror"
+ "gvisor.dev/gvisor/pkg/waiter"
+)
+
+// epollCycleMu serializes attempts to register EpollInstances with other
+// EpollInstances in order to check for cycles.
+var epollCycleMu sync.Mutex
+
+// EpollInstance represents an epoll instance, as described by epoll(7).
+type EpollInstance struct {
+ vfsfd FileDescription
+ FileDescriptionDefaultImpl
+ DentryMetadataFileDescriptionImpl
+
+ // q holds waiters on this EpollInstance.
+ q waiter.Queue
+
+ // interest is the set of file descriptors that are registered with the
+ // EpollInstance for monitoring. interest is protected by interestMu.
+ interestMu sync.Mutex
+ interest map[epollInterestKey]*epollInterest
+
+ // mu protects fields in registered epollInterests.
+ mu sync.Mutex
+
+ // ready is the set of file descriptors that may be "ready" for I/O. Note
+ // that this must be an ordered list, not a map: "If more than maxevents
+ // file descriptors are ready when epoll_wait() is called, then successive
+ // epoll_wait() calls will round robin through the set of ready file
+ // descriptors. This behavior helps avoid starvation scenarios, where a
+ // process fails to notice that additional file descriptors are ready
+ // because it focuses on a set of file descriptors that are already known
+ // to be ready." - epoll_wait(2)
+ ready epollInterestList
+}
+
+type epollInterestKey struct {
+ // file is the registered FileDescription. No reference is held on file;
+ // instead, when the last reference is dropped, FileDescription.DecRef()
+ // removes the FileDescription from all EpollInstances. file is immutable.
+ file *FileDescription
+
+ // num is the file descriptor number with which this entry was registered.
+ // num is immutable.
+ num int32
+}
+
+// epollInterest represents an EpollInstance's interest in a file descriptor.
+type epollInterest struct {
+ // epoll is the owning EpollInstance. epoll is immutable.
+ epoll *EpollInstance
+
+ // key is the file to which this epollInterest applies. key is immutable.
+ key epollInterestKey
+
+ // waiter is registered with key.file. entry is protected by epoll.mu.
+ waiter waiter.Entry
+
+ // mask is the event mask associated with this registration, including
+ // flags EPOLLET and EPOLLONESHOT. mask is protected by epoll.mu.
+ mask uint32
+
+ // ready is true if epollInterestEntry is linked into epoll.ready. ready
+ // and epollInterestEntry are protected by epoll.mu.
+ ready bool
+ epollInterestEntry
+
+ // userData is the epoll_data_t associated with this epollInterest.
+ // userData is protected by epoll.mu.
+ userData [2]int32
+}
+
+// NewEpollInstanceFD returns a FileDescription representing a new epoll
+// instance. A reference is taken on the returned FileDescription.
+func (vfs *VirtualFilesystem) NewEpollInstanceFD() (*FileDescription, error) {
+ vd := vfs.NewAnonVirtualDentry("[eventpoll]")
+ defer vd.DecRef()
+ ep := &EpollInstance{
+ interest: make(map[epollInterestKey]*epollInterest),
+ }
+ if err := ep.vfsfd.Init(ep, linux.O_RDWR, vd.Mount(), vd.Dentry(), &FileDescriptionOptions{
+ UseDentryMetadata: true,
+ }); err != nil {
+ return nil, err
+ }
+ return &ep.vfsfd, nil
+}
+
+// Release implements FileDescriptionImpl.Release.
+func (ep *EpollInstance) Release() {
+ // Unregister all polled fds.
+ ep.interestMu.Lock()
+ defer ep.interestMu.Unlock()
+ for key, epi := range ep.interest {
+ file := key.file
+ file.epollMu.Lock()
+ delete(file.epolls, epi)
+ file.epollMu.Unlock()
+ file.EventUnregister(&epi.waiter)
+ }
+ ep.interest = nil
+}
+
+// Readiness implements waiter.Waitable.Readiness.
+func (ep *EpollInstance) Readiness(mask waiter.EventMask) waiter.EventMask {
+ if mask&waiter.EventIn == 0 {
+ return 0
+ }
+ ep.mu.Lock()
+ for epi := ep.ready.Front(); epi != nil; epi = epi.Next() {
+ wmask := waiter.EventMaskFromLinux(epi.mask)
+ if epi.key.file.Readiness(wmask)&wmask != 0 {
+ ep.mu.Unlock()
+ return waiter.EventIn
+ }
+ }
+ ep.mu.Unlock()
+ return 0
+}
+
+// EventRegister implements waiter.Waitable.EventRegister.
+func (ep *EpollInstance) EventRegister(e *waiter.Entry, mask waiter.EventMask) {
+ ep.q.EventRegister(e, mask)
+}
+
+// EventUnregister implements waiter.Waitable.EventUnregister.
+func (ep *EpollInstance) EventUnregister(e *waiter.Entry) {
+ ep.q.EventUnregister(e)
+}
+
+// Seek implements FileDescriptionImpl.Seek.
+func (ep *EpollInstance) Seek(ctx context.Context, offset int64, whence int32) (int64, error) {
+ // Linux: fs/eventpoll.c:eventpoll_fops.llseek == noop_llseek
+ return 0, nil
+}
+
+// AddInterest implements the semantics of EPOLL_CTL_ADD.
+//
+// Preconditions: A reference must be held on file.
+func (ep *EpollInstance) AddInterest(file *FileDescription, num int32, mask uint32, userData [2]int32) error {
+ // Check for cyclic polling if necessary.
+ subep, _ := file.impl.(*EpollInstance)
+ if subep != nil {
+ epollCycleMu.Lock()
+ // epollCycleMu must be locked for the rest of AddInterest to ensure
+ // that cyclic polling is not introduced after the check.
+ defer epollCycleMu.Unlock()
+ if subep.mightPoll(ep) {
+ return syserror.ELOOP
+ }
+ }
+
+ ep.interestMu.Lock()
+ defer ep.interestMu.Unlock()
+
+ // Fail if the key is already registered.
+ key := epollInterestKey{
+ file: file,
+ num: num,
+ }
+ if _, ok := ep.interest[key]; ok {
+ return syserror.EEXIST
+ }
+
+ // Register interest in file.
+ mask |= linux.EPOLLERR | linux.EPOLLRDHUP
+ epi := &epollInterest{
+ epoll: ep,
+ key: key,
+ mask: mask,
+ userData: userData,
+ }
+ ep.interest[key] = epi
+ wmask := waiter.EventMaskFromLinux(mask)
+ file.EventRegister(&epi.waiter, wmask)
+
+ // Check if the file is already ready.
+ if file.Readiness(wmask)&wmask != 0 {
+ epi.Callback(nil)
+ }
+
+ // Add epi to file.epolls so that it is removed when the last
+ // FileDescription reference is dropped.
+ file.epollMu.Lock()
+ file.epolls[epi] = struct{}{}
+ file.epollMu.Unlock()
+
+ return nil
+}
+
+func (ep *EpollInstance) mightPoll(ep2 *EpollInstance) bool {
+ return ep.mightPollRecursive(ep2, 4) // Linux: fs/eventpoll.c:EP_MAX_NESTS
+}
+
+func (ep *EpollInstance) mightPollRecursive(ep2 *EpollInstance, remainingRecursion int) bool {
+ ep.interestMu.Lock()
+ defer ep.interestMu.Unlock()
+ for key := range ep.interest {
+ nextep, ok := key.file.impl.(*EpollInstance)
+ if !ok {
+ continue
+ }
+ if nextep == ep2 {
+ return true
+ }
+ if remainingRecursion == 0 {
+ return true
+ }
+ if nextep.mightPollRecursive(ep2, remainingRecursion-1) {
+ return true
+ }
+ }
+ return false
+}
+
+// ModifyInterest implements the semantics of EPOLL_CTL_MOD.
+//
+// Preconditions: A reference must be held on file.
+func (ep *EpollInstance) ModifyInterest(file *FileDescription, num int32, mask uint32, userData [2]int32) error {
+ ep.interestMu.Lock()
+ defer ep.interestMu.Unlock()
+
+ // Fail if the key is not already registered.
+ epi, ok := ep.interest[epollInterestKey{
+ file: file,
+ num: num,
+ }]
+ if !ok {
+ return syserror.ENOENT
+ }
+
+ // Update epi for the next call to ep.ReadEvents().
+ ep.mu.Lock()
+ epi.mask = mask
+ epi.userData = userData
+ ep.mu.Unlock()
+
+ // Re-register with the new mask.
+ mask |= linux.EPOLLERR | linux.EPOLLRDHUP
+ file.EventUnregister(&epi.waiter)
+ wmask := waiter.EventMaskFromLinux(mask)
+ file.EventRegister(&epi.waiter, wmask)
+
+ // Check if the file is already ready with the new mask.
+ if file.Readiness(wmask)&wmask != 0 {
+ epi.Callback(nil)
+ }
+
+ return nil
+}
+
+// DeleteInterest implements the semantics of EPOLL_CTL_DEL.
+//
+// Preconditions: A reference must be held on file.
+func (ep *EpollInstance) DeleteInterest(file *FileDescription, num int32) error {
+ ep.interestMu.Lock()
+ defer ep.interestMu.Unlock()
+
+ // Fail if the key is not already registered.
+ epi, ok := ep.interest[epollInterestKey{
+ file: file,
+ num: num,
+ }]
+ if !ok {
+ return syserror.ENOENT
+ }
+
+ // Unregister from the file so that epi will no longer be readied.
+ file.EventUnregister(&epi.waiter)
+
+ // Forget about epi.
+ ep.removeLocked(epi)
+
+ file.epollMu.Lock()
+ delete(file.epolls, epi)
+ file.epollMu.Unlock()
+
+ return nil
+}
+
+// Callback implements waiter.EntryCallback.Callback.
+func (epi *epollInterest) Callback(*waiter.Entry) {
+ newReady := false
+ epi.epoll.mu.Lock()
+ if !epi.ready {
+ newReady = true
+ epi.ready = true
+ epi.epoll.ready.PushBack(epi)
+ }
+ epi.epoll.mu.Unlock()
+ if newReady {
+ epi.epoll.q.Notify(waiter.EventIn)
+ }
+}
+
+// Preconditions: ep.interestMu must be locked.
+func (ep *EpollInstance) removeLocked(epi *epollInterest) {
+ delete(ep.interest, epi.key)
+ ep.mu.Lock()
+ if epi.ready {
+ epi.ready = false
+ ep.ready.Remove(epi)
+ }
+ ep.mu.Unlock()
+}
+
+// ReadEvents reads up to len(events) ready events into events and returns the
+// number of events read.
+//
+// Preconditions: len(events) != 0.
+func (ep *EpollInstance) ReadEvents(events []linux.EpollEvent) int {
+ i := 0
+ // Hot path: avoid defer.
+ ep.mu.Lock()
+ var next *epollInterest
+ var requeue epollInterestList
+ for epi := ep.ready.Front(); epi != nil; epi = next {
+ next = epi.Next()
+ // Regardless of what else happens, epi is initially removed from the
+ // ready list.
+ ep.ready.Remove(epi)
+ wmask := waiter.EventMaskFromLinux(epi.mask)
+ ievents := epi.key.file.Readiness(wmask) & wmask
+ if ievents == 0 {
+ // Leave epi off the ready list.
+ epi.ready = false
+ continue
+ }
+ // Determine what we should do with epi.
+ switch {
+ case epi.mask&linux.EPOLLONESHOT != 0:
+ // Clear all events from the mask; they must be re-added by
+ // EPOLL_CTL_MOD.
+ epi.mask &= linux.EP_PRIVATE_BITS
+ fallthrough
+ case epi.mask&linux.EPOLLET != 0:
+ // Leave epi off the ready list.
+ epi.ready = false
+ default:
+ // Queue epi to be moved to the end of the ready list.
+ requeue.PushBack(epi)
+ }
+ // Report ievents.
+ events[i] = linux.EpollEvent{
+ Events: ievents.ToLinux(),
+ Fd: epi.userData[0],
+ Data: epi.userData[1],
+ }
+ i++
+ if i == len(events) {
+ break
+ }
+ }
+ ep.ready.PushBackList(&requeue)
+ ep.mu.Unlock()
+ return i
+}