forked from inyo-global/ducklake-kafka-connect
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathDucklakeSinkTask.java
More file actions
1251 lines (1133 loc) · 44.9 KB
/
DucklakeSinkTask.java
File metadata and controls
1251 lines (1133 loc) · 44.9 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* Copyright 2025 Inyo Contributors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.inyo.ducklake.connect;
import com.inyo.ducklake.ingestor.DucklakeWriter;
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.ReentrantLock;
import java.util.stream.Stream;
import org.apache.arrow.memory.BufferAllocator;
import org.apache.arrow.memory.RootAllocator;
import org.apache.arrow.vector.VectorSchemaRoot;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.sink.ErrantRecordReporter;
import org.apache.kafka.connect.sink.SinkRecord;
import org.apache.kafka.connect.sink.SinkTask;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class DucklakeSinkTask extends SinkTask {
private static final Logger LOG = LoggerFactory.getLogger(DucklakeSinkTask.class);
private DucklakeSinkConfig config;
private DucklakeConnectionFactory connectionFactory;
private DucklakeWriterFactory writerFactory;
private Map<TopicPartition, DucklakeWriter> writers;
private BufferAllocator allocator;
private SinkRecordToArrowConverter converter;
// Buffering state
private Map<TopicPartition, PartitionBuffer> buffers;
private Map<TopicPartition, SpillablePartitionBuffer> spillableBuffers;
private int flushSize;
private long flushIntervalMs;
private long fileSizeBytes;
private ScheduledExecutorService flushScheduler;
private ExecutorService partitionExecutor;
private boolean parallelPartitionFlush;
private long memoryPressureBytes;
// Spill configuration
private boolean spillEnabled;
private Path spillDirectory;
// Per-partition locks for buffer operations (fast operations like adding records)
private final ConcurrentHashMap<TopicPartition, ReentrantLock> partitionLocks =
new ConcurrentHashMap<>();
// Per-partition locks for flush operations (slow I/O - only one flush at a time per partition)
private final ConcurrentHashMap<TopicPartition, ReentrantLock> flushLocks =
new ConcurrentHashMap<>();
// Track skipped time-based flushes per partition
private final ConcurrentHashMap<TopicPartition, AtomicInteger> consecutiveFlushSkips =
new ConcurrentHashMap<>();
private static final int MAX_CONSECUTIVE_SKIPS_BEFORE_WARNING = 5;
private static final long FLUSH_LOCK_TIMEOUT_MS = 100;
// Errant record reporter for sending bad records to DLQ
private ErrantRecordReporter errantRecordReporter;
private Metrics metricsRegistry;
// Initialized to no-op so stop() can call close() unconditionally even if start() fails
private DucklakeMetricsInterface ducklakeMetrics = NoopDucklakeMetrics.INSTANCE;
// Jitter configuration to stagger flushes and avoid PG row-level contention
// Each partition gets a random jitter offset (0 to maxJitterMs) applied to its flush timing
private static final double JITTER_FACTOR = 0.25; // 25% of flush interval as max jitter
private final ConcurrentHashMap<TopicPartition, Long> partitionJitterMs =
new ConcurrentHashMap<>();
/** Per-partition buffer to accumulate records before writing */
private static class PartitionBuffer {
final List<VectorSchemaRoot> pendingBatches = new ArrayList<>();
long recordCount = 0;
long estimatedBytes = 0;
long lastFlushTime = System.currentTimeMillis();
void add(VectorSchemaRoot root) {
pendingBatches.add(root);
recordCount += root.getRowCount();
// Estimate bytes from Arrow buffer sizes
estimatedBytes += root.getFieldVectors().stream().mapToLong(v -> v.getBufferSize()).sum();
}
void clear() {
// Close all VectorSchemaRoot to free memory
for (VectorSchemaRoot root : pendingBatches) {
try {
root.close();
} catch (Exception e) {
// Ignore close errors during clear
}
}
pendingBatches.clear();
recordCount = 0;
estimatedBytes = 0;
lastFlushTime = System.currentTimeMillis();
}
boolean shouldFlush(int flushSize, long fileSizeBytes, long flushIntervalMs) {
if (pendingBatches.isEmpty()) {
return false;
}
// Flush if any threshold is exceeded
if (recordCount >= flushSize) {
return true;
}
if (estimatedBytes >= fileSizeBytes) {
return true;
}
if (System.currentTimeMillis() - lastFlushTime >= flushIntervalMs) {
return true;
}
return false;
}
}
@Override
public String version() {
return DucklakeSinkConfig.VERSION;
}
@Override
public void start(Map<String, String> map) {
this.config = new DucklakeSinkConfig(DucklakeSinkConfig.CONFIG_DEF, map);
String connectorName = map.getOrDefault("name", "ducklake-sink");
String taskId = map.getOrDefault("task.id", "0");
this.metricsRegistry = DucklakeMetrics.createMetricsRegistry();
this.ducklakeMetrics = new DucklakeMetrics(metricsRegistry, connectorName, taskId);
this.connectionFactory = new DucklakeConnectionFactory(config);
this.writers = new HashMap<>();
this.buffers = new HashMap<>();
this.spillableBuffers = new HashMap<>();
this.allocator = new RootAllocator();
this.converter = new SinkRecordToArrowConverter(allocator);
// Initialize buffering configuration
this.flushSize = config.getFlushSize();
this.flushIntervalMs = config.getFlushIntervalMs();
this.fileSizeBytes = config.getFileSizeBytes();
this.parallelPartitionFlush = config.isParallelPartitionFlushEnabled();
this.memoryPressureBytes = config.getMemoryPressurePerPartitionBytes();
// Initialize spill configuration
this.spillEnabled = config.isSpillEnabled();
if (spillEnabled) {
String spillDir = config.getSpillDirectory();
if (spillDir == null || spillDir.isEmpty()) {
// Use a fixed directory path in system temp instead of creating new temp dirs each time.
// This prevents orphaned directories from accumulating across task restarts.
Path tempDir = Path.of(System.getProperty("java.io.tmpdir"));
this.spillDirectory = tempDir.resolve("ducklake-spill");
// Clean up any orphaned ducklake-spill-* directories from previous task instances
// that used Files.createTempDirectory() (old behavior)
cleanupOrphanedSpillDirectories(tempDir);
} else {
this.spillDirectory = Path.of(spillDir);
}
// Ensure the spill directory exists
try {
Files.createDirectories(spillDirectory);
} catch (IOException e) {
throw new RuntimeException("Failed to create spill directory: " + spillDirectory, e);
}
LOG.info("Spill enabled, using directory: {}", spillDirectory);
}
int threadCount = config.getDuckDbThreads();
LOG.info(
"Buffering config: flushSize={}, flushIntervalMs={}, fileSizeBytes={}, "
+ "parallelPartitionFlush={}, duckdbThreads={}, memoryPressureBytes={}, spillEnabled={}",
flushSize,
flushIntervalMs,
fileSizeBytes,
parallelPartitionFlush,
threadCount,
memoryPressureBytes,
spillEnabled);
// Create executor for parallel partition processing
if (parallelPartitionFlush) {
this.partitionExecutor =
Executors.newFixedThreadPool(
Math.max(4, threadCount),
r -> {
Thread t = new Thread(r, "ducklake-partition-worker");
t.setDaemon(true);
return t;
});
}
// Start scheduled flush checker (runs every 1 second to check time-based flush)
this.flushScheduler =
Executors.newSingleThreadScheduledExecutor(
r -> {
Thread t = new Thread(r, "ducklake-flush-scheduler");
t.setDaemon(true);
return t;
});
flushScheduler.scheduleAtFixedRate(this::checkTimeBasedFlush, 1, 1, TimeUnit.SECONDS);
// Initialize errant record reporter for DLQ support (may be null if not configured)
try {
this.errantRecordReporter = context.errantRecordReporter();
if (errantRecordReporter != null) {
LOG.info("Errant record reporter initialized - bad records will be sent to DLQ");
} else {
LOG.info("No errant record reporter available - bad records will cause task failure");
}
} catch (NoSuchMethodError | NoClassDefFoundError e) {
// Older Kafka Connect versions may not have errantRecordReporter()
LOG.warn("ErrantRecordReporter not available in this Kafka Connect version");
this.errantRecordReporter = null;
}
}
/**
* Gets or assigns a random jitter offset for a partition. The jitter is assigned once per
* partition and remains stable to ensure consistent flush timing. This staggering prevents
* multiple partitions from flushing simultaneously, reducing PostgreSQL row-level contention.
*/
private long getPartitionJitter(TopicPartition partition) {
return partitionJitterMs.computeIfAbsent(
partition,
p -> {
long maxJitter = (long) (flushIntervalMs * JITTER_FACTOR);
long jitter = ThreadLocalRandom.current().nextLong(maxJitter + 1);
LOG.debug(
"Assigned flush jitter {}ms to partition {} (maxJitter={}ms)",
jitter,
partition,
maxJitter);
return jitter;
});
}
/** Periodic check for time-based flush - uses per-partition locking with jitter */
private void checkTimeBasedFlush() {
long now = System.currentTimeMillis();
if (spillEnabled) {
checkTimeBasedFlushSpillable(now);
} else {
checkTimeBasedFlushInMemory(now);
}
}
/** Time-based flush check for spillable buffers */
private void checkTimeBasedFlushSpillable(long now) {
for (Map.Entry<TopicPartition, SpillablePartitionBuffer> entry : spillableBuffers.entrySet()) {
TopicPartition partition = entry.getKey();
SpillablePartitionBuffer buffer = entry.getValue();
// Get or assign jitter for this partition
long jitter = getPartitionJitter(partition);
long effectiveFlushInterval = flushIntervalMs + jitter;
// Skip if buffer is empty or not due for flush
if (buffer.isEmpty() || (now - buffer.getLastFlushTime()) < effectiveFlushInterval) {
continue;
}
// Try to acquire per-partition lock
ReentrantLock lock = partitionLocks.computeIfAbsent(partition, k -> new ReentrantLock());
boolean lockAcquired = false;
try {
lockAcquired = lock.tryLock(FLUSH_LOCK_TIMEOUT_MS, TimeUnit.MILLISECONDS);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
return;
}
if (!lockAcquired) {
AtomicInteger skips =
consecutiveFlushSkips.computeIfAbsent(partition, k -> new AtomicInteger(0));
int skipCount = skips.incrementAndGet();
ducklakeMetrics.recordFlushSkip();
if (skipCount >= MAX_CONSECUTIVE_SKIPS_BEFORE_WARNING) {
LOG.warn(
"Flush check for partition {} skipped {} times - possible lock contention",
partition,
skipCount);
}
continue;
}
try {
consecutiveFlushSkips.computeIfAbsent(partition, k -> new AtomicInteger(0)).set(0);
// Re-check condition under lock
if (!buffer.isEmpty() && (now - buffer.getLastFlushTime()) >= effectiveFlushInterval) {
LOG.info(
"Time-based flush triggered for partition {} (age={}ms, jitter={}ms, records={}, bytes={})",
partition,
now - buffer.getLastFlushTime(),
jitter,
buffer.getRecordCount(),
buffer.getEstimatedBytes());
// Read batches from disk and flush
List<VectorSchemaRoot> batches = buffer.readBatches(allocator);
FlushData flushData =
new FlushData(batches, buffer.getRecordCount(), buffer.getEstimatedBytes());
buffer.clear();
// Release lock before slow I/O
lock.unlock();
lockAcquired = false;
try {
flushBatches(partition, flushData);
} catch (Exception e) {
LOG.warn(
"Error during time-based flush for partition {}: {}", partition, e.getMessage());
}
}
} finally {
if (lockAcquired) {
lock.unlock();
}
}
}
}
/** Time-based flush check for in-memory buffers */
private void checkTimeBasedFlushInMemory(long now) {
for (Map.Entry<TopicPartition, PartitionBuffer> entry : buffers.entrySet()) {
TopicPartition partition = entry.getKey();
PartitionBuffer buffer = entry.getValue();
// Get or assign jitter for this partition (stagger commits to avoid PG contention)
long jitter = getPartitionJitter(partition);
long effectiveFlushInterval = flushIntervalMs + jitter;
// Skip if buffer is empty or not due for flush (accounting for jitter)
if (buffer.pendingBatches.isEmpty()
|| (now - buffer.lastFlushTime) < effectiveFlushInterval) {
continue;
}
// Try to acquire per-partition lock
ReentrantLock lock = partitionLocks.computeIfAbsent(partition, k -> new ReentrantLock());
boolean lockAcquired = false;
try {
lockAcquired = lock.tryLock(FLUSH_LOCK_TIMEOUT_MS, TimeUnit.MILLISECONDS);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
return;
}
if (!lockAcquired) {
AtomicInteger skips =
consecutiveFlushSkips.computeIfAbsent(partition, k -> new AtomicInteger(0));
int skipCount = skips.incrementAndGet();
ducklakeMetrics.recordFlushSkip();
if (skipCount >= MAX_CONSECUTIVE_SKIPS_BEFORE_WARNING) {
LOG.warn(
"Flush check for partition {} skipped {} times - possible lock contention",
partition,
skipCount);
}
continue;
}
// Phase 1: Extract flush data under lock
FlushData flushData = null;
try {
// Reset skip counter on successful lock acquisition
consecutiveFlushSkips.computeIfAbsent(partition, k -> new AtomicInteger(0)).set(0);
// Re-check condition under lock (double-check pattern)
if (!buffer.pendingBatches.isEmpty()
&& (now - buffer.lastFlushTime) >= effectiveFlushInterval) {
LOG.info(
"Time-based flush triggered for partition {} (age={}ms, jitter={}ms, records={},"
+ " bytes={})",
partition,
now - buffer.lastFlushTime,
jitter,
buffer.recordCount,
buffer.estimatedBytes);
flushData = extractFlushDataFromBuffer(buffer);
}
} finally {
lock.unlock();
}
// Phase 2: Perform slow I/O flush OUTSIDE the lock
if (flushData != null) {
try {
flushBatches(partition, flushData);
} catch (Exception e) {
LOG.warn("Error during time-based flush for partition {}: {}", partition, e.getMessage());
}
}
}
}
@Override
@SuppressFBWarnings(
value = "UWF_FIELD_NOT_INITIALIZED_IN_CONSTRUCTOR",
justification = "Values are initialized in start() method")
public void open(Collection<TopicPartition> partitions) {
super.open(partitions);
try {
this.connectionFactory.create();
this.writerFactory =
new DucklakeWriterFactory(config, connectionFactory.getConnection(), ducklakeMetrics);
// Create one writer and buffer for each partition
for (TopicPartition partition : partitions) {
DucklakeWriter writer = writerFactory.create(partition.topic());
writers.put(partition, writer);
if (spillEnabled) {
// Create spillable buffer with partition-specific subdirectory
Path partitionSpillDir =
spillDirectory.resolve(partition.topic() + "-" + partition.partition());
// Clean up any orphaned spill files from a previous task instance that may have
// crashed or been terminated without proper cleanup during rebalancing
if (Files.exists(partitionSpillDir)) {
LOG.info(
"Cleaning up existing partition spill directory before use: {}", partitionSpillDir);
try {
deleteDirectoryRecursively(partitionSpillDir);
} catch (IOException e) {
LOG.warn(
"Failed to clean up partition spill directory {}: {}",
partitionSpillDir,
e.getMessage());
}
}
spillableBuffers.put(
partition, new SpillablePartitionBuffer(partitionSpillDir, ducklakeMetrics));
LOG.info("Created writer and spillable buffer for partition: {}", partition);
} else {
buffers.put(partition, new PartitionBuffer());
LOG.info("Created writer and in-memory buffer for partition: {}", partition);
}
}
} catch (SQLException e) {
throw new RuntimeException("Failed to create writers for partitions", e);
}
}
@Override
@SuppressFBWarnings(
value = "UWF_FIELD_NOT_INITIALIZED_IN_CONSTRUCTOR",
justification = "Values are initialized in start() method")
public void put(Collection<SinkRecord> records) {
if (records == null || records.isEmpty()) {
return;
}
// Group records by partition first (no lock needed)
Map<TopicPartition, List<SinkRecord>> recordsByPartition =
SinkRecordToArrowConverter.groupRecordsByPartition(records);
if (parallelPartitionFlush && recordsByPartition.size() > 1) {
// Process partitions in parallel for better throughput
processPartitionsInParallel(recordsByPartition);
} else {
// Sequential processing for single partition or when parallel is disabled
for (Map.Entry<TopicPartition, List<SinkRecord>> entry : recordsByPartition.entrySet()) {
processPartition(entry.getKey(), entry.getValue());
}
}
}
/** Process partitions in parallel using the partition executor. */
private void processPartitionsInParallel(
Map<TopicPartition, List<SinkRecord>> recordsByPartition) {
List<CompletableFuture<Void>> futures = new ArrayList<>(recordsByPartition.size());
for (Map.Entry<TopicPartition, List<SinkRecord>> entry : recordsByPartition.entrySet()) {
TopicPartition partition = entry.getKey();
List<SinkRecord> partitionRecords = entry.getValue();
CompletableFuture<Void> future =
CompletableFuture.runAsync(
() -> processPartition(partition, partitionRecords), partitionExecutor);
futures.add(future);
}
// Wait for all partitions to complete and collect any errors
try {
CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).join();
} catch (Exception e) {
LOG.error("Error during parallel partition processing", e);
throw new RuntimeException("Failed to process partitions in parallel", e);
}
}
/** Data extracted from buffer for flushing outside the lock. */
private static class FlushData {
final List<VectorSchemaRoot> batches;
final long recordCount;
final long estimatedBytes;
FlushData(List<VectorSchemaRoot> batches, long recordCount, long estimatedBytes) {
this.batches = batches;
this.recordCount = recordCount;
this.estimatedBytes = estimatedBytes;
}
}
/** Process a single partition with proper locking. */
private void processPartition(TopicPartition partition, List<SinkRecord> partitionRecords) {
if (spillEnabled) {
processPartitionSpillable(partition, partitionRecords);
} else {
processPartitionInMemory(partition, partitionRecords);
}
}
/** Process a single partition using spillable buffer. */
private void processPartitionSpillable(
TopicPartition partition, List<SinkRecord> partitionRecords) {
ReentrantLock lock = partitionLocks.computeIfAbsent(partition, k -> new ReentrantLock());
SpillablePartitionBuffer buffer = spillableBuffers.get(partition);
if (buffer == null) {
LOG.warn("No spillable buffer found for partition: {}", partition);
return;
}
FlushData flushData = null;
lock.lock();
try {
// Convert records to Arrow and spill to disk
boolean hasArrowIpcData =
partitionRecords.stream().anyMatch(record -> record.value() instanceof VectorSchemaRoot);
if (hasArrowIpcData) {
// Arrow IPC data - spill each VectorSchemaRoot directly
for (SinkRecord record : partitionRecords) {
if (record.value() instanceof VectorSchemaRoot root) {
buffer.add(root);
}
}
} else {
// Traditional data - convert to Arrow first, then spill
Map<TopicPartition, List<SinkRecord>> singlePartitionMap = new HashMap<>();
singlePartitionMap.put(partition, partitionRecords);
try {
Map<TopicPartition, VectorSchemaRoot> vectorsByPartition =
converter.convertRecordsByPartition(singlePartitionMap);
VectorSchemaRoot root = vectorsByPartition.get(partition);
if (root != null) {
buffer.add(root); // This spills to disk and closes the root
}
} catch (RuntimeException e) {
if (isSchemaConflictError(e)) {
LOG.warn(
"Schema conflict in spillable mode for partition {}: {}",
partition,
e.getMessage());
// In spillable mode, we can't easily do per-record DLQ handling
// because we need to write to disk. For now, just throw.
throw e;
}
throw e;
}
}
// Check if flush needed
if (buffer.shouldFlush(flushSize, fileSizeBytes, flushIntervalMs)) {
String reason;
if (buffer.getRecordCount() >= flushSize) {
reason = "record count";
} else if (buffer.getEstimatedBytes() >= fileSizeBytes) {
reason = "file size";
} else {
reason = "time interval";
}
LOG.info(
"Flush triggered for partition {} (reason={}, records={}, bytes={})",
partition,
reason,
buffer.getRecordCount(),
buffer.getEstimatedBytes());
// Read batches back from disk
List<VectorSchemaRoot> batches = buffer.readBatches(allocator);
flushData = new FlushData(batches, buffer.getRecordCount(), buffer.getEstimatedBytes());
buffer.clear();
}
} catch (Exception e) {
LOG.error("Error processing records for partition {}", partition, e);
throw new RuntimeException("Failed to process sink records", e);
} finally {
lock.unlock();
}
// Phase 2: Perform slow I/O flush OUTSIDE the lock
if (flushData != null) {
flushBatches(partition, flushData);
}
}
/** Process a single partition using in-memory buffer. */
private void processPartitionInMemory(
TopicPartition partition, List<SinkRecord> partitionRecords) {
ReentrantLock lock = partitionLocks.computeIfAbsent(partition, k -> new ReentrantLock());
// Phase 1: Buffer records and check if flush needed (under lock)
FlushData flushData = null;
lock.lock();
try {
// Detect if we have Arrow IPC data (VectorSchemaRoot) or traditional data
boolean hasArrowIpcData =
partitionRecords.stream().anyMatch(record -> record.value() instanceof VectorSchemaRoot);
if (hasArrowIpcData) {
bufferArrowIpcRecordsForPartition(partition, partitionRecords);
} else {
bufferTraditionalRecordsForPartition(partition, partitionRecords);
}
// Check if this partition needs flushing and extract data if so
flushData = checkAndExtractFlushData(partition);
} catch (Exception e) {
// Build a concise metadata sample for easier debugging (topic:partition@offset)
var sb = new StringBuilder();
sb.append("Error processing records for partition ")
.append(partition)
.append(". batchSize=")
.append(partitionRecords.size())
.append(". sampleOffsets=");
var idx = 0;
for (SinkRecord r : partitionRecords) {
var offset = String.valueOf(r.kafkaOffset());
sb.append("[")
.append(r.topic())
.append(":")
.append(r.kafkaPartition())
.append("@")
.append(offset)
.append("]");
if (++idx >= 10) {
sb.append("(truncated)");
break;
} else {
sb.append(',');
}
}
LOG.error(sb.toString(), e);
throw new RuntimeException("Failed to process sink records", e);
} finally {
lock.unlock();
}
// Phase 2: Perform slow I/O flush OUTSIDE the lock
if (flushData != null) {
flushBatches(partition, flushData);
}
}
/**
* Check if flush is needed and extract data for flushing (called under partition lock). Returns
* FlushData if flush is needed, null otherwise. This method extracts the batches from the buffer
* and resets the buffer state, allowing the actual I/O to happen outside the lock.
*/
private FlushData checkAndExtractFlushData(TopicPartition partition) {
PartitionBuffer buffer = buffers.get(partition);
if (buffer == null || buffer.pendingBatches.isEmpty()) {
return null;
}
// Check for per-partition memory pressure (0 = disabled)
boolean memoryPressure = memoryPressureBytes > 0 && buffer.estimatedBytes > memoryPressureBytes;
if (memoryPressure) {
LOG.warn(
"Memory pressure detected for partition {}: bufferBytes={}, threshold={}",
partition,
buffer.estimatedBytes,
memoryPressureBytes);
}
// Flush if normal thresholds exceeded OR if under memory pressure with data buffered
boolean shouldFlush =
buffer.shouldFlush(flushSize, fileSizeBytes, flushIntervalMs) || memoryPressure;
if (!shouldFlush) {
return null;
}
String reason;
if (memoryPressure && !buffer.shouldFlush(flushSize, fileSizeBytes, flushIntervalMs)) {
reason = "memory pressure";
} else if (buffer.recordCount >= flushSize) {
reason = "record count";
} else if (buffer.estimatedBytes >= fileSizeBytes) {
reason = "file size";
} else {
reason = "time interval";
}
LOG.info(
"Flush triggered for partition {} (reason={}, records={}, bytes={})",
partition,
reason,
buffer.recordCount,
buffer.estimatedBytes);
// Extract batches and reset buffer state (still under lock)
return extractFlushDataFromBuffer(buffer);
}
/**
* Extract flush data from buffer and reset buffer state. Must be called under partition lock.
* Returns the extracted FlushData containing batches to write.
*/
private FlushData extractFlushDataFromBuffer(PartitionBuffer buffer) {
// Move batches out of buffer
List<VectorSchemaRoot> batches = new ArrayList<>(buffer.pendingBatches);
long recordCount = buffer.recordCount;
long estimatedBytes = buffer.estimatedBytes;
// Clear buffer state (but don't close the batches - they'll be closed after flush)
buffer.pendingBatches.clear();
buffer.recordCount = 0;
buffer.estimatedBytes = 0;
buffer.lastFlushTime = System.currentTimeMillis();
return new FlushData(batches, recordCount, estimatedBytes);
}
/**
* Perform the actual flush I/O operation. This method does NOT require holding the partition
* buffer lock, allowing buffer operations to proceed while the slow I/O happens. However, it
* acquires a separate flush lock to ensure only one flush happens at a time per partition,
* preventing concurrent writes to DuckLake which could cause metadata inconsistencies.
*/
private void flushBatches(TopicPartition partition, FlushData flushData) {
DucklakeWriter writer = writers.get(partition);
if (writer == null) {
LOG.warn(
"No writer found for partition: {}, closing {} batches",
partition,
flushData.batches.size());
closeBatches(flushData.batches);
return;
}
// Acquire flush lock to ensure only one flush at a time per partition
// This prevents concurrent DuckLake writes which could cause metadata inconsistencies
ReentrantLock flushLock = flushLocks.computeIfAbsent(partition, k -> new ReentrantLock());
flushLock.lock();
try {
long actualAllocatedBytes = allocator.getAllocatedMemory();
LOG.info(
"Flushing partition {}: {} batches, {} records, estimatedBytes={}, allocatorBytes={}",
partition,
flushData.batches.size(),
flushData.recordCount,
flushData.estimatedBytes,
actualAllocatedBytes);
List<VectorSchemaRoot> consolidated;
try {
try (var timer = ducklakeMetrics.startConsolidationTimer()) {
consolidated = BatchConsolidator.consolidate(flushData.batches);
}
if (consolidated.size() > 1) {
ducklakeMetrics.recordSchemaMismatch();
}
} catch (RuntimeException e) {
// Consolidation failed mid-append — source batches may be partially consumed.
// Close all original batches to avoid leaking Arrow memory.
closeBatches(flushData.batches);
throw e;
}
try {
String partitionKey = partition.topic() + "-" + partition.partition();
try (var timer = ducklakeMetrics.startFlushTimer(partitionKey)) {
for (VectorSchemaRoot root : consolidated) {
if (root.getRowCount() > 0) {
writer.write(root);
}
}
}
if (flushData.recordCount > 0) {
ducklakeMetrics.recordBatchProcessed(flushData.recordCount);
}
} catch (RuntimeException e) {
LOG.error("Failed to write buffered data for partition: {}", partition, e);
throw e;
} finally {
closeBatches(consolidated);
}
} finally {
flushLock.unlock();
}
}
/** Close a list of VectorSchemaRoot batches, logging any errors. */
private void closeBatches(List<VectorSchemaRoot> batches) {
for (VectorSchemaRoot root : batches) {
try {
root.close();
} catch (Exception closeEx) {
LOG.warn("Failed to close VectorSchemaRoot: {}", closeEx.getMessage());
}
}
}
/**
* Flush all buffered data for a partition. This method is used during stop() where the lock is
* already held. It extracts the data and performs the flush synchronously.
*/
private void flushPartition(TopicPartition partition) {
PartitionBuffer buffer = buffers.get(partition);
if (buffer == null || buffer.pendingBatches.isEmpty()) {
return;
}
// Extract flush data and perform flush
FlushData flushData = extractFlushDataFromBuffer(buffer);
flushBatches(partition, flushData);
}
/**
* Flush all spillable buffered data for a partition. This method is used during stop() where the
* lock is already held. It reads batches from disk and performs the flush synchronously.
*/
private void flushSpillablePartition(TopicPartition partition) {
SpillablePartitionBuffer buffer = spillableBuffers.get(partition);
if (buffer == null || buffer.isEmpty()) {
return;
}
LOG.info(
"Flushing spillable partition {} during stop: {} batches, {} records",
partition,
buffer.getBatchCount(),
buffer.getRecordCount());
// Read batches from disk and flush
List<VectorSchemaRoot> batches = buffer.readBatches(allocator);
FlushData flushData =
new FlushData(batches, buffer.getRecordCount(), buffer.getEstimatedBytes());
buffer.clear();
flushBatches(partition, flushData);
}
/**
* Cleans up orphaned ducklake-spill-* directories in the temp directory. These directories may
* have been left behind by previous task instances that crashed or were terminated without proper
* cleanup. Only cleans up directories matching the old temp directory naming pattern
* (ducklake-spill*).
*
* <p>Package-private for testing.
*/
void cleanupOrphanedSpillDirectories(Path tempDir) {
try (Stream<Path> paths = Files.list(tempDir)) {
paths
.filter(Files::isDirectory)
.filter(
p -> {
String name = p.getFileName().toString();
// Match old-style temp directories: ducklake-spill followed by random suffix
// e.g., ducklake-spill1234567890
// But NOT our new fixed directory name "ducklake-spill" (exact match)
return name.startsWith("ducklake-spill") && !name.equals("ducklake-spill");
})
.forEach(
orphanedDir -> {
LOG.info("Cleaning up orphaned spill directory: {}", orphanedDir);
try {
deleteDirectoryRecursively(orphanedDir);
} catch (IOException e) {
LOG.warn(
"Failed to clean up orphaned spill directory {}: {}",
orphanedDir,
e.getMessage());
}
});
} catch (IOException e) {
LOG.warn("Failed to scan for orphaned spill directories in {}: {}", tempDir, e.getMessage());
}
}
/** Recursively delete a directory and all its contents. */
private void deleteDirectoryRecursively(Path directory) throws IOException {
if (directory == null || !Files.exists(directory)) {
return;
}
Files.walk(directory)
.sorted(
java.util.Comparator
.reverseOrder()) // Reverse order so files are deleted before directories
.forEach(
path -> {
try {
Files.deleteIfExists(path);
} catch (IOException e) {
LOG.warn("Failed to delete {}: {}", path, e.getMessage());
}
});
}
/** Buffer Arrow IPC records for a single partition (called under partition lock) */
private void bufferArrowIpcRecordsForPartition(
TopicPartition partition, List<SinkRecord> records) {
LOG.debug("Buffering {} Arrow IPC records for partition {}", records.size(), partition);
PartitionBuffer buffer = buffers.get(partition);
if (buffer == null) {
LOG.warn("No buffer found for partition: {}", partition);
// Close VectorSchemaRoot objects if no buffer
for (SinkRecord record : records) {
if (record.value() instanceof VectorSchemaRoot root) {
try {
root.close();
} catch (Exception e) {
// Ignore
}
}
}
return;
}
for (SinkRecord record : records) {
if (record.value() instanceof VectorSchemaRoot vectorSchemaRoot) {
buffer.add(vectorSchemaRoot);
} else {
LOG.warn(
"Mixed data types detected - record value is not VectorSchemaRoot: {}",
record.value().getClass().getName());
}
}
}
/** Buffer traditional records for a single partition (called under partition lock) */
private void bufferTraditionalRecordsForPartition(
TopicPartition partition, List<SinkRecord> records) {
LOG.debug("Buffering {} traditional records for partition {}", records.size(), partition);
PartitionBuffer buffer = buffers.get(partition);
if (buffer == null) {
LOG.warn("No buffer found for partition: {}", partition);
return;
}
// Convert records for this single partition
Map<TopicPartition, List<SinkRecord>> singlePartitionMap = new HashMap<>();
singlePartitionMap.put(partition, records);
try {
// Fast path: try batch conversion
Map<TopicPartition, VectorSchemaRoot> vectorsByPartition =
converter.convertRecordsByPartition(singlePartitionMap);
VectorSchemaRoot vectorSchemaRoot = vectorsByPartition.get(partition);
if (vectorSchemaRoot != null) {
buffer.add(vectorSchemaRoot);
}
} catch (RuntimeException e) {
// Check if this is a schema conflict error (e.g., string vs timestamp type mismatch)
if (isSchemaConflictError(e)) {
LOG.warn(
"Schema conflict detected for partition {}, attempting per-record conversion to "
+ "identify bad records: {}",
partition,