Implement membarrier(2) commands other than *_SYNC_CORE.

Updates #267

PiperOrigin-RevId: 335713923
This commit is contained in:
Jamie Liu 2020-10-06 13:53:26 -07:00 committed by gVisor bot
parent 3dc3fb2375
commit 1336af78d5
20 changed files with 589 additions and 5 deletions

View File

@ -38,6 +38,7 @@ go_library(
"ipc.go",
"limits.go",
"linux.go",
"membarrier.go",
"mm.go",
"netdevice.go",
"netfilter.go",

View File

@ -0,0 +1,27 @@
// 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 linux
// membarrier(2) commands, from include/uapi/linux/membarrier.h
const (
MEMBARRIER_CMD_QUERY = 0
MEMBARRIER_CMD_GLOBAL = (1 << 0)
MEMBARRIER_CMD_GLOBAL_EXPEDITED = (1 << 1)
MEMBARRIER_CMD_REGISTER_GLOBAL_EXPEDITED = (1 << 2)
MEMBARRIER_CMD_PRIVATE_EXPEDITED = (1 << 3)
MEMBARRIER_CMD_REGISTER_PRIVATE_EXPEDITED = (1 << 4)
MEMBARRIER_CMD_PRIVATE_EXPEDITED_SYNC_CORE = (1 << 5)
MEMBARRIER_CMD_REGISTER_PRIVATE_EXPEDITED_SYNC_CORE = (1 << 6)
)

View File

@ -7,11 +7,14 @@ go_library(
srcs = [
"cgroup.go",
"hostmm.go",
"membarrier.go",
],
visibility = ["//pkg/sentry:internal"],
deps = [
"//pkg/abi/linux",
"//pkg/fd",
"//pkg/log",
"//pkg/usermem",
"@org_golang_x_sys//unix:go_default_library",
],
)

View File

@ -0,0 +1,90 @@
// 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 hostmm
import (
"syscall"
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/log"
)
var (
haveMembarrierGlobal = false
haveMembarrierPrivateExpedited = false
)
func init() {
supported, _, e := syscall.RawSyscall(unix.SYS_MEMBARRIER, linux.MEMBARRIER_CMD_QUERY, 0 /* flags */, 0 /* unused */)
if e != 0 {
if e != syscall.ENOSYS {
log.Warningf("membarrier(MEMBARRIER_CMD_QUERY) failed: %s", e.Error())
}
return
}
// We don't use MEMBARRIER_CMD_GLOBAL_EXPEDITED because this sends IPIs to
// all CPUs running tasks that have previously invoked
// MEMBARRIER_CMD_REGISTER_GLOBAL_EXPEDITED, which presents a DOS risk.
// (MEMBARRIER_CMD_GLOBAL is synchronize_rcu(), i.e. it waits for an RCU
// grace period to elapse without bothering other CPUs.
// MEMBARRIER_CMD_PRIVATE_EXPEDITED sends IPIs only to CPUs running tasks
// sharing the caller's MM.)
if supported&linux.MEMBARRIER_CMD_GLOBAL != 0 {
haveMembarrierGlobal = true
}
if req := uintptr(linux.MEMBARRIER_CMD_PRIVATE_EXPEDITED | linux.MEMBARRIER_CMD_REGISTER_PRIVATE_EXPEDITED); supported&req == req {
if _, _, e := syscall.RawSyscall(unix.SYS_MEMBARRIER, linux.MEMBARRIER_CMD_REGISTER_PRIVATE_EXPEDITED, 0 /* flags */, 0 /* unused */); e != 0 {
log.Warningf("membarrier(MEMBARRIER_CMD_REGISTER_PRIVATE_EXPEDITED) failed: %s", e.Error())
} else {
haveMembarrierPrivateExpedited = true
}
}
}
// HaveGlobalMemoryBarrier returns true if GlobalMemoryBarrier is supported.
func HaveGlobalMemoryBarrier() bool {
return haveMembarrierGlobal
}
// GlobalMemoryBarrier blocks until "all running threads [in the host OS] have
// passed through a state where all memory accesses to user-space addresses
// match program order between entry to and return from [GlobalMemoryBarrier]",
// as for membarrier(2).
//
// Preconditions: HaveGlobalMemoryBarrier() == true.
func GlobalMemoryBarrier() error {
if _, _, e := syscall.Syscall(unix.SYS_MEMBARRIER, linux.MEMBARRIER_CMD_GLOBAL, 0 /* flags */, 0 /* unused */); e != 0 {
return e
}
return nil
}
// HaveProcessMemoryBarrier returns true if ProcessMemoryBarrier is supported.
func HaveProcessMemoryBarrier() bool {
return haveMembarrierPrivateExpedited
}
// ProcessMemoryBarrier is equivalent to GlobalMemoryBarrier, but only
// synchronizes with threads sharing a virtual address space (from the host OS'
// perspective) with the calling thread.
//
// Preconditions: HaveProcessMemoryBarrier() == true.
func ProcessMemoryBarrier() error {
if _, _, e := syscall.RawSyscall(unix.SYS_MEMBARRIER, linux.MEMBARRIER_CMD_PRIVATE_EXPEDITED, 0 /* flags */, 0 /* unused */); e != 0 {
return e
}
return nil
}

View File

@ -235,6 +235,14 @@ type MemoryManager struct {
// vdsoSigReturnAddr is the address of 'vdso_sigreturn'.
vdsoSigReturnAddr uint64
// membarrierPrivateEnabled is non-zero if EnableMembarrierPrivate has
// previously been called. Since, as of this writing,
// MEMBARRIER_CMD_PRIVATE_EXPEDITED is implemented as a global memory
// barrier, membarrierPrivateEnabled has no other effect.
//
// membarrierPrivateEnabled is accessed using atomic memory operations.
membarrierPrivateEnabled uint32
}
// vma represents a virtual memory area.

View File

@ -17,6 +17,7 @@ package mm
import (
"fmt"
mrand "math/rand"
"sync/atomic"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
@ -1274,3 +1275,15 @@ func (mm *MemoryManager) VirtualDataSize() uint64 {
defer mm.mappingMu.RUnlock()
return mm.dataAS
}
// EnableMembarrierPrivate causes future calls to IsMembarrierPrivateEnabled to
// return true.
func (mm *MemoryManager) EnableMembarrierPrivate() {
atomic.StoreUint32(&mm.membarrierPrivateEnabled, 1)
}
// IsMembarrierPrivateEnabled returns true if mm.EnableMembarrierPrivate() has
// previously been called.
func (mm *MemoryManager) IsMembarrierPrivateEnabled() bool {
return atomic.LoadUint32(&mm.membarrierPrivateEnabled) != 0
}

View File

@ -15,6 +15,7 @@ go_library(
"//pkg/context",
"//pkg/seccomp",
"//pkg/sentry/arch",
"//pkg/sentry/hostmm",
"//pkg/sentry/memmap",
"//pkg/usermem",
],

View File

@ -56,6 +56,7 @@ go_library(
"//pkg/sentry/time",
"//pkg/sync",
"//pkg/usermem",
"@org_golang_x_sys//unix:go_default_library",
],
)

View File

@ -17,14 +17,23 @@ package kvm
import (
"syscall"
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/seccomp"
)
// SyscallFilters returns syscalls made exclusively by the KVM platform.
func (*KVM) SyscallFilters() seccomp.SyscallRules {
return seccomp.SyscallRules{
syscall.SYS_ARCH_PRCTL: {},
syscall.SYS_IOCTL: {},
syscall.SYS_ARCH_PRCTL: {},
syscall.SYS_IOCTL: {},
unix.SYS_MEMBARRIER: []seccomp.Rule{
{
seccomp.EqualTo(linux.MEMBARRIER_CMD_PRIVATE_EXPEDITED),
seccomp.EqualTo(0),
},
},
syscall.SYS_MMAP: {},
syscall.SYS_RT_SIGSUSPEND: {},
syscall.SYS_RT_SIGTIMEDWAIT: {},

View File

@ -17,13 +17,22 @@ package kvm
import (
"syscall"
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/seccomp"
)
// SyscallFilters returns syscalls made exclusively by the KVM platform.
func (*KVM) SyscallFilters() seccomp.SyscallRules {
return seccomp.SyscallRules{
syscall.SYS_IOCTL: {},
syscall.SYS_IOCTL: {},
unix.SYS_MEMBARRIER: []seccomp.Rule{
{
seccomp.EqualTo(linux.MEMBARRIER_CMD_PRIVATE_EXPEDITED),
seccomp.EqualTo(0),
},
},
syscall.SYS_MMAP: {},
syscall.SYS_RT_SIGSUSPEND: {},
syscall.SYS_RT_SIGTIMEDWAIT: {},

View File

@ -63,6 +63,9 @@ type runData struct {
type KVM struct {
platform.NoCPUPreemptionDetection
// KVM never changes mm_structs.
platform.UseHostProcessMemoryBarrier
// machine is the backing VM.
machine *machine
}

View File

@ -25,6 +25,7 @@ import (
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/seccomp"
"gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/hostmm"
"gvisor.dev/gvisor/pkg/sentry/memmap"
"gvisor.dev/gvisor/pkg/usermem"
)
@ -52,6 +53,10 @@ type Platform interface {
// can reliably return ErrContextCPUPreempted.
DetectsCPUPreemption() bool
// HaveGlobalMemoryBarrier returns true if the GlobalMemoryBarrier method
// is supported.
HaveGlobalMemoryBarrier() bool
// MapUnit returns the alignment used for optional mappings into this
// platform's AddressSpaces. Higher values indicate lower per-page costs
// for AddressSpace.MapFile. As a special case, a MapUnit of 0 indicates
@ -97,6 +102,15 @@ type Platform interface {
// called.
PreemptAllCPUs() error
// GlobalMemoryBarrier blocks until all threads running application code
// (via Context.Switch) and all task goroutines "have passed through a
// state where all memory accesses to user-space addresses match program
// order between entry to and return from [GlobalMemoryBarrier]", as for
// membarrier(2).
//
// Preconditions: HaveGlobalMemoryBarrier() == true.
GlobalMemoryBarrier() error
// SyscallFilters returns syscalls made exclusively by this platform.
SyscallFilters() seccomp.SyscallRules
}
@ -115,6 +129,43 @@ func (NoCPUPreemptionDetection) PreemptAllCPUs() error {
panic("This platform does not support CPU preemption detection")
}
// UseHostGlobalMemoryBarrier implements Platform.HaveGlobalMemoryBarrier and
// Platform.GlobalMemoryBarrier by invoking equivalent functionality on the
// host.
type UseHostGlobalMemoryBarrier struct{}
// HaveGlobalMemoryBarrier implements Platform.HaveGlobalMemoryBarrier.
func (UseHostGlobalMemoryBarrier) HaveGlobalMemoryBarrier() bool {
return hostmm.HaveGlobalMemoryBarrier()
}
// GlobalMemoryBarrier implements Platform.GlobalMemoryBarrier.
func (UseHostGlobalMemoryBarrier) GlobalMemoryBarrier() error {
return hostmm.GlobalMemoryBarrier()
}
// UseHostProcessMemoryBarrier implements Platform.HaveGlobalMemoryBarrier and
// Platform.GlobalMemoryBarrier by invoking a process-local memory barrier.
// This is faster than UseHostGlobalMemoryBarrier, but is only appropriate for
// platforms for which application code executes while using the sentry's
// mm_struct.
type UseHostProcessMemoryBarrier struct{}
// HaveGlobalMemoryBarrier implements Platform.HaveGlobalMemoryBarrier.
func (UseHostProcessMemoryBarrier) HaveGlobalMemoryBarrier() bool {
// Fall back to a global memory barrier if a process-local one isn't
// available.
return hostmm.HaveProcessMemoryBarrier() || hostmm.HaveGlobalMemoryBarrier()
}
// GlobalMemoryBarrier implements Platform.GlobalMemoryBarrier.
func (UseHostProcessMemoryBarrier) GlobalMemoryBarrier() error {
if hostmm.HaveProcessMemoryBarrier() {
return hostmm.ProcessMemoryBarrier()
}
return hostmm.GlobalMemoryBarrier()
}
// MemoryManager represents an abstraction above the platform address space
// which manages memory mappings and their contents.
type MemoryManager interface {

View File

@ -192,6 +192,7 @@ func (c *context) PullFullState(as platform.AddressSpace, ac arch.Context) {}
type PTrace struct {
platform.MMapMinAddr
platform.NoCPUPreemptionDetection
platform.UseHostGlobalMemoryBarrier
}
// New returns a new ptrace-based implementation of the platform interface.

View File

@ -21,6 +21,7 @@ go_library(
"sys_identity.go",
"sys_inotify.go",
"sys_lseek.go",
"sys_membarrier.go",
"sys_mempolicy.go",
"sys_mmap.go",
"sys_mount.go",

View File

@ -376,7 +376,7 @@ var AMD64 = &kernel.SyscallTable{
321: syscalls.CapError("bpf", linux.CAP_SYS_ADMIN, "", nil),
322: syscalls.Supported("execveat", Execveat),
323: syscalls.ErrorWithEvent("userfaultfd", syserror.ENOSYS, "", []string{"gvisor.dev/issue/266"}), // TODO(b/118906345)
324: syscalls.ErrorWithEvent("membarrier", syserror.ENOSYS, "", []string{"gvisor.dev/issue/267"}), // TODO(gvisor.dev/issue/267)
324: syscalls.PartiallySupported("membarrier", Membarrier, "Not supported on all platforms.", nil),
325: syscalls.PartiallySupported("mlock2", Mlock2, "Stub implementation. The sandbox lacks appropriate permissions.", nil),
// Syscalls implemented after 325 are "backports" from versions
@ -695,7 +695,7 @@ var ARM64 = &kernel.SyscallTable{
280: syscalls.CapError("bpf", linux.CAP_SYS_ADMIN, "", nil),
281: syscalls.Supported("execveat", Execveat),
282: syscalls.ErrorWithEvent("userfaultfd", syserror.ENOSYS, "", []string{"gvisor.dev/issue/266"}), // TODO(b/118906345)
283: syscalls.ErrorWithEvent("membarrier", syserror.ENOSYS, "", []string{"gvisor.dev/issue/267"}), // TODO(gvisor.dev/issue/267)
283: syscalls.PartiallySupported("membarrier", Membarrier, "Not supported on all platforms.", nil),
284: syscalls.PartiallySupported("mlock2", Mlock2, "Stub implementation. The sandbox lacks appropriate permissions.", nil),
// Syscalls after 284 are "backports" from versions of Linux after 4.4.

View File

@ -0,0 +1,70 @@
// 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 linux
import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/kernel"
"gvisor.dev/gvisor/pkg/syserror"
)
// Membarrier implements syscall membarrier(2).
func Membarrier(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) {
cmd := args[0].Int()
flags := args[1].Int()
p := t.Kernel().Platform
if !p.HaveGlobalMemoryBarrier() {
// Event for applications that want membarrier on a configuration that
// doesn't support them.
t.Kernel().EmitUnimplementedEvent(t)
return 0, nil, syserror.ENOSYS
}
if flags != 0 {
return 0, nil, syserror.EINVAL
}
switch cmd {
case linux.MEMBARRIER_CMD_QUERY:
const supportedCommands = linux.MEMBARRIER_CMD_GLOBAL |
linux.MEMBARRIER_CMD_GLOBAL_EXPEDITED |
linux.MEMBARRIER_CMD_REGISTER_GLOBAL_EXPEDITED |
linux.MEMBARRIER_CMD_PRIVATE_EXPEDITED |
linux.MEMBARRIER_CMD_REGISTER_PRIVATE_EXPEDITED
return supportedCommands, nil, nil
case linux.MEMBARRIER_CMD_PRIVATE_EXPEDITED:
if !t.MemoryManager().IsMembarrierPrivateEnabled() {
return 0, nil, syserror.EPERM
}
fallthrough
case linux.MEMBARRIER_CMD_GLOBAL, linux.MEMBARRIER_CMD_GLOBAL_EXPEDITED:
return 0, nil, p.GlobalMemoryBarrier()
case linux.MEMBARRIER_CMD_REGISTER_GLOBAL_EXPEDITED:
// no-op
return 0, nil, nil
case linux.MEMBARRIER_CMD_REGISTER_PRIVATE_EXPEDITED:
t.MemoryManager().EnableMembarrierPrivate()
return 0, nil, nil
case linux.MEMBARRIER_CMD_PRIVATE_EXPEDITED_SYNC_CORE, linux.MEMBARRIER_CMD_REGISTER_PRIVATE_EXPEDITED_SYNC_CORE:
// We're aware of these, but they aren't implemented since no platform
// supports them yet.
t.Kernel().EmitUnimplementedEvent(t)
fallthrough
default:
return 0, nil, syserror.EINVAL
}
}

View File

@ -162,6 +162,12 @@ var allowedSyscalls = seccomp.SyscallRules{
},
syscall.SYS_LSEEK: {},
syscall.SYS_MADVISE: {},
unix.SYS_MEMBARRIER: []seccomp.Rule{
{
seccomp.EqualTo(linux.MEMBARRIER_CMD_GLOBAL),
seccomp.EqualTo(0),
},
},
syscall.SYS_MINCORE: {},
// Used by the Go runtime as a temporarily workaround for a Linux
// 5.2-5.4 bug.

View File

@ -285,6 +285,10 @@ syscall_test(
test = "//test/syscalls/linux:madvise_test",
)
syscall_test(
test = "//test/syscalls/linux:membarrier_test",
)
syscall_test(
test = "//test/syscalls/linux:memory_accounting_test",
)

View File

@ -1155,6 +1155,24 @@ cc_binary(
],
)
cc_binary(
name = "membarrier_test",
testonly = 1,
srcs = ["membarrier.cc"],
linkstatic = 1,
deps = [
"@com_google_absl//absl/time",
gtest,
"//test/util:cleanup",
"//test/util:logging",
"//test/util:memory_util",
"//test/util:posix_error",
"//test/util:test_main",
"//test/util:test_util",
"//test/util:thread_util",
],
)
cc_binary(
name = "mempolicy_test",
testonly = 1,

View File

@ -0,0 +1,268 @@
// 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.
#include <errno.h>
#include <signal.h>
#include <sys/syscall.h>
#include <sys/types.h>
#include <unistd.h>
#include <atomic>
#include "absl/time/clock.h"
#include "absl/time/time.h"
#include "test/util/cleanup.h"
#include "test/util/logging.h"
#include "test/util/memory_util.h"
#include "test/util/posix_error.h"
#include "test/util/test_util.h"
#include "test/util/thread_util.h"
namespace gvisor {
namespace testing {
namespace {
// This is the classic test case for memory fences on architectures with total
// store ordering; see e.g. Intel SDM Vol. 3A Sec. 8.2.3.4 "Loads May Be
// Reordered with Earlier Stores to Different Locations". In each iteration of
// the test, given two variables X and Y initially set to 0
// (MembarrierTestSharedState::local_var and remote_var in the code), two
// threads execute as follows:
//
// T1 T2
// -- --
//
// X = 1 Y = 1
// T1fence() T2fence()
// read Y read X
//
// On architectures where memory writes may be locally buffered by each CPU
// (essentially all architectures), if T1fence() and T2fence() are omitted or
// ineffective, it is possible for both T1 and T2 to read 0 because the memory
// write from the other CPU is not yet visible outside that CPU. T1fence() and
// T2fence() are expected to perform the necessary synchronization to restore
// sequential consistency: both threads agree on a order of memory accesses that
// is consistent with program order in each thread, such that at least one
// thread reads 1.
//
// In the NoMembarrier test, T1fence() and T2fence() are both ordinary memory
// fences establishing ordering between memory accesses before and after the
// fence (std::atomic_thread_fence). In all other test cases, T1fence() is not a
// memory fence at all, but only prevents compiler reordering of memory accesses
// (std::atomic_signal_fence); T2fence() is an invocation of the membarrier()
// syscall, which establishes ordering of memory accesses before and after the
// syscall on both threads.
template <typename F>
int DoMembarrierTestSide(std::atomic<int>* our_var,
std::atomic<int> const& their_var,
F const& test_fence) {
our_var->store(1, std::memory_order_relaxed);
test_fence();
return their_var.load(std::memory_order_relaxed);
}
struct MembarrierTestSharedState {
std::atomic<int64_t> remote_iter_cur;
std::atomic<int64_t> remote_iter_done;
std::atomic<int> local_var;
std::atomic<int> remote_var;
int remote_obs_of_local_var;
void Init() {
remote_iter_cur.store(-1, std::memory_order_relaxed);
remote_iter_done.store(-1, std::memory_order_relaxed);
}
};
// Special value for MembarrierTestSharedState::remote_iter_cur indicating that
// the remote thread should terminate.
constexpr int64_t kRemoteIterStop = -2;
// Must be async-signal-safe.
template <typename F>
void RunMembarrierTestRemoteSide(MembarrierTestSharedState* state,
F const& test_fence) {
int64_t i = 0;
int64_t cur;
while (true) {
while ((cur = state->remote_iter_cur.load(std::memory_order_acquire)) < i) {
if (cur == kRemoteIterStop) {
return;
}
// spin
}
state->remote_obs_of_local_var =
DoMembarrierTestSide(&state->remote_var, state->local_var, test_fence);
state->remote_iter_done.store(i, std::memory_order_release);
i++;
}
}
template <typename F>
void RunMembarrierTestLocalSide(MembarrierTestSharedState* state,
F const& test_fence) {
// On test completion, instruct the remote thread to terminate.
Cleanup cleanup_remote([&] {
state->remote_iter_cur.store(kRemoteIterStop, std::memory_order_relaxed);
});
int64_t i = 0;
absl::Time end = absl::Now() + absl::Seconds(5); // arbitrary test duration
while (absl::Now() < end) {
// Reset both vars to 0.
state->local_var.store(0, std::memory_order_relaxed);
state->remote_var.store(0, std::memory_order_relaxed);
// Instruct the remote thread to begin this iteration.
state->remote_iter_cur.store(i, std::memory_order_release);
// Perform our side of the test.
auto local_obs_of_remote_var =
DoMembarrierTestSide(&state->local_var, state->remote_var, test_fence);
// Wait for the remote thread to finish this iteration.
while (state->remote_iter_done.load(std::memory_order_acquire) < i) {
// spin
}
ASSERT_TRUE(local_obs_of_remote_var != 0 ||
state->remote_obs_of_local_var != 0);
i++;
}
}
TEST(MembarrierTest, NoMembarrier) {
MembarrierTestSharedState state;
state.Init();
ScopedThread remote_thread([&] {
RunMembarrierTestRemoteSide(
&state, [] { std::atomic_thread_fence(std::memory_order_seq_cst); });
});
RunMembarrierTestLocalSide(
&state, [] { std::atomic_thread_fence(std::memory_order_seq_cst); });
}
enum membarrier_cmd {
MEMBARRIER_CMD_QUERY = 0,
MEMBARRIER_CMD_GLOBAL = (1 << 0),
MEMBARRIER_CMD_GLOBAL_EXPEDITED = (1 << 1),
MEMBARRIER_CMD_REGISTER_GLOBAL_EXPEDITED = (1 << 2),
MEMBARRIER_CMD_PRIVATE_EXPEDITED = (1 << 3),
MEMBARRIER_CMD_REGISTER_PRIVATE_EXPEDITED = (1 << 4),
};
int membarrier(membarrier_cmd cmd, int flags) {
return syscall(SYS_membarrier, cmd, flags);
}
PosixErrorOr<int> SupportedMembarrierCommands() {
int cmds = membarrier(MEMBARRIER_CMD_QUERY, 0);
if (cmds < 0) {
if (errno == ENOSYS) {
// No commands are supported.
return 0;
}
return PosixError(errno, "membarrier(MEMBARRIER_CMD_QUERY) failed");
}
return cmds;
}
TEST(MembarrierTest, Global) {
SKIP_IF((ASSERT_NO_ERRNO_AND_VALUE(SupportedMembarrierCommands()) &
MEMBARRIER_CMD_GLOBAL) == 0);
Mapping m = ASSERT_NO_ERRNO_AND_VALUE(
MmapAnon(kPageSize, PROT_READ | PROT_WRITE, MAP_SHARED));
auto state = static_cast<MembarrierTestSharedState*>(m.ptr());
state->Init();
pid_t const child_pid = fork();
if (child_pid == 0) {
// In child process.
RunMembarrierTestRemoteSide(
state, [] { TEST_PCHECK(membarrier(MEMBARRIER_CMD_GLOBAL, 0) == 0); });
_exit(0);
}
// In parent process.
ASSERT_THAT(child_pid, SyscallSucceeds());
Cleanup cleanup_child([&] {
int status;
ASSERT_THAT(waitpid(child_pid, &status, 0),
SyscallSucceedsWithValue(child_pid));
EXPECT_TRUE(WIFEXITED(status) && WEXITSTATUS(status) == 0)
<< " status " << status;
});
RunMembarrierTestLocalSide(
state, [] { std::atomic_signal_fence(std::memory_order_seq_cst); });
}
TEST(MembarrierTest, GlobalExpedited) {
constexpr int kRequiredCommands = MEMBARRIER_CMD_GLOBAL_EXPEDITED |
MEMBARRIER_CMD_REGISTER_GLOBAL_EXPEDITED;
SKIP_IF((ASSERT_NO_ERRNO_AND_VALUE(SupportedMembarrierCommands()) &
kRequiredCommands) != kRequiredCommands);
ASSERT_THAT(membarrier(MEMBARRIER_CMD_REGISTER_GLOBAL_EXPEDITED, 0),
SyscallSucceeds());
Mapping m = ASSERT_NO_ERRNO_AND_VALUE(
MmapAnon(kPageSize, PROT_READ | PROT_WRITE, MAP_SHARED));
auto state = static_cast<MembarrierTestSharedState*>(m.ptr());
state->Init();
pid_t const child_pid = fork();
if (child_pid == 0) {
// In child process.
RunMembarrierTestRemoteSide(state, [] {
TEST_PCHECK(membarrier(MEMBARRIER_CMD_GLOBAL_EXPEDITED, 0) == 0);
});
_exit(0);
}
// In parent process.
ASSERT_THAT(child_pid, SyscallSucceeds());
Cleanup cleanup_child([&] {
int status;
ASSERT_THAT(waitpid(child_pid, &status, 0),
SyscallSucceedsWithValue(child_pid));
EXPECT_TRUE(WIFEXITED(status) && WEXITSTATUS(status) == 0)
<< " status " << status;
});
RunMembarrierTestLocalSide(
state, [] { std::atomic_signal_fence(std::memory_order_seq_cst); });
}
TEST(MembarrierTest, PrivateExpedited) {
constexpr int kRequiredCommands = MEMBARRIER_CMD_PRIVATE_EXPEDITED |
MEMBARRIER_CMD_REGISTER_PRIVATE_EXPEDITED;
SKIP_IF((ASSERT_NO_ERRNO_AND_VALUE(SupportedMembarrierCommands()) &
kRequiredCommands) != kRequiredCommands);
ASSERT_THAT(membarrier(MEMBARRIER_CMD_REGISTER_PRIVATE_EXPEDITED, 0),
SyscallSucceeds());
MembarrierTestSharedState state;
state.Init();
ScopedThread remote_thread([&] {
RunMembarrierTestRemoteSide(&state, [] {
TEST_PCHECK(membarrier(MEMBARRIER_CMD_PRIVATE_EXPEDITED, 0) == 0);
});
});
RunMembarrierTestLocalSide(
&state, [] { std::atomic_signal_fence(std::memory_order_seq_cst); });
}
} // namespace
} // namespace testing
} // namespace gvisor