rocksdb/port/win/port_win.h

430 lines
10 KiB
C
Raw Normal View History

// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
//
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.
//
// See port_example.h for documentation for the following types/functions.
#pragma once
// Always want minimum headers
#ifndef WIN32_LEAN_AND_MEAN
#define WIN32_LEAN_AND_MEAN
#endif
#include <windows.h>
#include <string>
#include <thread>
#include <string.h>
#include <mutex>
2015-11-21 00:31:47 +01:00
#include <limits>
#include <condition_variable>
#include <malloc.h>
#include <intrin.h>
#include <process.h>
#include <stdint.h>
#include "port/win/win_thread.h"
#include "rocksdb/options.h"
#undef min
#undef max
#undef DeleteFile
#undef GetCurrentTime
#ifndef strcasecmp
#define strcasecmp _stricmp
#endif
#undef GetCurrentTime
#undef DeleteFile
#ifndef _SSIZE_T_DEFINED
using ssize_t = SSIZE_T;
#endif
// size_t printf formatting named in the manner of C99 standard formatting
// strings such as PRIu64
// in fact, we could use that one
#ifndef ROCKSDB_PRIszt
#define ROCKSDB_PRIszt "Iu"
#endif
#ifdef _MSC_VER
#define __attribute__(A)
// Thread local storage on Linux
// There is thread_local in C++11
#ifndef __thread
#define __thread __declspec(thread)
#endif
#endif
namespace ROCKSDB_NAMESPACE {
#define PREFETCH(addr, rw, locality)
extern const bool kDefaultToAdaptiveMutex;
namespace port {
// VS < 2015
#if defined(_MSC_VER) && (_MSC_VER < 1900)
2015-11-21 00:31:47 +01:00
// VS 15 has snprintf
#define snprintf _snprintf
2015-11-21 00:31:47 +01:00
#define ROCKSDB_NOEXCEPT
// std::numeric_limits<size_t>::max() is not constexpr just yet
// therefore, use the same limits
2015-11-21 00:31:47 +01:00
// For use at db/file_indexer.h kLevelMaxIndex
const uint32_t kMaxUint32 = UINT32_MAX;
2015-11-21 00:31:47 +01:00
const int kMaxInt32 = INT32_MAX;
const int kMinInt32 = INT32_MIN;
const int64_t kMaxInt64 = INT64_MAX;
const int64_t kMinInt64 = INT64_MIN;
2015-11-21 00:31:47 +01:00
const uint64_t kMaxUint64 = UINT64_MAX;
#ifdef _WIN64
const size_t kMaxSizet = UINT64_MAX;
#else
const size_t kMaxSizet = UINT_MAX;
#endif
#else // VS >= 2015 or MinGW
#define ROCKSDB_NOEXCEPT noexcept
// For use at db/file_indexer.h kLevelMaxIndex
const uint32_t kMaxUint32 = std::numeric_limits<uint32_t>::max();
const int kMaxInt32 = std::numeric_limits<int>::max();
const int kMinInt32 = std::numeric_limits<int>::min();
const uint64_t kMaxUint64 = std::numeric_limits<uint64_t>::max();
const int64_t kMaxInt64 = std::numeric_limits<int64_t>::max();
const int64_t kMinInt64 = std::numeric_limits<int64_t>::min();
const size_t kMaxSizet = std::numeric_limits<size_t>::max();
2015-11-21 00:31:47 +01:00
#endif //_MSC_VER
// "Windows is designed to run on little-endian computer architectures."
// https://docs.microsoft.com/en-us/windows/win32/sysinfo/registry-value-types
constexpr bool kLittleEndian = true;
#undef PLATFORM_IS_LITTLE_ENDIAN
class CondVar;
class Mutex {
public:
2015-10-13 00:41:20 +02:00
/* implicit */ Mutex(bool adaptive = kDefaultToAdaptiveMutex)
2015-10-13 23:28:11 +02:00
#ifndef NDEBUG
: locked_(false)
#endif
{ }
2015-10-13 00:41:20 +02:00
~Mutex();
2015-10-13 00:41:20 +02:00
void Lock() {
mutex_.lock();
#ifndef NDEBUG
locked_ = true;
#endif
}
void Unlock() {
#ifndef NDEBUG
locked_ = false;
#endif
mutex_.unlock();
}
bool TryLock() {
bool ret = mutex_.try_lock();
#ifndef NDEBUG
if (ret) {
locked_ = true;
}
#endif
return ret;
}
// this will assert if the mutex is not locked
// it does NOT verify that mutex is held by a calling thread
2015-10-13 00:41:20 +02:00
void AssertHeld() {
#ifndef NDEBUG
assert(locked_);
#endif
}
2015-10-13 00:41:20 +02:00
// Mutex is move only with lock ownership transfer
Mutex(const Mutex&) = delete;
void operator=(const Mutex&) = delete;
private:
2015-10-13 00:41:20 +02:00
friend class CondVar;
2015-10-13 00:41:20 +02:00
std::mutex& getLock() {
return mutex_;
}
std::mutex mutex_;
#ifndef NDEBUG
bool locked_;
#endif
};
class RWMutex {
public:
RWMutex() { InitializeSRWLock(&srwLock_); }
// No copying allowed
RWMutex(const RWMutex&) = delete;
void operator=(const RWMutex&) = delete;
void ReadLock() { AcquireSRWLockShared(&srwLock_); }
void WriteLock() { AcquireSRWLockExclusive(&srwLock_); }
void ReadUnlock() { ReleaseSRWLockShared(&srwLock_); }
void WriteUnlock() { ReleaseSRWLockExclusive(&srwLock_); }
// Empty as in POSIX
void AssertHeld() {}
private:
SRWLOCK srwLock_;
};
class CondVar {
public:
2015-10-13 00:41:20 +02:00
explicit CondVar(Mutex* mu) : mu_(mu) {
}
~CondVar();
void Wait();
bool TimedWait(uint64_t expiration_time);
void Signal();
void SignalAll();
2015-10-13 00:41:20 +02:00
// Condition var is not copy/move constructible
CondVar(const CondVar&) = delete;
CondVar& operator=(const CondVar&) = delete;
CondVar(CondVar&&) = delete;
CondVar& operator=(CondVar&&) = delete;
private:
std::condition_variable cv_;
Mutex* mu_;
};
#ifdef _POSIX_THREADS
using Thread = std::thread;
#else
// Wrapper around the platform efficient
// or otherwise preferrable implementation
using Thread = WindowsThread;
#endif
2015-11-21 00:31:47 +01:00
// OnceInit type helps emulate
// Posix semantics with initialization
// adopted in the project
struct OnceType {
struct Init {};
OnceType() {}
OnceType(const Init&) {}
OnceType(const OnceType&) = delete;
OnceType& operator=(const OnceType&) = delete;
std::once_flag flag_;
};
#define LEVELDB_ONCE_INIT port::OnceType::Init()
extern void InitOnce(OnceType* once, void (*initializer)());
#ifndef CACHE_LINE_SIZE
#define CACHE_LINE_SIZE 64U
#endif
#ifdef ROCKSDB_JEMALLOC
// Separate inlines so they can be replaced if needed
void* jemalloc_aligned_alloc(size_t size, size_t alignment) ROCKSDB_NOEXCEPT;
void jemalloc_aligned_free(void* p) ROCKSDB_NOEXCEPT;
#endif
inline void *cacheline_aligned_alloc(size_t size) {
#ifdef ROCKSDB_JEMALLOC
return jemalloc_aligned_alloc(size, CACHE_LINE_SIZE);
#else
return _aligned_malloc(size, CACHE_LINE_SIZE);
#endif
}
inline void cacheline_aligned_free(void *memblock) {
#ifdef ROCKSDB_JEMALLOC
jemalloc_aligned_free(memblock);
#else
_aligned_free(memblock);
#endif
}
extern const size_t kPageSize;
// https://gcc.gnu.org/bugzilla/show_bug.cgi?id=52991 for MINGW32
// could not be worked around with by -mno-ms-bitfields
#ifndef __MINGW32__
#define ALIGN_AS(n) __declspec(align(n))
#else
#define ALIGN_AS(n)
#endif
support for concurrent adds to memtable Summary: This diff adds support for concurrent adds to the skiplist memtable implementations. Memory allocation is made thread-safe by the addition of a spinlock, with small per-core buffers to avoid contention. Concurrent memtable writes are made via an additional method and don't impose a performance overhead on the non-concurrent case, so parallelism can be selected on a per-batch basis. Write thread synchronization is an increasing bottleneck for higher levels of concurrency, so this diff adds --enable_write_thread_adaptive_yield (default off). This feature causes threads joining a write batch group to spin for a short time (default 100 usec) using sched_yield, rather than going to sleep on a mutex. If the timing of the yield calls indicates that another thread has actually run during the yield then spinning is avoided. This option improves performance for concurrent situations even without parallel adds, although it has the potential to increase CPU usage (and the heuristic adaptation is not yet mature). Parallel writes are not currently compatible with inplace updates, update callbacks, or delete filtering. Enable it with --allow_concurrent_memtable_write (and --enable_write_thread_adaptive_yield). Parallel memtable writes are performance neutral when there is no actual parallelism, and in my experiments (SSD server-class Linux and varying contention and key sizes for fillrandom) they are always a performance win when there is more than one thread. Statistics are updated earlier in the write path, dropping the number of DB mutex acquisitions from 2 to 1 for almost all cases. This diff was motivated and inspired by Yahoo's cLSM work. It is more conservative than cLSM: RocksDB's write batch group leader role is preserved (along with all of the existing flush and write throttling logic) and concurrent writers are blocked until all memtable insertions have completed and the sequence number has been advanced, to preserve linearizability. My test config is "db_bench -benchmarks=fillrandom -threads=$T -batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T -level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999 -disable_auto_compactions --max_write_buffer_number=8 -max_background_flushes=8 --disable_wal --write_buffer_size=160000000 --block_size=16384 --allow_concurrent_memtable_write" on a two-socket Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1 thread I get ~440Kops/sec. Peak performance for 1 socket (numactl -N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance across both sockets happens at 30 threads, and is ~900Kops/sec, although with fewer threads there is less performance loss when the system has background work. Test Plan: 1. concurrent stress tests for InlineSkipList and DynamicBloom 2. make clean; make check 3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench 4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench 5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench 6. make clean; OPT=-DROCKSDB_LITE make check 7. verify no perf regressions when disabled Reviewers: igor, sdong Reviewed By: sdong Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
static inline void AsmVolatilePause() {
Fix MSVC-related build issues (#7439) Summary: This PR addresses some build and functional issues on MSVC targets, as a step towards an eventual goal of having RocksDB build successfully for Windows on ARM64. Addressed issues include: - BitsSetToOne and CountTrailingZeroBits do not compile on non-x64 MSVC targets. A fallback implementation of BitsSetToOne when Intel intrinsics are not available is added, based on the C++20 `<bit>` popcount implementation in Microsoft's STL. - The implementation of FloorLog2 for MSVC targets (including x64) gives incorrect results. The unit test easily detects this, but CircleCI is currently configured to only run a specific set of tests for Windows CMake builds, so this seems to have been unnoticed. - AsmVolatilePause does not use YieldProcessor on Windows ARM64 targets, even though it is available. - When CondVar::TimedWait calls Microsoft STL's condition_variable::wait_for, it can potentially trigger a bug (just recently fixed in the upcoming VS 16.8's STL) that deadlocks various tests that wait for a timer to execute, since `Timer::Run` doesn't get a chance to execute before being blocked by the test function acquiring the mutex. - In c_test, `GetTempDir` assumes a POSIX-style temp path. - `NormalizePath` did not eliminate consecutive POSIX-style path separators on Windows, resulting in test failures in e.g., wal_manager_test. - Various other test failures. In a followup PR I hope to modify CircleCI's config.yml to invoke all RocksDB unit tests in Windows CMake builds with CTest, instead of the current use of `run_ci_db_test.ps1` which requires individual tests to be specified and is missing many of the existing tests. Notes from peterd: FloorLog2 is not yet used in production code (it's for something in progress). I also added a few more inexpensive platform-dependent tests to Windows CircleCI runs. And included facebook/folly#1461 as requested Pull Request resolved: https://github.com/facebook/rocksdb/pull/7439 Reviewed By: jay-zhuang Differential Revision: D24021563 Pulled By: pdillinger fbshipit-source-id: 0ec2027c0d6a494d8a0fe38d9667fc2f7e29f7e7
2020-10-01 18:21:30 +02:00
#if defined(_M_IX86) || defined(_M_X64) || defined(_M_ARM64) || defined(_M_ARM)
YieldProcessor();
support for concurrent adds to memtable Summary: This diff adds support for concurrent adds to the skiplist memtable implementations. Memory allocation is made thread-safe by the addition of a spinlock, with small per-core buffers to avoid contention. Concurrent memtable writes are made via an additional method and don't impose a performance overhead on the non-concurrent case, so parallelism can be selected on a per-batch basis. Write thread synchronization is an increasing bottleneck for higher levels of concurrency, so this diff adds --enable_write_thread_adaptive_yield (default off). This feature causes threads joining a write batch group to spin for a short time (default 100 usec) using sched_yield, rather than going to sleep on a mutex. If the timing of the yield calls indicates that another thread has actually run during the yield then spinning is avoided. This option improves performance for concurrent situations even without parallel adds, although it has the potential to increase CPU usage (and the heuristic adaptation is not yet mature). Parallel writes are not currently compatible with inplace updates, update callbacks, or delete filtering. Enable it with --allow_concurrent_memtable_write (and --enable_write_thread_adaptive_yield). Parallel memtable writes are performance neutral when there is no actual parallelism, and in my experiments (SSD server-class Linux and varying contention and key sizes for fillrandom) they are always a performance win when there is more than one thread. Statistics are updated earlier in the write path, dropping the number of DB mutex acquisitions from 2 to 1 for almost all cases. This diff was motivated and inspired by Yahoo's cLSM work. It is more conservative than cLSM: RocksDB's write batch group leader role is preserved (along with all of the existing flush and write throttling logic) and concurrent writers are blocked until all memtable insertions have completed and the sequence number has been advanced, to preserve linearizability. My test config is "db_bench -benchmarks=fillrandom -threads=$T -batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T -level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999 -disable_auto_compactions --max_write_buffer_number=8 -max_background_flushes=8 --disable_wal --write_buffer_size=160000000 --block_size=16384 --allow_concurrent_memtable_write" on a two-socket Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1 thread I get ~440Kops/sec. Peak performance for 1 socket (numactl -N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance across both sockets happens at 30 threads, and is ~900Kops/sec, although with fewer threads there is less performance loss when the system has background work. Test Plan: 1. concurrent stress tests for InlineSkipList and DynamicBloom 2. make clean; make check 3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench 4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench 5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench 6. make clean; OPT=-DROCKSDB_LITE make check 7. verify no perf regressions when disabled Reviewers: igor, sdong Reviewed By: sdong Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
#endif
// it would be nice to get "wfe" on ARM here
}
extern int PhysicalCoreID();
// For Thread Local Storage abstraction
using pthread_key_t = DWORD;
inline int pthread_key_create(pthread_key_t* key, void (*destructor)(void*)) {
// Not used
(void)destructor;
pthread_key_t k = TlsAlloc();
if (TLS_OUT_OF_INDEXES == k) {
return ENOMEM;
}
*key = k;
return 0;
}
inline int pthread_key_delete(pthread_key_t key) {
if (!TlsFree(key)) {
return EINVAL;
}
return 0;
}
inline int pthread_setspecific(pthread_key_t key, const void* value) {
if (!TlsSetValue(key, const_cast<void*>(value))) {
return ENOMEM;
}
return 0;
}
inline void* pthread_getspecific(pthread_key_t key) {
void* result = TlsGetValue(key);
if (!result) {
if (GetLastError() != ERROR_SUCCESS) {
errno = EINVAL;
} else {
errno = NOERROR;
}
}
return result;
}
// UNIX equiv although errno numbers will be off
// using C-runtime to implement. Note, this does not
// feel space with zeros in case the file is extended.
int truncate(const char* path, int64_t length);
int Truncate(std::string path, int64_t length);
void Crash(const std::string& srcfile, int srcline);
extern int GetMaxOpenFiles();
std::string utf16_to_utf8(const std::wstring& utf16);
std::wstring utf8_to_utf16(const std::string& utf8);
using ThreadId = int;
extern void SetCpuPriority(ThreadId id, CpuPriority priority);
int64_t GetProcessID();
Built-in support for generating unique IDs, bug fix (#8708) Summary: Env::GenerateUniqueId() works fine on Windows and on POSIX where /proc/sys/kernel/random/uuid exists. Our other implementation is flawed and easily produces collision in a new multi-threaded test. As we rely more heavily on DB session ID uniqueness, this becomes a serious issue. This change combines several individually suitable entropy sources for reliable generation of random unique IDs, with goal of uniqueness and portability, not cryptographic strength nor maximum speed. Specifically: * Moves code for getting UUIDs from the OS to port::GenerateRfcUuid rather than in Env implementation details. Callers are now told whether the operation fails or succeeds. * Adds an internal API GenerateRawUniqueId for generating high-quality 128-bit unique identifiers, by combining entropy from three "tracks": * Lots of info from default Env like time, process id, and hostname. * std::random_device * port::GenerateRfcUuid (when working) * Built-in implementations of Env::GenerateUniqueId() will now always produce an RFC 4122 UUID string, either from platform-specific API or by converting the output of GenerateRawUniqueId. DB session IDs now use GenerateRawUniqueId while DB IDs (not as critical) try to use port::GenerateRfcUuid but fall back on GenerateRawUniqueId with conversion to an RFC 4122 UUID. GenerateRawUniqueId is declared and defined under env/ rather than util/ or even port/ because of the Env dependency. Likely follow-up: enhance GenerateRawUniqueId to be faster after the first call and to guarantee uniqueness within the lifetime of a single process (imparting the same property onto DB session IDs). Pull Request resolved: https://github.com/facebook/rocksdb/pull/8708 Test Plan: A new mini-stress test in env_test checks the various public and internal APIs for uniqueness, including each track of GenerateRawUniqueId individually. We can't hope to verify anywhere close to 128 bits of entropy, but it can at least detect flaws as bad as the old code. Serial execution of the new tests takes about 350 ms on my machine. Reviewed By: zhichao-cao, mrambacher Differential Revision: D30563780 Pulled By: pdillinger fbshipit-source-id: de4c9ff4b2f581cf784fcedb5f39f16e5185c364
2021-08-31 00:19:39 +02:00
// Uses platform APIs to generate a 36-character RFC-4122 UUID. Returns
// true on success or false on failure.
bool GenerateRfcUuid(std::string* output);
} // namespace port
#ifdef ROCKSDB_WINDOWS_UTF8_FILENAMES
#define RX_FILESTRING std::wstring
#define RX_FN(a) ROCKSDB_NAMESPACE::port::utf8_to_utf16(a)
#define FN_TO_RX(a) ROCKSDB_NAMESPACE::port::utf16_to_utf8(a)
#define RX_FNCMP(a, b) ::wcscmp(a, RX_FN(b).c_str())
#define RX_FNLEN(a) ::wcslen(a)
#define RX_DeleteFile DeleteFileW
#define RX_CreateFile CreateFileW
#define RX_CreateFileMapping CreateFileMappingW
#define RX_GetFileAttributesEx GetFileAttributesExW
#define RX_FindFirstFileEx FindFirstFileExW
#define RX_FindNextFile FindNextFileW
#define RX_WIN32_FIND_DATA WIN32_FIND_DATAW
#define RX_CreateDirectory CreateDirectoryW
#define RX_RemoveDirectory RemoveDirectoryW
#define RX_GetFileAttributesEx GetFileAttributesExW
#define RX_MoveFileEx MoveFileExW
#define RX_CreateHardLink CreateHardLinkW
#define RX_PathIsRelative PathIsRelativeW
#define RX_GetCurrentDirectory GetCurrentDirectoryW
#define RX_GetDiskFreeSpaceEx GetDiskFreeSpaceExW
#define RX_PathIsDirectory PathIsDirectoryW
#else
#define RX_FILESTRING std::string
#define RX_FN(a) a
#define FN_TO_RX(a) a
#define RX_FNCMP(a, b) strcmp(a, b)
#define RX_FNLEN(a) strlen(a)
#define RX_DeleteFile DeleteFileA
#define RX_CreateFile CreateFileA
#define RX_CreateFileMapping CreateFileMappingA
#define RX_GetFileAttributesEx GetFileAttributesExA
#define RX_FindFirstFileEx FindFirstFileExA
#define RX_CreateDirectory CreateDirectoryA
#define RX_FindNextFile FindNextFileA
#define RX_WIN32_FIND_DATA WIN32_FIND_DATAA
#define RX_CreateDirectory CreateDirectoryA
#define RX_RemoveDirectory RemoveDirectoryA
#define RX_GetFileAttributesEx GetFileAttributesExA
#define RX_MoveFileEx MoveFileExA
#define RX_CreateHardLink CreateHardLinkA
#define RX_PathIsRelative PathIsRelativeA
#define RX_GetCurrentDirectory GetCurrentDirectoryA
#define RX_GetDiskFreeSpaceEx GetDiskFreeSpaceExA
#define RX_PathIsDirectory PathIsDirectoryA
#endif
using port::pthread_key_t;
using port::pthread_key_create;
using port::pthread_key_delete;
using port::pthread_setspecific;
using port::pthread_getspecific;
using port::truncate;
} // namespace ROCKSDB_NAMESPACE