nzw921rx opened a new issue, #10607:
URL: https://github.com/apache/seatunnel/issues/10607

   ### Search before asking
   
   - [x] I had searched in the 
[feature](https://github.com/apache/seatunnel/issues?q=is%3Aissue+label%3A%22Feature%22)
 and found no similar feature requirement.
   
   
   ### Description
   
   Currently, JDBC sink only supports batch-size-based flushing. Data is 
flushed to the database only when the batch reaches `batch_size`, or during 
`prepareCommit` / `close`.
   This causes a problem in low-throughput scenarios (e.g., CDC real-time sync 
with sparse updates): if data comes slowly and the batch never fills up, 
records stay in memory for a long time before being written to the target 
database, leading to high data latency.
   Apache Flink's JDBC connector already supports this feature with 
`sink.buffer-flush.interval` (default 1s). SeaTunnel's `JdbcOutputFormat` was 
originally derived from Flink's design but is missing this time-based flush 
capability.
   ### Proposed Solution
   Add a `batch_interval_ms` option to the JDBC sink connector.
   **Config example:**
   ```hocon
   sink {
     jdbc {
       url = "jdbc:mysql://localhost:3306/test"
       driver = "com.mysql.cj.jdbc.Driver"
       batch_size = 1000
       batch_interval_ms = 1000  # flush every 1 second
     }
   }
   ```
   **Implementation approach:**
   In `JdbcOutputFormat.open()`, start a `ScheduledExecutorService` that 
periodically calls `flush()`:
   ```java
   if (batchIntervalMs > 0 && batchSize != 1) {
       this.scheduler = Executors.newScheduledThreadPool(1);
       this.scheduledFuture = scheduler.scheduleWithFixedDelay(
           () -> {
               synchronized (JdbcOutputFormat.this) {
                   if (!closed) {
                       try {
                           flush();
                       } catch (Exception e) {
                           flushException = e;
                       }
                   }
               }
           },
           batchIntervalMs, batchIntervalMs, TimeUnit.MILLISECONDS);
   }
   ```
   Shut down the scheduler in `close()` before flushing remaining data.
   **Key design points:**
   - Default value: `0` (disabled, preserving current behavior for backward 
compatibility)
   - Thread safety: use `synchronized` on the existing `writeRecord` / `flush` 
lock
   - Exception handling: timer thread stores exception in `flushException`, 
checked on next `writeRecord` call (same pattern as Flink)
   - Shutdown order: cancel scheduler → flush remaining → close connection
   **Changes needed:**
   1. `JdbcSinkOptions`: add `BATCH_INTERVAL_MS` option
   2. `JdbcConnectionConfig`: add `batchIntervalMs` field
   3. `JdbcOutputFormat`: add scheduler lifecycle (open/close) and periodic 
flush logic
   4. Documentation: update JDBC sink docs (en & zh)
   5. Tests: unit test for config parsing + E2E test verifying flush behavior
   ### Use Cases
   - CDC real-time sync with low write frequency
   - Any scenario where data latency matters more than throughput
   - Align with Flink JDBC connector behavior for users migrating from Flink
   
   ### Usage Scenario
   
   _No response_
   
   ### Related issues
   
   _No response_
   
   ### Are you willing to submit a PR?
   
   - [x] Yes I am willing to submit a PR!
   
   ### Code of Conduct
   
   - [x] I agree to follow this project's [Code of 
Conduct](https://www.apache.org/foundation/policies/conduct)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to