Skip to content

Commit cda43c3

Browse files
committed
[Enhancement](txn) Add config flag for transaction edit log outside-lock optimization
Add enable_txn_log_outside_lock config (default true) to control whether transaction edit log writes happen inside or outside the write lock. When enabled, edit log entries are enqueued (FIFO) inside the write lock and awaited outside it, preserving ordering via the batch queue while reducing lock hold time. This resolves the potential out-of-order edit log issue from the previous outside-lock optimization. - Add submitEdit() split API to EditLog for enqueue-only writes - Make EditLogItem public with await() method - Add enqueueTransactionState/awaitTransactionState helpers - Update all 8 persist call sites in DatabaseTransactionMgr
1 parent a105828 commit cda43c3

File tree

5 files changed

+200
-23
lines changed

5 files changed

+200
-23
lines changed

fe/fe-common/src/main/java/org/apache/doris/common/Config.java

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -744,6 +744,18 @@ public class Config extends ConfigBase {
744744
"Txn manager will reject coming txns."})
745745
public static int max_running_txn_num_per_db = 10000;
746746

747+
@ConfField(mutable = true, masterOnly = true, description = {
748+
"是否将事务的 edit log 写入移到写锁之外以减少锁竞争。"
749+
+ "开启后,edit log 条目在写锁内入队(FIFO 保证顺序),"
750+
+ "在写锁外等待持久化完成,从而降低写锁持有时间,提高并发事务吞吐量。"
751+
+ "默认开启。关闭后使用传统的锁内同步写入模式。",
752+
"Whether to move transaction edit log writes outside the write lock to reduce lock contention. "
753+
+ "When enabled, edit log entries are enqueued inside the write lock (FIFO preserves ordering) "
754+
+ "and awaited outside the lock, reducing write lock hold time "
755+
+ "and improving concurrent transaction throughput. "
756+
+ "Default is true. Set to false to use the traditional in-lock synchronous write mode."})
757+
public static boolean enable_txn_log_outside_lock = true;
758+
747759
@ConfField(masterOnly = true, description = {"pending load task 执行线程数。这个配置可以限制当前等待的导入作业数。"
748760
+ "并且应小于 `max_running_txn_num_per_db`。",
749761
"The pending load task executor pool size. "

fe/fe-core/src/main/java/org/apache/doris/DorisFE.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -593,6 +593,11 @@ private static void fuzzyConfigs() {
593593
LOG.info("fuzzy set random_add_cluster_keys_for_mow={}", Config.random_add_cluster_keys_for_mow);
594594
}
595595

596+
Config.enable_txn_log_outside_lock = (LocalDate.now().getDayOfMonth() % 2 == 0);
597+
LOG.info("fuzzy set enable_txn_log_outside_lock={}", Config.enable_txn_log_outside_lock);
598+
Config.enable_batch_editlog = (LocalDate.now().getDayOfMonth() % 2 == 1);
599+
LOG.info("fuzzy set enable_batch_editlog={}", Config.enable_batch_editlog);
600+
596601
setFuzzyForCatalog();
597602
}
598603

fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java

Lines changed: 65 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -124,13 +124,14 @@
124124
public class EditLog {
125125
public static final Logger LOG = LogManager.getLogger(EditLog.class);
126126

127-
// Helper class to hold log edit requests
128-
private static class EditLogItem {
127+
// Helper class to hold log edit requests.
128+
// Public so that callers can enqueue inside a lock and await outside it.
129+
public static class EditLogItem {
129130
static AtomicLong nextUid = new AtomicLong(0);
130131
final short op;
131132
final Writable writable;
132133
final Object lock = new Object();
133-
boolean finished = false;
134+
volatile boolean finished = false;
134135
long logId = -1;
135136
long uid = -1;
136137

@@ -139,6 +140,24 @@ private static class EditLogItem {
139140
this.writable = writable;
140141
uid = nextUid.getAndIncrement();
141142
}
143+
144+
/**
145+
* Wait for this edit log entry to be flushed to persistent storage.
146+
* Returns the assigned log ID.
147+
*/
148+
public long await() {
149+
synchronized (lock) {
150+
while (!finished) {
151+
try {
152+
lock.wait();
153+
} catch (InterruptedException e) {
154+
LOG.error("Fatal Error : write stream Exception");
155+
System.exit(-1);
156+
}
157+
}
158+
}
159+
return logId;
160+
}
142161
}
143162

144163
private final BlockingQueue<EditLogItem> logEditQueue = new LinkedBlockingQueue<>();
@@ -1534,6 +1553,49 @@ public long logEditWithQueue(short op, Writable writable) {
15341553
return req.logId;
15351554
}
15361555

1556+
/**
1557+
* Submit an edit log entry to the batch queue without waiting for it to be flushed.
1558+
* The entry is enqueued in FIFO order, so calling this inside a write lock guarantees
1559+
* that edit log entries are ordered by lock acquisition order.
1560+
*
1561+
* <p>The caller MUST call {@link EditLogItem#await()} after releasing the lock to ensure
1562+
* the entry is persisted before proceeding.
1563+
*
1564+
* <p>If batch edit log is disabled, this falls back to a synchronous direct write
1565+
* and the returned item is already completed.
1566+
*
1567+
* @return an {@link EditLogItem} handle to await completion
1568+
*/
1569+
public EditLogItem submitEdit(short op, Writable writable) {
1570+
if (this.getNumEditStreams() == 0) {
1571+
LOG.error("Fatal Error : no editLog stream", new Exception());
1572+
throw new Error("Fatal Error : no editLog stream");
1573+
}
1574+
1575+
EditLogItem req = new EditLogItem(op, writable);
1576+
if (Config.enable_batch_editlog && op != OperationType.OP_TIMESTAMP) {
1577+
while (true) {
1578+
try {
1579+
logEditQueue.put(req);
1580+
break;
1581+
} catch (InterruptedException e) {
1582+
LOG.warn("Interrupted during put, will sleep and retry.");
1583+
try {
1584+
Thread.sleep(100);
1585+
} catch (InterruptedException ex) {
1586+
LOG.warn("interrupted during sleep, will retry.", ex);
1587+
}
1588+
}
1589+
}
1590+
} else {
1591+
// Non-batch mode: write directly (synchronous)
1592+
long logId = logEditDirectly(op, writable);
1593+
req.logId = logId;
1594+
req.finished = true;
1595+
}
1596+
return req;
1597+
}
1598+
15371599
private synchronized long logEditDirectly(short op, Writable writable) {
15381600
long logId = -1;
15391601
try {

0 commit comments

Comments
 (0)