20
20
import com .alibaba .fluss .config .ConfigOptions ;
21
21
import com .alibaba .fluss .config .Configuration ;
22
22
import com .alibaba .fluss .exception .CorruptRecordException ;
23
+ import com .alibaba .fluss .exception .DuplicateSequenceException ;
23
24
import com .alibaba .fluss .exception .FlussRuntimeException ;
24
25
import com .alibaba .fluss .exception .InvalidTimestampException ;
25
26
import com .alibaba .fluss .exception .LogOffsetOutOfRangeException ;
@@ -105,6 +106,9 @@ public final class LogTablet {
105
106
@ GuardedBy ("lock" )
106
107
private volatile LogOffsetMetadata highWatermarkMetadata ;
107
108
109
+ /** The leader end offset snapshot when become leader. */
110
+ private volatile long leaderEndOffsetSnapshot = -1L ;
111
+
108
112
// The minimum offset that should be retained in the local log. This is used to ensure that,
109
113
// the offset of kv snapshot should be retained, otherwise, kv recovery will fail.
110
114
private volatile long minRetainOffset ;
@@ -259,6 +263,10 @@ public LogFormat getLogFormat() {
259
263
return logFormat ;
260
264
}
261
265
266
+ public long getLeaderEndOffsetSnapshot () {
267
+ return leaderEndOffsetSnapshot ;
268
+ }
269
+
262
270
@ VisibleForTesting
263
271
public WriterStateManager writerStateManager () {
264
272
return writerStateManager ;
@@ -336,6 +344,16 @@ public void registerMetrics(BucketMetricGroup bucketMetricGroup) {
336
344
MetricNames .LOG_FLUSH_LATENCY_MS , localLog .getFlushLatencyHistogram ());
337
345
}
338
346
347
+ public void updateLeaderEndOffsetSnapshot () {
348
+ synchronized (lock ) {
349
+ LOG .info (
350
+ "Update leaderEndOffsetSnapshot to {} for tb {} while become leader" ,
351
+ localLogEndOffset (),
352
+ localLog .getTableBucket ());
353
+ leaderEndOffsetSnapshot = localLog .getLocalLogEndOffset ();
354
+ }
355
+ }
356
+
339
357
/**
340
358
* Append this message set to the active segment of the local log, assigning offsets and Bucket
341
359
* Leader Epochs.
@@ -582,10 +600,10 @@ private LogOffsetMetadata convertToOffsetMetadataOrThrow(long offset) throws IOE
582
600
* segment if necessary.
583
601
*
584
602
* <p>This method will generally be responsible for assigning offsets to the messages, however
585
- * if the needAssignOffsetAndTimestamp =false flag is passed we will only check that the existing
586
- * offsets are valid.
603
+ * if the appendAsLeader =false flag is passed we will only check that the existing offsets are
604
+ * valid.
587
605
*/
588
- private LogAppendInfo append (MemoryLogRecords records , boolean needAssignOffsetAndTimestamp )
606
+ private LogAppendInfo append (MemoryLogRecords records , boolean appendAsLeader )
589
607
throws Exception {
590
608
LogAppendInfo appendInfo = analyzeAndValidateRecords (records );
591
609
@@ -599,7 +617,7 @@ private LogAppendInfo append(MemoryLogRecords records, boolean needAssignOffsetA
599
617
600
618
synchronized (lock ) {
601
619
localLog .checkIfMemoryMappedBufferClosed ();
602
- if (needAssignOffsetAndTimestamp ) {
620
+ if (appendAsLeader ) {
603
621
long offset = localLog .getLocalLogEndOffset ();
604
622
// assign offsets to the message set.
605
623
appendInfo .setFirstOffset (offset );
@@ -630,11 +648,24 @@ private LogAppendInfo append(MemoryLogRecords records, boolean needAssignOffsetA
630
648
// have duplicated batch metadata, skip the append and update append info.
631
649
WriterStateEntry .BatchMetadata duplicatedBatch = validateResult .left ();
632
650
long startOffset = duplicatedBatch .firstOffset ();
633
- appendInfo .setFirstOffset (startOffset );
634
- appendInfo .setLastOffset (duplicatedBatch .lastOffset );
635
- appendInfo .setMaxTimestamp (duplicatedBatch .timestamp );
636
- appendInfo .setStartOffsetOfMaxTimestamp (startOffset );
637
- appendInfo .setDuplicated (true );
651
+ if (appendAsLeader ) {
652
+ appendInfo .setFirstOffset (startOffset );
653
+ appendInfo .setLastOffset (duplicatedBatch .lastOffset );
654
+ appendInfo .setMaxTimestamp (duplicatedBatch .timestamp );
655
+ appendInfo .setStartOffsetOfMaxTimestamp (startOffset );
656
+ appendInfo .setDuplicated (true );
657
+ } else {
658
+ String errorMsg =
659
+ String .format (
660
+ "Found duplicated batch for table bucket %s, duplicated offset is %s, "
661
+ + "writer id is %s and batch sequence is: %s" ,
662
+ getTableBucket (),
663
+ duplicatedBatch .lastOffset ,
664
+ duplicatedBatch .writerId ,
665
+ duplicatedBatch .batchSequence );
666
+ LOG .error (errorMsg );
667
+ throw new DuplicateSequenceException (errorMsg );
668
+ }
638
669
} else {
639
670
// Append the records, and increment the local log end offset immediately after
640
671
// append because write to the transaction index below may fail, and we want to
0 commit comments