-
Notifications
You must be signed in to change notification settings - Fork 9
/
Copy pathtablestate.go
690 lines (622 loc) · 23 KB
/
tablestate.go
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
// Copyright 2023 Rivian Automotive, Inc.
// 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 delta contains the resources required to interact with a Delta table.
package delta
import (
"bytes"
"context"
"errors"
"fmt"
"reflect"
"sort"
"strconv"
"strings"
"sync"
"time"
"github.com/apache/arrow/go/v14/arrow"
"github.com/apache/arrow/go/v14/arrow/array"
"github.com/apache/arrow/go/v14/arrow/memory"
"github.com/apache/arrow/go/v14/parquet/file"
"github.com/apache/arrow/go/v14/parquet/pqarrow"
"github.com/chelseajonesr/rfarrow"
"github.com/rivian/delta-go/storage"
)
// TableState maintains the current known state of a table
// This is used in reading and generating checkpoints
// If on-disk optimization is enabled, some of the information here is empty as the
// state is offloaded to disk to reduce memory use
type TableState struct {
// current table version represented by this table state
Version int64
// A remove action should remain in the state of the table as a tombstone until it has expired.
// A tombstone expires when the creation timestamp of the Delta file exceeds the expiration
// This is empty if on-disk optimization is enabled
Tombstones map[string]Remove
// Active files for table state
// This is empty if on-disk optimization is enabled
Files map[string]Add
// Information added to individual commits
CommitInfos []CommitInfo
AppTransactionVersion map[string]int64
MinReaderVersion int32
MinWriterVersion int32
ReaderFeatures map[string]bool
WriterFeatures map[string]bool
// Table metadata corresponding to current version
CurrentMetadata *TableMetaData
// Retention period for tombstones as time.Duration (nanoseconds)
TombstoneRetention time.Duration
// Retention period for log entries as time.Duration (nanoseconds)
LogRetention time.Duration
// Expired log cleanup has not been thoroughly tested, so marking as experimental
ExperimentalEnableExpiredLogCleanup bool
// Additional state for on-disk optimizations for large checkpoints
onDiskOptimization bool
OnDiskTableState
}
var (
// ErrMissingMetadata is returned if trying to create a checkpoint with no metadata
ErrMissingMetadata error = errors.New("missing metadata")
// ErrConvertingCheckpointAdd is returned if there is an error converting an Add action to checkpoint format
ErrConvertingCheckpointAdd error = errors.New("unable to generate checkpoint add")
// ErrCDCNotSupported is returned if a CDC action is seen when generating a checkpoint
ErrCDCNotSupported error = errors.New("cdc is not supported")
// ErrReadingCheckpoint is returned if there is an error reading a checkpoint
ErrReadingCheckpoint error = errors.New("unable to read checkpoint")
// ErrVersionOutOfOrder is returned if the versions are out of order when loading the table state
// This would indicate an internal logic error
ErrVersionOutOfOrder error = errors.New("versions out of order during update")
)
// NewTableState creates an empty table state for the given version
func NewTableState(version int64) *TableState {
tableState := new(TableState)
tableState.Version = version
tableState.Files = make(map[string]Add)
tableState.Tombstones = make(map[string]Remove)
tableState.AppTransactionVersion = make(map[string]int64)
// Default 7 days
tableState.TombstoneRetention = time.Hour * 24 * 7
// Default 30 days
tableState.LogRetention = time.Hour * 24 * 30
tableState.ExperimentalEnableExpiredLogCleanup = false
tableState.concurrentUpdateMutex = new(sync.Mutex)
tableState.ReaderFeatures = make(map[string]bool)
tableState.WriterFeatures = make(map[string]bool)
return tableState
}
func setupOnDiskOptimization(config *OptimizeCheckpointConfiguration, tableState *TableState, initialFileCount int) error {
if config != nil && config.OnDiskOptimization {
if config.WorkingStore == nil {
return errors.Join(ErrCheckpointOptimizationWorkingFolder, errors.New("the optimization working store is not set"))
}
existingFiles, err := config.WorkingStore.List(config.WorkingFolder, nil)
if err != nil {
return errors.Join(ErrCheckpointOptimizationWorkingFolder, err)
}
if len(existingFiles.Objects) > 0 {
// List may return a single result consisting of the folder itself
if len(existingFiles.Objects) > 1 || !strings.HasSuffix(existingFiles.Objects[0].Location.Raw, "/") {
return errors.Join(ErrCheckpointOptimizationWorkingFolder, errors.New("the optimization working folder is not empty"))
}
}
tableState.onDiskOptimization = true
tableState.onDiskTempFiles = make([]storage.Path, 0, initialFileCount)
}
return nil
}
// FileCount returns the total number of Parquet files making up the table at the loaded version
func (tableState *TableState) FileCount() int {
if tableState.onDiskOptimization {
return tableState.onDiskFileCount
}
return len(tableState.Files)
}
// TombstoneCount returns the total number of tombstones (logically but not physically deleted files) in the table at the loaded version
func (tableState *TableState) TombstoneCount() int {
if tableState.onDiskOptimization {
return tableState.onDiskTombstoneCount
}
return len(tableState.Tombstones)
}
// NewTableStateFromCommit reads a specific commit version and returns the contained TableState
func NewTableStateFromCommit(table *Table, version int64) (*TableState, error) {
actions, err := table.ReadCommitVersion(version)
if err != nil {
return nil, err
}
return NewTableStateFromActions(actions, version)
}
// NewTableStateFromActions generates table state from a list of actions
func NewTableStateFromActions(actions []Action, version int64) (*TableState, error) {
tableState := NewTableState(version)
for _, action := range actions {
err := tableState.processAction(action)
if err != nil {
return nil, err
}
}
return tableState, nil
}
// Update the table state by applying a single action
func (tableState *TableState) processAction(actionInterface Action) error {
switch action := actionInterface.(type) {
case *Add:
tableState.Files[action.Path] = *action
case *Remove:
// TODO - do we need to decode as in delta-rs?
tableState.Tombstones[action.Path] = *action
case *MetaData:
if action.Configuration != nil {
// Parse the configuration options that we make use of
option, ok := action.Configuration[string(DeletedFileRetentionDurationDeltaConfigKey)]
if ok {
duration, err := parseInterval(option)
if err != nil {
return err
}
tableState.TombstoneRetention = duration
}
option, ok = action.Configuration[string(LogRetentionDurationDeltaConfigKey)]
if ok {
duration, err := parseInterval(option)
if err != nil {
return err
}
tableState.LogRetention = duration
}
option, ok = action.Configuration[string(EnableExpiredLogCleanupDeltaConfigKey)]
if ok {
boolOption, err := strconv.ParseBool(option)
if err != nil {
return err
}
tableState.ExperimentalEnableExpiredLogCleanup = boolOption
}
}
deltaTableMetadata, err := action.toTableMetadata()
if err != nil {
return err
}
tableState.CurrentMetadata = &deltaTableMetadata
case *Txn:
tableState.AppTransactionVersion[action.AppID] = action.Version
case *Protocol:
tableState.MinReaderVersion = action.MinReaderVersion
tableState.MinWriterVersion = action.MinWriterVersion
tableState.ReaderFeatures = make(map[string]bool, len(action.ReaderFeatures))
tableState.WriterFeatures = make(map[string]bool, len(action.WriterFeatures))
for _, f := range action.ReaderFeatures {
tableState.ReaderFeatures[f] = true
}
for _, f := range action.WriterFeatures {
tableState.WriterFeatures[f] = true
}
case *CommitInfo:
tableState.CommitInfos = append(tableState.CommitInfos, *action)
case *CDC:
return ErrCDCNotSupported
default:
return errors.Join(ErrActionUnknown, fmt.Errorf("unknown %v", action))
}
return nil
}
// Merges new state information into our state
func (tableState *TableState) merge(newTableState *TableState, maxRowsPerPart int, config *OptimizeCheckpointConfiguration, finalMerge bool) error {
var err error
if tableState.onDiskOptimization {
err = tableState.mergeOnDiskState(newTableState, maxRowsPerPart, config, finalMerge)
if err != nil {
return err
}
// the final merge is to resolve pending adds/tombstones and does not include a new table state
if finalMerge {
return nil
}
}
// In memory file updates
for k, v := range newTableState.Tombstones {
// Remove deleted files from existing added files
delete(tableState.Files, k)
// Add deleted file tombstones to state so they're available for vacuum
tableState.Tombstones[k] = v
}
for k, v := range newTableState.Files {
// If files were deleted and then re-added, remove from updated tombstones
delete(tableState.Tombstones, k)
tableState.Files[k] = v
}
if newTableState.MinReaderVersion > 0 {
tableState.MinReaderVersion = newTableState.MinReaderVersion
tableState.MinWriterVersion = newTableState.MinWriterVersion
tableState.ReaderFeatures = newTableState.ReaderFeatures
tableState.WriterFeatures = newTableState.WriterFeatures
}
if newTableState.CurrentMetadata != nil {
tableState.TombstoneRetention = newTableState.TombstoneRetention
tableState.LogRetention = newTableState.LogRetention
tableState.ExperimentalEnableExpiredLogCleanup = newTableState.ExperimentalEnableExpiredLogCleanup
tableState.CurrentMetadata = newTableState.CurrentMetadata
}
for k, v := range newTableState.AppTransactionVersion {
tableState.AppTransactionVersion[k] = v
}
tableState.CommitInfos = append(tableState.CommitInfos, newTableState.CommitInfos...)
if newTableState.Version <= tableState.Version {
return ErrVersionOutOfOrder
}
tableState.Version = newTableState.Version
return nil
}
func stateFromCheckpoint(table *Table, checkpoint *CheckPoint, config *OptimizeCheckpointConfiguration) (*TableState, error) {
newState := NewTableState(checkpoint.Version)
checkpointDataPaths := table.GetCheckpointDataPaths(checkpoint)
err := setupOnDiskOptimization(config, newState, len(checkpointDataPaths))
if err != nil {
return nil, err
}
// Optional concurrency support
if newState.onDiskOptimization && config.ConcurrentCheckpointRead > 1 {
err := newState.applyCheckpointConcurrently(table.Store, checkpointDataPaths, config)
if err != nil {
return nil, err
}
} else {
// No concurrency
for i, location := range checkpointDataPaths {
task := checkpointProcessingTask{location: location, state: newState, part: i, config: config, store: table.Store}
err := stateFromCheckpointPart(task)
if err != nil {
return nil, err
}
}
}
return newState, nil
}
type checkpointProcessingTask struct {
location storage.Path
state *TableState
part int
config *OptimizeCheckpointConfiguration
store storage.ObjectStore
}
func stateFromCheckpointPart(task checkpointProcessingTask) error {
checkpointBytes, err := task.store.Get(task.location)
if err != nil {
return err
}
if len(checkpointBytes) > 0 {
err = task.state.processCheckpointBytes(checkpointBytes, task.part, task.config)
if err != nil {
return err
}
} else {
return errors.Join(ErrCheckpointIncomplete, fmt.Errorf("zero size checkpoint at %s", task.location.Raw))
}
return nil
}
func actionFromCheckpointEntry(checkpointEntry *CheckpointEntry) (Action, error) {
var action Action
if checkpointEntry.Add != nil {
action = checkpointEntry.Add
}
if checkpointEntry.Remove != nil {
if action != nil {
return action, ErrCheckpointEntryMultipleActions
}
action = checkpointEntry.Remove
}
if checkpointEntry.MetaData != nil {
if action != nil {
return action, ErrCheckpointEntryMultipleActions
}
action = checkpointEntry.MetaData
}
if checkpointEntry.Protocol != nil {
if action != nil {
return action, ErrCheckpointEntryMultipleActions
}
action = checkpointEntry.Protocol
}
if checkpointEntry.Txn != nil {
if action != nil {
return action, ErrCheckpointEntryMultipleActions
}
action = checkpointEntry.Txn
}
return action, nil
}
func (tableState *TableState) processCheckpointBytes(checkpointBytes []byte, part int, config *OptimizeCheckpointConfiguration) (returnErr error) {
concurrentCheckpointRead := tableState.onDiskOptimization && config.ConcurrentCheckpointRead > 1
var processEntryAction = func(checkpointEntry *CheckpointEntry) error {
action, err := actionFromCheckpointEntry(checkpointEntry)
if err != nil {
return err
}
if action != nil {
if concurrentCheckpointRead {
tableState.concurrentUpdateMutex.Lock()
defer tableState.concurrentUpdateMutex.Unlock()
}
err := tableState.processAction(action)
if err != nil {
return err
}
} else {
if !tableState.onDiskOptimization {
// This is expected during optimized on-disk reading but not otherwise
return errors.New("no action found in checkpoint record")
}
}
return nil
}
bytesReader := bytes.NewReader(checkpointBytes)
parquetReader, err := file.NewParquetReader(bytesReader)
if err != nil {
return err
}
defer func() {
if err := parquetReader.Close(); err != nil {
returnErr = errors.Join(errors.New("failed to close Parquet reader"), err)
}
}()
parquetSchema := parquetReader.MetaData().Schema
fileReader, err := pqarrow.NewFileReader(parquetReader, pqarrow.ArrowReadProperties{BatchSize: 10, Parallel: true}, memory.DefaultAllocator)
if err != nil {
return err
}
arrowSchema, err := fileReader.Schema()
if err != nil {
return err
}
arrowFieldList := arrowSchema.Fields()
// For on-disk optimization, don't load add/remove into memory
inMemoryCols := make([]int, 0, 150)
for i := 0; i < parquetSchema.NumColumns(); i++ {
columnPath := parquetSchema.Column(i).ColumnPath().String()
if !tableState.onDiskOptimization || (!strings.HasPrefix(columnPath, "add") && !strings.HasPrefix(columnPath, "remove")) {
inMemoryCols = append(inMemoryCols, i)
}
}
// Get mappings between struct member names and parquet/arrow names so we don't have to look them up repeatedly
// during record assignments
var fieldExclusions []string
if tableState.onDiskOptimization {
fieldExclusions = []string{"Add", "Remove"}
}
inMemoryIndexMappings, err := rfarrow.MapGoStructFieldNamesToArrowIndices[CheckpointEntry](arrowFieldList, fieldExclusions, true, true)
if err != nil {
return err
}
// Read all row groups and process in-memory actions
var tbl arrow.Table
if tableState.onDiskOptimization {
rgs := []int{}
for i := 0; i < parquetReader.NumRowGroups(); i++ {
rgs = append(rgs, i)
}
tbl, err = fileReader.ReadRowGroups(context.Background(), inMemoryCols, rgs)
} else {
tbl, err = fileReader.ReadTable(context.Background())
}
if err != nil {
return err
}
defer tbl.Release()
tableReader := array.NewTableReader(tbl, 0)
defer tableReader.Release()
for tableReader.Next() {
// the record contains a batch of rows
record := tableReader.Record()
entries := make([]*CheckpointEntry, record.NumRows())
entryValues := make([]reflect.Value, record.NumRows())
for j := int64(0); j < record.NumRows(); j++ {
t := new(CheckpointEntry)
entries[j] = t
entryValues[j] = reflect.ValueOf(t)
}
err = rfarrow.SetGoStructsFromArrowArrays(entryValues, record.Columns(), inMemoryIndexMappings, 0)
if err != nil {
return err
}
for j := int64(0); j < record.NumRows(); j++ {
err = processEntryAction(entries[j])
if err != nil {
return err
}
}
if err != nil {
return err
}
}
// Save the part file for on disk optimization
if tableState.onDiskOptimization {
// The non-add, non-remove columns will be almost entirely nulls, so picking out just add and remove
// slows us down here for a very minimal improvement in file size.
// Instead we just write out the entire file.
onDiskFile := storage.PathFromIter([]string{config.WorkingFolder.Raw, fmt.Sprintf("intermediate.%d.parquet", part)})
if err := config.WorkingStore.Put(onDiskFile, checkpointBytes); err != nil {
return errors.Join(errors.New("failed to add checkpoint bytes to on-disk file"), err)
}
func() {
tableState.concurrentUpdateMutex.Lock()
defer tableState.concurrentUpdateMutex.Unlock()
tableState.onDiskTempFiles = append(tableState.onDiskTempFiles, onDiskFile)
}()
// Store the number of add and remove records locally
// These counts are required later for generating new checkpoints
err = countAddsAndTombstones(tableState, checkpointBytes, arrowSchema, nil)
if err != nil {
return err
}
}
return nil
}
// Prepare the table state for checkpointing by updating tombstones
func (tableState *TableState) prepareStateForCheckpoint(config *OptimizeCheckpointConfiguration) error {
if tableState.CurrentMetadata == nil {
return ErrMissingMetadata
}
retentionTimestamp := time.Now().UnixMilli() - tableState.TombstoneRetention.Milliseconds()
if tableState.onDiskOptimization {
return tableState.prepareOnDiskStateForCheckpoint(retentionTimestamp, config)
}
// Don't keep expired tombstones
// Also check if any of the non-expired Remove actions had ExtendedFileMetadata = false
doNotUseExtendedFileMetadata := false
unexpiredTombstones := make(map[string]Remove, len(tableState.Tombstones))
for path, remove := range tableState.Tombstones {
if remove.DeletionTimestamp == nil || *remove.DeletionTimestamp > retentionTimestamp {
unexpiredTombstones[path] = remove
doNotUseExtendedFileMetadata = doNotUseExtendedFileMetadata && (!remove.ExtendedFileMetadata)
}
}
tableState.Tombstones = unexpiredTombstones
// If any Remove has ExtendedFileMetadata = false, set all to false
if doNotUseExtendedFileMetadata {
for path, remove := range tableState.Tombstones {
remove.ExtendedFileMetadata = false
tableState.Tombstones[path] = remove
// TODO - remove the extended fields (remove.size, remove.partitionValues) from the schema
}
}
return nil
}
// Retrieve the next batch of checkpoint entries to write to Parquet
func checkpointRows(
tableState *TableState, startOffset int, config *CheckpointConfiguration) ([]CheckpointEntry, error) {
var maxRowCount int
maxRowCount = 2 + len(tableState.AppTransactionVersion) + tableState.FileCount() + tableState.TombstoneCount()
if config.MaxRowsPerPart < maxRowCount {
maxRowCount = config.MaxRowsPerPart
}
checkpointRows := make([]CheckpointEntry, 0, maxRowCount)
currentOffset := 0
// Row 1: protocol
if startOffset <= currentOffset {
protocol := new(Protocol)
protocol.MinReaderVersion = tableState.MinReaderVersion
protocol.MinWriterVersion = tableState.MinWriterVersion
// From the specifications:
// If a table has Reader Version 3, then a writer must write checkpoints with a not-null readerFeatures in the schema.
// If a table has Writer Version 7, then a writer must write checkpoints with a not-null writerFeatures in the schema.
// If a table has neither of the above, then a writer chooses whether to write readerFeatures and/or writerFeatures into the checkpoint schema. But if it does, their values must be null.
if protocol.MinReaderVersion >= 3 {
protocol.ReaderFeatures = make([]string, 0, len(tableState.ReaderFeatures))
for k := range tableState.ReaderFeatures {
protocol.ReaderFeatures = append(protocol.ReaderFeatures, k)
}
} else {
protocol.ReaderFeatures = nil
}
if protocol.MinWriterVersion >= 7 {
protocol.WriterFeatures = make([]string, 0, len(tableState.WriterFeatures))
for k := range tableState.WriterFeatures {
protocol.WriterFeatures = append(protocol.WriterFeatures, k)
}
} else {
protocol.WriterFeatures = nil
}
checkpointRows = append(checkpointRows, CheckpointEntry{Protocol: protocol})
}
currentOffset++
// Row 2: metadata
if startOffset <= currentOffset && len(checkpointRows) < config.MaxRowsPerPart {
metadata := tableState.CurrentMetadata.toMetaData()
checkpointRows = append(checkpointRows, CheckpointEntry{MetaData: &metadata})
}
currentOffset++
// Next, optional Txn entries per app id
if startOffset < currentOffset+len(tableState.AppTransactionVersion) && len(tableState.AppTransactionVersion) > 0 && len(checkpointRows) < config.MaxRowsPerPart {
keys := make([]string, 0, len(tableState.AppTransactionVersion))
for k := range tableState.AppTransactionVersion {
keys = append(keys, k)
}
sort.Strings(keys)
for i, appID := range keys {
if startOffset < currentOffset+i {
txn := new(Txn)
txn.AppID = appID
version := tableState.AppTransactionVersion[appID]
txn.Version = version
checkpointRows = append(checkpointRows, CheckpointEntry{Txn: txn})
if len(checkpointRows) >= config.MaxRowsPerPart {
break
}
}
}
}
currentOffset += len(tableState.AppTransactionVersion)
// Tombstone / Remove entries
tombstoneCount := tableState.TombstoneCount()
if startOffset < currentOffset+tombstoneCount && tombstoneCount > 0 && len(checkpointRows) < config.MaxRowsPerPart {
if tableState.onDiskOptimization {
initialOffset := startOffset - currentOffset
if initialOffset < 0 {
initialOffset = 0
}
if err := onDiskTombstoneCheckpointRows(tableState, initialOffset, &checkpointRows, config); err != nil {
return nil, errors.Join(errors.New("failed to retrieve on-disk tombstone checkpoint rows"), err)
}
} else {
keys := make([]string, 0, tombstoneCount)
for k := range tableState.Tombstones {
keys = append(keys, k)
}
sort.Strings(keys)
for i, path := range keys {
if startOffset <= currentOffset+i {
checkpointRemove := new(Remove)
*checkpointRemove = tableState.Tombstones[path]
checkpointRows = append(checkpointRows, CheckpointEntry{Remove: checkpointRemove})
if len(checkpointRows) >= config.MaxRowsPerPart {
break
}
}
}
}
}
currentOffset += tombstoneCount
// Add entries
fileCount := tableState.FileCount()
if startOffset < currentOffset+fileCount && fileCount > 0 && len(checkpointRows) < config.MaxRowsPerPart {
if tableState.onDiskOptimization {
initialOffset := startOffset - currentOffset
if initialOffset < 0 {
initialOffset = 0
}
if err := onDiskAddCheckpointRows(tableState, initialOffset, &checkpointRows, config); err != nil {
return nil, errors.Join(errors.New("failed to retrieve on-disk add checkpoint rows"), err)
}
} else {
keys := make([]string, 0, fileCount)
for k := range tableState.Files {
keys = append(keys, k)
}
sort.Strings(keys)
for i, path := range keys {
if startOffset <= currentOffset+i {
add := tableState.Files[path]
checkpointAdd, err := checkpointAdd(&add)
if err != nil {
return nil, errors.Join(ErrConvertingCheckpointAdd, err)
}
checkpointRows = append(checkpointRows, CheckpointEntry{Add: checkpointAdd})
if len(checkpointRows) >= config.MaxRowsPerPart {
break
}
}
}
}
}
return checkpointRows, nil
}