Skip to content

Commit

Permalink
[K/N] Widen thread_id to uintptr_t
Browse files Browse the repository at this point in the history
Currently inside the K/N runtime thread id is assumed to be 32-bit.

But in practice it's not always the case. So, when thread id provided
by `pthread_threadid_np` on iOS doesn't fit into 32 bits, runtime 
crashes. We've seen this in practice.

This commit makes thread id pointer size instead, and adjusts the
related code in runtime and compiler. Having it pointer size is
practically enough to guarantee that atomic operations on it will
still be possible.

^KT-70365
  • Loading branch information
troelsbjerre authored and Space Cloud committed Aug 7, 2024
1 parent 74b9b93 commit e642ca7
Show file tree
Hide file tree
Showing 17 changed files with 56 additions and 44 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -511,6 +511,7 @@ internal class CodegenLlvmHelpers(private val generationState: NativeGenerationS
val int16Type = LLVMInt16TypeInContext(llvmContext)!!
val int32Type = LLVMInt32TypeInContext(llvmContext)!!
val int64Type = LLVMInt64TypeInContext(llvmContext)!!
val intptrType = LLVMIntPtrTypeInContext(llvmContext, runtime.targetData)!!
val floatType = LLVMFloatTypeInContext(llvmContext)!!
val doubleType = LLVMDoubleTypeInContext(llvmContext)!!
val vector128Type = LLVMVectorType(floatType, 4)!!
Expand Down Expand Up @@ -540,11 +541,13 @@ internal class CodegenLlvmHelpers(private val generationState: NativeGenerationS
fun char16(value: Char): LLVMValueRef = constChar16(value).llvm
fun int32(value: Int): LLVMValueRef = constInt32(value).llvm
fun int64(value: Long): LLVMValueRef = constInt64(value).llvm
fun intptr(value: Int): LLVMValueRef = LLVMConstInt(intptrType, value.toLong(), 1)!!
fun float32(value: Float): LLVMValueRef = constFloat32(value).llvm
fun float64(value: Double): LLVMValueRef = constFloat64(value).llvm

val kNullInt8Ptr by lazy { LLVMConstNull(int8PtrType)!! }
val kNullInt32Ptr by lazy { LLVMConstNull(pointerType(int32Type))!! }
val kNullIntptrPtr by lazy { LLVMConstNull(pointerType(intptrType))!! }
val kImmInt32Zero by lazy { int32(0) }
val kImmInt32One by lazy { int32(1) }

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -545,7 +545,7 @@ internal class CodeGeneratorVisitor(
storeHeapRef(codegen.kNullObjHeaderPtr, address)
}
state.globalInitState?.let {
store(llvm.int32(FILE_NOT_INITIALIZED), it)
store(llvm.intptr(FILE_NOT_INITIALIZED), it)
}
ret(null)
}
Expand Down Expand Up @@ -785,17 +785,17 @@ internal class CodeGeneratorVisitor(

private fun getGlobalInitStateFor(container: IrDeclarationContainer): LLVMValueRef =
llvm.initializersGenerationState.fileGlobalInitStates.getOrPut(container) {
codegen.addGlobal("state_global$${container.initVariableSuffix}", llvm.int32Type, false).also {
LLVMSetInitializer(it, llvm.int32(FILE_NOT_INITIALIZED))
codegen.addGlobal("state_global$${container.initVariableSuffix}", llvm.intptrType, false).also {
LLVMSetInitializer(it, llvm.intptr(FILE_NOT_INITIALIZED))
LLVMSetLinkage(it, LLVMLinkage.LLVMInternalLinkage)
}
}

private fun getThreadLocalInitStateFor(container: IrDeclarationContainer): AddressAccess =
llvm.initializersGenerationState.fileThreadLocalInitStates.getOrPut(container) {
codegen.addKotlinThreadLocal("state_thread_local$${container.initVariableSuffix}", llvm.int32Type,
LLVMPreferredAlignmentOfType(llvm.runtime.targetData, llvm.int32Type), false).also {
LLVMSetInitializer((it as GlobalAddressAccess).getAddress(null), llvm.int32(FILE_NOT_INITIALIZED))
codegen.addKotlinThreadLocal("state_thread_local$${container.initVariableSuffix}", llvm.intptrType,
LLVMPreferredAlignmentOfType(llvm.runtime.targetData, llvm.intptrType), false).also {
LLVMSetInitializer((it as GlobalAddressAccess).getAddress(null), llvm.intptr(FILE_NOT_INITIALIZED))
}
}

Expand Down Expand Up @@ -2434,8 +2434,8 @@ internal class CodeGeneratorVisitor(
val bbExit = basicBlock("label_continue", null)
moveBlockAfterEntry(bbExit)
moveBlockAfterEntry(bbInit)
val state = load(llvm.int32Type, statePtr, memoryOrder = LLVMAtomicOrdering.LLVMAtomicOrderingAcquire)
condBr(icmpEq(state, llvm.int32(FILE_INITIALIZED)), bbExit, bbInit)
val state = load(llvm.intptrType, statePtr, memoryOrder = LLVMAtomicOrdering.LLVMAtomicOrderingAcquire)
condBr(icmpEq(state, llvm.intptr(FILE_INITIALIZED)), bbExit, bbInit)
positionAtEnd(bbInit)
call(llvm.callInitGlobalPossiblyLock, listOf(statePtr, initializerPtr),
exceptionHandler = currentCodeContext.exceptionHandler)
Expand All @@ -2456,13 +2456,13 @@ internal class CodeGeneratorVisitor(
moveBlockAfterEntry(bbExit)
moveBlockAfterEntry(bbCheckLocalState)
moveBlockAfterEntry(bbInit)
val globalState = load(llvm.int32Type, globalStatePtr)
val globalState = load(llvm.intptrType, globalStatePtr)
LLVMSetVolatile(globalState, 1)
// Make sure we're not in the middle of global initializer invocation -
// thread locals can be initialized only after all shared globals have been initialized.
condBr(icmpNe(globalState, llvm.int32(FILE_INITIALIZED)), bbExit, bbCheckLocalState)
condBr(icmpNe(globalState, llvm.intptr(FILE_INITIALIZED)), bbExit, bbCheckLocalState)
positionAtEnd(bbCheckLocalState)
condBr(icmpNe(load(llvm.int32Type, localStatePtr), llvm.int32(FILE_INITIALIZED)), bbInit, bbExit)
condBr(icmpNe(load(llvm.intptrType, localStatePtr), llvm.intptr(FILE_INITIALIZED)), bbInit, bbExit)
positionAtEnd(bbInit)
call(llvm.callInitThreadLocal, listOf(globalStatePtr, localStatePtr, initializerPtr),
exceptionHandler = currentCodeContext.exceptionHandler)
Expand All @@ -2480,9 +2480,9 @@ internal class CodeGeneratorVisitor(
val bbExit = basicBlock("label_continue", null)
moveBlockAfterEntry(bbExit)
moveBlockAfterEntry(bbInit)
condBr(icmpEq(load(llvm.int32Type, statePtr), llvm.int32(FILE_INITIALIZED)), bbExit, bbInit)
condBr(icmpEq(load(llvm.intptrType, statePtr), llvm.intptr(FILE_INITIALIZED)), bbExit, bbInit)
positionAtEnd(bbInit)
call(llvm.callInitThreadLocal, listOf(llvm.kNullInt32Ptr, statePtr, initializerPtr),
call(llvm.callInitThreadLocal, listOf(llvm.kNullIntptrPtr, statePtr, initializerPtr),
exceptionHandler = currentCodeContext.exceptionHandler)
br(bbExit)
positionAtEnd(bbExit)
Expand Down
2 changes: 1 addition & 1 deletion kotlin-native/runtime/src/gc/cms/cpp/ConcurrentMark.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -115,7 +115,7 @@ void gc::mark::ConcurrentMark::tryCollectRootSet(mm::ThreadData& thread, MarkTra
auto& gcData = thread.gc().impl().gc();
if (!gcData.tryLockRootSet()) return;

GCLogDebug(gcHandle().getEpoch(), "Root set collection on thread %d for thread %d", konan::currentThreadId(), thread.threadId());
GCLogDebug(gcHandle().getEpoch(), "Root set collection on thread %" PRIuPTR " for thread %" PRIuPTR, konan::currentThreadId(), thread.threadId());
gcData.publish();
collectRootSetForThread<MarkTraits>(gcHandle(), markQueue, thread);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,9 +27,9 @@ namespace {
template <typename Body>
ScopedThread createGCThread(const char* name, Body&& body) {
return ScopedThread(ScopedThread::attributes().name(name), [name, body] {
RuntimeLogDebug({kTagGC}, "%s %d starts execution", name, konan::currentThreadId());
RuntimeLogDebug({kTagGC}, "%s %" PRIuPTR " starts execution", name, konan::currentThreadId());
body();
RuntimeLogDebug({kTagGC}, "%s %d finishes execution", name, konan::currentThreadId());
RuntimeLogDebug({kTagGC}, "%s %" PRIuPTR " finishes execution", name, konan::currentThreadId());
});
}

Expand Down Expand Up @@ -63,7 +63,7 @@ bool gc::ConcurrentMarkAndSweep::ThreadData::tryLockRootSet() {
bool locked = rootSetLocked_.compare_exchange_strong(expected, true, std::memory_order_acq_rel);
if (locked) {
RuntimeLogDebug(
{kTagGC}, "Thread %d have exclusively acquired thread %d's root set", konan::currentThreadId(), threadData_.threadId());
{kTagGC}, "Thread %" PRIuPTR " have exclusively acquired thread %" PRIuPTR "'s root set", konan::currentThreadId(), threadData_.threadId());
}
return locked;
}
Expand Down
2 changes: 1 addition & 1 deletion kotlin-native/runtime/src/gc/common/cpp/GCStatistics.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -472,7 +472,7 @@ GCHandle::GCThreadRootSetScope::~GCThreadRootSetScope(){
if (!handle_.isValid()) return;
handle_.threadRootSetCollected(threadData_, threadLocalRoots_, stackRoots_);
GCLogDebug(
handle_.getEpoch(), "Collected root set for thread #%d: stack=%" PRIu64 " tls=%" PRIu64 " in %" PRIu64 " microseconds.",
handle_.getEpoch(), "Collected root set for thread #%" PRIuPTR ": stack=%" PRIu64 " tls=%" PRIu64 " in %" PRIu64 " microseconds.",
threadData_.threadId(), stackRoots_, threadLocalRoots_, getStageTime());
}

Expand Down
2 changes: 1 addition & 1 deletion kotlin-native/runtime/src/gc/pmcs/cpp/ParallelMark.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -198,7 +198,7 @@ void gc::mark::ParallelMark::tryCollectRootSet(mm::ThreadData& thread, MarkTrait
auto& gcData = thread.gc().impl().gc();
if (!gcData.tryLockRootSet()) return;

GCLogDebug(gcHandle().getEpoch(), "Root set collection on thread %d for thread %d",
GCLogDebug(gcHandle().getEpoch(), "Root set collection on thread %" PRIuPTR " for thread %" PRIuPTR,
konan::currentThreadId(), thread.threadId());
gcData.publish();
collectRootSetForThread<MarkTraits>(gcHandle(), markQueue, thread);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,9 +27,9 @@ namespace {
template<typename Body>
ScopedThread createGCThread(const char* name, Body&& body) {
return ScopedThread(ScopedThread::attributes().name(name), [name, body] {
RuntimeLogDebug({kTagGC}, "%s %d starts execution", name, konan::currentThreadId());
RuntimeLogDebug({kTagGC}, "%s %" PRIuPTR " starts execution", name, konan::currentThreadId());
body();
RuntimeLogDebug({kTagGC}, "%s %d finishes execution", name, konan::currentThreadId());
RuntimeLogDebug({kTagGC}, "%s %" PRIuPTR " finishes execution", name, konan::currentThreadId());
});
}

Expand Down Expand Up @@ -64,7 +64,7 @@ bool gc::ParallelMarkConcurrentSweep::ThreadData::tryLockRootSet() {
bool expected = false;
bool locked = rootSetLocked_.compare_exchange_strong(expected, true, std::memory_order_acq_rel);
if (locked) {
RuntimeLogDebug({kTagGC}, "Thread %d have exclusively acquired thread %d's root set", konan::currentThreadId(), threadData_.threadId());
RuntimeLogDebug({kTagGC}, "Thread %" PRIuPTR " have exclusively acquired thread %" PRIuPTR "'s root set", konan::currentThreadId(), threadData_.threadId());
}
return locked;
}
Expand Down
3 changes: 2 additions & 1 deletion kotlin-native/runtime/src/main/cpp/KAssert.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@
#include "KAssert.h"

#include <array>
#include <cinttypes>
#include <cstdarg>
#include <cstdlib>

Expand Down Expand Up @@ -40,7 +41,7 @@ void PrintAssert(bool allowStacktrace, const char* location, const char* format,
std::array<char, 1024> bufferStorage;
std_support::span<char> buffer(bufferStorage);

buffer = FormatToSpan(buffer, "[tid#%d] ", konan::currentThreadId());
buffer = FormatToSpan(buffer, "[tid#%" PRIuPTR "] ", konan::currentThreadId());

// Write the title with a source location.
if (location != nullptr) {
Expand Down
7 changes: 5 additions & 2 deletions kotlin-native/runtime/src/main/cpp/Porting.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
* limitations under the License.
*/

#include <cstdint>
#ifdef KONAN_ANDROID
#include <android/log.h>
#endif
Expand Down Expand Up @@ -234,12 +235,14 @@ NO_EXTERNAL_CALLS_CHECK NO_INLINE int gettid() {
}
#endif

NO_EXTERNAL_CALLS_CHECK int currentThreadId() {
NO_EXTERNAL_CALLS_CHECK uintptr_t currentThreadId() {
#if defined(KONAN_OSX) or defined(KONAN_IOS) or defined(KONAN_TVOS) or defined(KONAN_WATCHOS)
uint64_t tid;
pthread_t self = pthread_self();
RuntimeCheck(!pthread_threadid_np(self, &tid), "Error getting thread id");
RuntimeCheck((*(reinterpret_cast<int32_t*>(&tid) + 1)) == 0, "Thread id is not a uint32");
if constexpr (sizeof(uintptr_t) < sizeof(uint64_t)) {
RuntimeCheck(*(reinterpret_cast<int32_t*>(&tid) + 1) == 0, "Thread id does not fit in a pointer");
}
return tid;
#elif KONAN_ANDROID
return gettid();
Expand Down
3 changes: 2 additions & 1 deletion kotlin-native/runtime/src/main/cpp/Porting.h
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
#ifndef RUNTIME_PORTING_H
#define RUNTIME_PORTING_H

#include <cstdint>
#include <stdarg.h>
#include <stdint.h>
#include <stddef.h>
Expand All @@ -38,7 +39,7 @@ void consoleFlush();
// Thread control.
void onThreadExit(void (*destructor)(void*), void* destructorParameter);
bool isOnThreadExitNotSetOrAlreadyStarted();
int currentThreadId();
uintptr_t currentThreadId();

// Time operations.
uint64_t getTimeMillis();
Expand Down
13 changes: 7 additions & 6 deletions kotlin-native/runtime/src/main/cpp/Runtime.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
#include "Worker.h"
#include "KString.h"
#include <atomic>
#include <cstdint>
#include <cstdlib>
#include <thread>

Expand Down Expand Up @@ -385,8 +386,8 @@ RUNTIME_NOTHROW KBoolean Kotlin_Debugging_isLocal(KRef obj) {
return obj->local();
}

static void CallInitGlobalAwaitInitialized(int *state) {
int localState;
static void CallInitGlobalAwaitInitialized(uintptr_t* state) {
uintptr_t localState;
// Switch to the native state to avoid dead-locks.
{
kotlin::ThreadStateGuard guard(kotlin::ThreadState::kNative);
Expand All @@ -397,12 +398,12 @@ static void CallInitGlobalAwaitInitialized(int *state) {
if (localState == FILE_FAILED_TO_INITIALIZE) ThrowFileFailedToInitializeException(nullptr);
}

NO_INLINE void CallInitGlobalPossiblyLock(int* state, void (*init)()) {
int localState = std_support::atomic_ref{*state}.load(std::memory_order_acquire);
NO_INLINE void CallInitGlobalPossiblyLock(uintptr_t* state, void (*init)()) {
uintptr_t localState = std_support::atomic_ref{*state}.load(std::memory_order_acquire);
if (localState == FILE_INITIALIZED) return;
if (localState == FILE_FAILED_TO_INITIALIZE)
ThrowFileFailedToInitializeException(nullptr);
int threadId = konan::currentThreadId();
uintptr_t threadId = konan::currentThreadId();
if ((localState & 3) == FILE_BEING_INITIALIZED) {
if ((localState & ~3) != (threadId << 2)) {
CallInitGlobalAwaitInitialized(state);
Expand All @@ -426,7 +427,7 @@ NO_INLINE void CallInitGlobalPossiblyLock(int* state, void (*init)()) {
}
}

void CallInitThreadLocal(int volatile* globalState, int* localState, void (*init)()) {
void CallInitThreadLocal(uintptr_t volatile* globalState, uintptr_t* localState, void (*init)()) {
if (*localState == FILE_FAILED_TO_INITIALIZE || (globalState != nullptr && *globalState == FILE_FAILED_TO_INITIALIZE))
ThrowFileFailedToInitializeException(nullptr);
*localState = FILE_INITIALIZED;
Expand Down
5 changes: 3 additions & 2 deletions kotlin-native/runtime/src/main/cpp/Runtime.h
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@
#ifndef RUNTIME_RUNTIME_H
#define RUNTIME_RUNTIME_H

#include <cstdint>
#include "Porting.h"
#include "Memory.h"

Expand All @@ -32,8 +33,8 @@ void Kotlin_shutdownRuntime();
// Appends given node to an initializer list.
RUNTIME_NOTHROW void AppendToInitializersTail(struct InitNode*);

void CallInitGlobalPossiblyLock(int* state, void (*init)());
void CallInitThreadLocal(int volatile* globalState, int* localState, void (*init)());
void CallInitGlobalPossiblyLock(uintptr_t* state, void (*init)());
void CallInitThreadLocal(uintptr_t volatile* globalState, uintptr_t* localState, void (*init)());

bool Kotlin_memoryLeakCheckerEnabled();

Expand Down
8 changes: 4 additions & 4 deletions kotlin-native/runtime/src/main/cpp/RuntimePrivate.hpp
Original file line number Diff line number Diff line change
Expand Up @@ -8,10 +8,10 @@
namespace kotlin {
namespace internal {

inline constexpr int FILE_NOT_INITIALIZED = 0;
inline constexpr int FILE_BEING_INITIALIZED = 1;
inline constexpr int FILE_INITIALIZED = 2;
inline constexpr int FILE_FAILED_TO_INITIALIZE = 3;
inline constexpr uintptr_t FILE_NOT_INITIALIZED = 0;
inline constexpr uintptr_t FILE_BEING_INITIALIZED = 1;
inline constexpr uintptr_t FILE_INITIALIZED = 2;
inline constexpr uintptr_t FILE_FAILED_TO_INITIALIZE = 3;

} // namespace internal
} // namespace kotlin
2 changes: 1 addition & 1 deletion kotlin-native/runtime/src/mm/cpp/MemoryDump.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -100,7 +100,7 @@ class MemoryDumper {
}

void DumpThread(ThreadData& thread) {
RuntimeLogDebug({kTagMemDump}, "Dumping thread %d", thread.threadId());
RuntimeLogDebug({kTagMemDump}, "Dumping thread %" PRIuPTR, thread.threadId());
DumpU8(TAG_THREAD);
DumpId(&thread);
}
Expand Down
2 changes: 1 addition & 1 deletion kotlin-native/runtime/src/mm/cpp/SafePoint.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ std::atomic<void (*)(mm::ThreadData&)> safePointAction = nullptr;
class SafePointSignpostInterval : private Pinned {
public:
explicit SafePointSignpostInterval(mm::ThreadData& threadData) noexcept : id_(os_signpost_id_make_with_pointer(logObject, &threadData)) {
os_signpost_interval_begin(logObject, id_, SAFEPOINT_SIGNPOST_NAME, "thread id: %d", threadData.threadId());
os_signpost_interval_begin(logObject, id_, SAFEPOINT_SIGNPOST_NAME, "thread id: %" PRIuPTR, threadData.threadId());
}

~SafePointSignpostInterval() {
Expand Down
7 changes: 4 additions & 3 deletions kotlin-native/runtime/src/mm/cpp/ThreadData.hpp
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@
#define RUNTIME_MM_THREAD_DATA_H

#include <atomic>
#include <cstdint>
#include <vector>

#include "GlobalData.hpp"
Expand All @@ -27,7 +28,7 @@ namespace mm {
// Pin it in memory to prevent accidental copying.
class ThreadData final : private Pinned {
public:
explicit ThreadData(int threadId) noexcept :
explicit ThreadData(uintptr_t threadId) noexcept :
threadId_(threadId),
globalsThreadQueue_(GlobalsRegistry::Instance()),
specialRefRegistry_(SpecialRefRegistry::instance()),
Expand All @@ -38,7 +39,7 @@ class ThreadData final : private Pinned {

~ThreadData() = default;

int threadId() const noexcept { return threadId_; }
uintptr_t threadId() const noexcept { return threadId_; }

GlobalsRegistry::ThreadQueue& globalsThreadQueue() noexcept { return globalsThreadQueue_; }

Expand Down Expand Up @@ -75,7 +76,7 @@ class ThreadData final : private Pinned {
}

private:
const int threadId_;
const uintptr_t threadId_;
GlobalsRegistry::ThreadQueue globalsThreadQueue_;
ThreadLocalStorage tls_;
SpecialRefRegistry::ThreadQueue specialRefRegistry_;
Expand Down
3 changes: 2 additions & 1 deletion kotlin-native/runtime/src/mm/cpp/ThreadSuspensionTest.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@

#include "ThreadSuspension.hpp"

#include <cstdint>
#include <future>
#include <iostream>
#include <vector>
Expand Down Expand Up @@ -268,7 +269,7 @@ TEST_F(ThreadSuspensionTest, FileInitializationWithSuspend) {
ASSERT_THAT(collectThreadData(), testing::IsEmpty());
ASSERT_FALSE(mm::IsThreadSuspensionRequested());

int lock = internal::FILE_NOT_INITIALIZED;
uintptr_t lock = internal::FILE_NOT_INITIALIZED;

auto scopedInitializationMock = ScopedInitializationMock();
EXPECT_CALL(*scopedInitializationMock, Call()).WillOnce([] {
Expand Down

0 comments on commit e642ca7

Please sign in to comment.