Skip to content

[fix][io] JDBC sink: prevent OOM from unbounded queue on connection failure#9

Merged
david-streamlio merged 2 commits intoapache:masterfrom
harangozop:fix/jdbc-sink-oom-unbounded-queue
Apr 3, 2026
Merged

[fix][io] JDBC sink: prevent OOM from unbounded queue on connection failure#9
david-streamlio merged 2 commits intoapache:masterfrom
harangozop:fix/jdbc-sink-oom-unbounded-queue

Conversation

@harangozop
Copy link
Copy Markdown
Contributor

@harangozop harangozop commented Mar 30, 2026

Motivation

Fixes apache/pulsar#25030

The JDBC sink's internal queue (incomingList) is an unbounded LinkedList. When the database connection drops:

  1. executeBatch() hangs until TCP socket timeout (can be minutes)
  2. isFlushing stays true, blocking all queue draining
  3. write() keeps accepting records without any limit
  4. incomingList grows until OutOfMemoryError: Java heap space

This is a production issue observed with both PostgreSQL and MariaDB JDBC sinks under Pulsar FunctionMesh.

Modifications

JdbcAbstractSink.java:

  • Bounded queue: write() rejects records when queue exceeds maxQueueSize, applying Pulsar-level back-pressure via record.fail() (negative ack → consumer redelivers later)
  • State check in write(): records are failed immediately when state != OPEN (after fatal() or close())
  • Connection validation & reconnection: ensureConnection() validates the JDBC connection via Connection.isValid() before each flush and reconnects automatically, allowing recovery from transient DB outages without pod restart
  • Scheduled flush cancellation: fatal() and close() cancel the periodic flush ScheduledFuture to stop repeated failures on a broken connection

JdbcSinkConfig.java:

  • Added maxQueueSize config (default: 0 = auto, which resolves to batchSize * 10). Users can override for their workload.

Verifying this change

  • Added testWriteRejectsRecordsAfterFatal — verifies write() fails records when state is FAILED
  • Added testBoundedQueueBackPressure — verifies records are failed when internal queue is full
  • All existing SQLite sink tests pass (including batch variant)

Does this pull request potentially affect one of the following parts of the Pulsar project?

  • Dependencies (add or upgrade a dependency)
  • The public API
  • The schema
  • The default values of configurations
  • The threading model
  • The binary protocol
  • The REST endpoints
  • The admin CLI options
  • The metrics
  • Anything that affects deployment (configuration default changes are backward compatible — maxQueueSize=0 preserves legacy unbounded behavior)

Documentation

  • doc-not-needed — Bug fix with backward-compatible config addition

Copy link
Copy Markdown

Copilot AI left a comment

Choose a reason for hiding this comment

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

Pull request overview

This PR addresses a production OOM scenario in the Pulsar JDBC sink when the database connection is slow/broken by adding queue back-pressure, improving sink state handling, and adding connection validation/reconnect behavior.

Changes:

  • Add maxQueueSize to bound the internal record queue (with auto-sizing behavior) and reject records when full.
  • Fail records immediately when the sink is not OPEN, and cancel periodic flush tasks on fatal() / close().
  • Add ensureConnection() to validate the JDBC connection before flushing and reconnect when invalid; add SQLite tests for the new behaviors.

Reviewed changes

Copilot reviewed 3 out of 3 changed files in this pull request and generated 5 comments.

File Description
jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/JdbcAbstractSink.java Adds bounded-queue back-pressure, state checks in write(), scheduled flush cancellation, and connection validation/reconnect in flush().
jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/JdbcSinkConfig.java Introduces maxQueueSize configuration and its FieldDoc.
jdbc/sqlite/src/test/java/org/apache/pulsar/io/jdbc/SqliteJdbcSinkTest.java Adds tests covering failing writes after fatal, bounded-queue back-pressure, and reconnection on invalid connection.
Comments suppressed due to low confidence (1)

jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/JdbcAbstractSink.java:233

  • There is a race between the initial state check and enqueuing/scheduling: close()/fatal() can flip state to CLOSED/FAILED and shut down the flushExecutor after the check at line 208, causing write() to enqueue a record and then throw RejectedExecutionException when scheduling flush (or leave a queued record that will never be acked/failed because flush() returns when CLOSED). Re-check state (and executor availability) under the same synchronization used for enqueuing, and handle schedule rejection by failing the record (or flushing synchronously).
    public void write(Record<T> record) throws Exception {
        if (state.get() != State.OPEN) {
            log.warn("Sink is not in OPEN state (current: {}), failing record", state.get());
            record.fail();
            return;
        }
        int number;
        synchronized (incomingList) {
            if (maxQueueSize > 0 && incomingList.size() >= maxQueueSize) {
                if (!queueFullLogged) {
                    log.warn("Internal queue is full ({} >= {}), failing records to apply back-pressure",
                            incomingList.size(), maxQueueSize);
                    queueFullLogged = true;
                }
                record.fail();
                return;
            }
            incomingList.add(record);
            number = incomingList.size();
        }
        if (batchSize > 0 && number >= batchSize) {
            if (log.isDebugEnabled()) {
                log.debug("flushing by batches, hit batch size {}", batchSize);
            }
            flushExecutor.schedule(this::flush, 0, TimeUnit.MILLISECONDS);
        }
    }

💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.

…ailure

The JDBC sink's internal queue (incomingList) is unbounded. When the
database connection drops, executeBatch() hangs until the TCP socket
times out. During this period, isFlushing stays true, preventing any
draining, while write() continues accepting records without limit.
This causes OutOfMemoryError in production.

This commit fixes 4 issues:

1. Bounded internal queue: write() now rejects records when queue
   exceeds maxQueueSize (configurable, defaults to 10x batchSize),
   applying Pulsar-level back-pressure via negative acknowledgment.

2. State check in write(): records are failed immediately when the
   sink state is not OPEN (after fatal() or close()).

3. Connection validation and reconnection: ensureConnection() validates
   the JDBC connection before each flush and reconnects automatically
   on failure, allowing recovery from transient database outages.

4. Scheduled flush cancellation: fatal() and close() now cancel the
   periodic flush task to prevent repeated failures on a broken
   connection.

Fixes apache/pulsar#25030
- Move record.fail() outside synchronized(incomingList) to avoid
  holding the lock during framework callbacks
- Move incomingList.size() check inside synchronized in flush() to
  fix data race on non-thread-safe LinkedList
- Change maxQueueSize default from 0 (auto-bounded) to -1 (unbounded)
  to preserve backwards-compatible legacy behavior; users opt-in to
  bounded queue by setting maxQueueSize=0 (auto) or a positive value
- Add overflow-safe auto-sizing (long arithmetic capped at MAX_VALUE)
- Validate maxQueueSize in JdbcSinkConfig.validate() — reject < -1
- Add test for invalid maxQueueSize rejection
@harangozop harangozop force-pushed the fix/jdbc-sink-oom-unbounded-queue branch from fb9f552 to c141a2d Compare April 3, 2026 06:20
@david-streamlio david-streamlio merged commit e689bc3 into apache:master Apr 3, 2026
2 checks passed
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

[Bug][io] JdbcSink: potential memory leak/OOM due to lack of backpressure in internal buffering

3 participants