Skip to content

[GLUTEN-11895][VL] Fix SIGSEGV on IOThreadPool threads during HDFS scan#11896

Open
guowangy wants to merge 1 commit intoapache:mainfrom
guowangy:velox-io-thread-fix
Open

[GLUTEN-11895][VL] Fix SIGSEGV on IOThreadPool threads during HDFS scan#11896
guowangy wants to merge 1 commit intoapache:mainfrom
guowangy:velox-io-thread-fix

Conversation

@guowangy
Copy link
Copy Markdown

@guowangy guowangy commented Apr 9, 2026

What changes are proposed in this pull request?

Fix SIGSEGV on CPUThreadPool threads during HDFS scan caused by DetachCurrentThread poisoning libhdfs.so's TLS-cached JNIEnv*.

Fixes #11895

Root cause

libhdfs.so caches JNIEnv* per thread in a two-level TLS structure:

  • Fast path: static __thread ThreadLocalState *quickTlsEnv (ELF linker-initialized, zero-cost read, no re-validation)
  • Slow path: pthread_getspecific(gTlsKey) (mutex-protected, only on first call per thread)

After the first AttachCurrentThread on a CPUThreadPool thread, libhdfs caches the JNIEnv* in quickTlsEnv. The fast path returns this pointer on all subsequent calls without checking validity.

Two Gluten destructors called vm_->DetachCurrentThread() unconditionally after JNI cleanup:

  • JniColumnarBatchIterator::~JniColumnarBatchIterator() (cpp/core/jni/JniCommon.cc)
  • JavaInputStreamAdaptor::Close() (cpp/core/jni/JniWrapper.cc)

Both used attachCurrentThreadAsDaemonOrThrow() followed by unconditional DetachCurrentThread(). This was not a proper attach/detach pair: attachCurrentThreadAsDaemonOrThrow only attaches if the thread is not already attached, but DetachCurrentThread ran regardless — detaching threads that libhdfs had attached. This freed the JVM's JavaThread object while quickTlsEnv still held the stale pointer.

The crash sequence:

  1. CPUThreadPool21 runs a preload task → libhdfs calls AttachCurrentThread, caches JNIEnv* in quickTlsEnv
  2. Object cleanup on the same thread → Gluten destructor calls DetachCurrentThreadJavaThread freed, but quickTlsEnv still holds stale pointer
  3. CPUThreadPool21 runs next preload task → hdfsGetPathInfo() → libhdfs fast path returns stale env → jni_NewStringUTF(stale_env)SIGSEGV

CPUThreadPool threads live for the entire executor JVM lifetime (created once in VeloxBackend::initConnector, destroyed only in VeloxBackend::tearDown), so the stale pointer persists across all subsequent queries.

Confirmed by core dump analysis (core.CPUThreadPool21.1770392 from TPC-DS on YARN):

  • RDI = 0x0 (NULL JavaThread*, set by block_if_vm_exited)
  • R12 = 0x7f3003a52200 (stale JNIEnv* from libhdfs TLS cache)
  • Memory at stale env shows JVM method resolution data (reused memory), not a valid JNI function table

Fix

Remove DetachCurrentThread() from both destructors with the following considerations:

  1. Broken ownership: the original code detached threads it didn't attach — attachCurrentThreadAsDaemonOrThrow is conditional (no-op if already attached) but DetachCurrentThread was unconditional
  2. Daemon threads (AttachCurrentThreadAsDaemon) do not block JVM shutdown
  3. folly::ThreadLocal<HdfsFile> destructors call hdfsCloseFile() (JNI) during thread exit — detaching mid-lifetime would crash these too
  4. libhdfs pthread_key destructor (hdfsThreadDestructor) calls DetachCurrentThread when the thread actually exits — proper cleanup still happens at thread exit when libhdfs is in use
  5. Non-HDFS paths (S3, GCS, local): libhdfs is absent, no pthread_key is registered, daemon-attached threads are safely reclaimed when the executor JVM exits

How was this patch tested?

  • JniThreadDetachTest.testIteratorDestructorDoesNotDetachThread: on a native std::thread (simulating CPUThreadPool), saves JNIEnv* (simulating libhdfs TLS cache), creates and destroys a real JniColumnarBatchIterator, then reuses the saved env for FindClass (simulating libhdfs's next hdfsGetPathInfo). With the bug: SIGSEGV crashes the JVM. With the fix: succeeds normally.
  • Existing surefire tests pass (mvn surefire:test -pl backends-velox)
  • Full TPC-DS benchmark on HDFS — no more CPUThreadPool SIGSEGV

Was this patch authored or co-authored using generative AI tooling?

Co-authored-by: Claude Opus 4.6

@github-actions github-actions bot added the VELOX label Apr 9, 2026
@guowangy guowangy force-pushed the velox-io-thread-fix branch from a962aba to 368f446 Compare April 9, 2026 04:57
@guowangy guowangy changed the title [GLUTEN-11895] [VL] Fix SIGSEGV on IOThreadPool threads during HDFS scan [GLUTEN-11895][VL] Fix SIGSEGV on IOThreadPool threads during HDFS scan Apr 9, 2026
@FelixYBW
Copy link
Copy Markdown
Contributor

FelixYBW commented Apr 9, 2026

Hold on to merge until we fix #11452 (comment)

Copy link
Copy Markdown
Member

@zhztheplayer zhztheplayer left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @guowangy. I wonder simply removing these detach statements will cause Java Thread object leak.

You can give it a try to see whether the heap usage keeps going up with this change. Or have a look at #11895 (comment).

@guowangy
Copy link
Copy Markdown
Author

guowangy commented Apr 15, 2026

@zhztheplayer

Thanks @guowangy. I wonder simply removing these detach statements will cause Java Thread object leak.

In my opinions, no leak.

For HDFS threads: libhdfs has its own cleanup — it calls DetachCurrentThread automatically when each thread exits, via hdfsThreadDestructor. The old code was calling DetachCurrentThread prematurely at object destruction time — before thread exit — which corrupted libhdfs's cached JNIEnv* and caused the crash.

For non-HDFS native threads: attachCurrentThreadAsDaemonOrThrow attaches them as daemon on first use. They are joined by ioExecutor_.reset() inside the JVM shutdown hook before the JVM exits. Daemon threads do not block JVM shutdown. The attached-thread count is bounded by pool size, not by query or operation count.

For Spark task threads and shutdown threads: these are JVM-managed and already attached — attachCurrentThreadAsDaemonOrThrow is a no-op (GetEnv returns JNI_OK). No new JavaThread is created.

You can give it a try to see whether the heap usage keeps going up with this change. Or have a look at #11895 (comment).

I don't see heap usage keeps going up in an 8-hours testing.

@zhztheplayer
Copy link
Copy Markdown
Member

Thanks for the experiment @guowangy.

I revisited the code and ioExecutor_ is global and should not cause leak as you said. It's worth checking spillExecutor_ , whether the spill threads have any callbacks to Gluten JNI code to cause them to be attached. The feature is turned on by setting spark.gluten.sql.columnar.backend.velox.spillThreadNum.

I don't see heap usage keeps going up in an 8-hours testing.

Perhaps, using jstack to view all Java threads is a simpler way to guess such leakages.

@FelixYBW
Copy link
Copy Markdown
Contributor

  1. CPUThreadPool21 runs a preload task → libhdfs calls AttachCurrentThread, caches JNIEnv* in quickTlsEnv
  2. Object cleanup on the same thread → Gluten destructor calls DetachCurrentThread → JavaThread freed, but quickTlsEnv still holds stale pointer
  3. CPUThreadPool21 runs next preload task → hdfsGetPathInfo() → libhdfs fast path returns stale env → jni_NewStringUTF(stale_env) → SIGSEGV

Should we cleanup the stale pointer at step2 and re-attach in 3?

@FelixYBW
Copy link
Copy Markdown
Contributor

It's a libhdfs only issue, right? Does the PR work on S3, abfs, gcs?

@guowangy
Copy link
Copy Markdown
Author

@FelixYBW

  1. CPUThreadPool21 runs a preload task → libhdfs calls AttachCurrentThread, caches JNIEnv* in quickTlsEnv
  2. Object cleanup on the same thread → Gluten destructor calls DetachCurrentThread → JavaThread freed, but quickTlsEnv still holds stale pointer
  3. CPUThreadPool21 runs next preload task → hdfsGetPathInfo() → libhdfs fast path returns stale env → jni_NewStringUTF(stale_env) → SIGSEGV

Should we cleanup the stale pointer at step2 and re-attach in 3?

It requires clearing quickTlsEnv after DetachCurrentThread. That's impossible from outside libhdfs because quickTlsEnv is a static __thread variable declared inside the body of threadLocalStorageGet() — it has no external symbol, is not exported from libhdfs.so, and no public API exists to zero it. The only code that can write it is libhdfs itself.

Without being able to clear quickTlsEnv, any HDFS call after DetachCurrentThread hits the fast path, sees the non-null but stale pointer, and crashes — the re-attach in step 3 never gets a chance to run.

It's a libhdfs only issue, right? Does the PR work on S3, abfs, gcs?

Yes, hdfs only. Others don't have such problem.

@guowangy
Copy link
Copy Markdown
Author

I revisited the code and ioExecutor_ is global and should not cause leak as you said. It's worth checking spillExecutor_ , whether the spill threads have any callbacks to Gluten JNI code to cause them to be attached. The feature is turned on by setting spark.gluten.sql.columnar.backend.velox.spillThreadNum.

@zhztheplayer Good catch. spillExecutor_ is the real issue — it's created per task, and its threads get attached to the JVM via SparkAllocationListener but never detached, so JavaThread objects accumulate over time.

Proposal Fix: Add a thin folly::ThreadFactory wrapper for spillExecutor_ that attaches threads at pool creation and calls DetachCurrentThread inside the thread body after all work completes. Spill threads never call libhdfs, so this is safe.

Does this approach sound reasonable to you?

@zhztheplayer
Copy link
Copy Markdown
Member

@guowangy

Proposal Fix: Add a thin folly::ThreadFactory wrapper for spillExecutor_ that attaches threads at pool creation and calls DetachCurrentThread inside the thread body after all work completes. Spill threads never call libhdfs, so this is safe.

This sounds reasonable to me.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

[VL] SIGSEGV in IOThreadPool during HDFS scan

3 participants