@ -3,6 +3,8 @@ package iceberg
import (
"bytes"
"context"
"crypto/rand"
"encoding/hex"
"encoding/json"
"errors"
"fmt"
@ -22,10 +24,11 @@ import (
"google.golang.org/grpc/status"
)
// compactionBin groups small data files from the same partition for merging.
// compactionBin groups small data files from the same partition and spec for merging.
type compactionBin struct {
PartitionKey string
Partition map [ int ] any
SpecID int32
Entries [ ] iceberg . ManifestEntry
TotalSize int64
}
@ -38,64 +41,116 @@ func (h *Handler) compactDataFiles(
filerClient filer_pb . SeaweedFilerClient ,
bucketName , tablePath string ,
config Config ,
) ( string , error ) {
onProgress func ( binIdx , totalBins int ) ,
) ( string , map [ string ] int64 , error ) {
start := time . Now ( )
meta , metadataFileName , err := loadCurrentMetadata ( ctx , filerClient , bucketName , tablePath )
if err != nil {
return "" , fmt . Errorf ( "load metadata: %w" , err )
return "" , nil , fmt . Errorf ( "load metadata: %w" , err )
}
currentSnap := meta . CurrentSnapshot ( )
if currentSnap == nil || currentSnap . ManifestList == "" {
return "no current snapshot" , nil
return "no current snapshot" , nil , nil
}
// Read manifest list
manifestListData , err := loadFileByIcebergPath ( ctx , filerClient , bucketName , tablePath , currentSnap . ManifestList )
if err != nil {
return "" , fmt . Errorf ( "read manifest list: %w" , err )
return "" , nil , fmt . Errorf ( "read manifest list: %w" , err )
}
manifests , err := iceberg . ReadManifestList ( bytes . NewReader ( manifestListData ) )
if err != nil {
return "" , fmt . Errorf ( "parse manifest list: %w" , err )
return "" , nil , fmt . Errorf ( "parse manifest list: %w" , err )
}
// Abort if delete manifests exist — the compactor does not apply deletes,
// so carrying them through could produce incorrect results.
// Also detect multiple partition specs — the compactor writes a single
// manifest under the current spec which is invalid for spec-evolved tables.
specIDs := make ( map [ int32 ] struct { } )
// Separate data manifests from delete manifests.
var dataManifests , deleteManifests [ ] iceberg . ManifestFile
for _ , mf := range manifests {
if mf . ManifestContent ( ) != iceberg . ManifestContentData {
return "compaction skipped: delete manifests present (not yet supported)" , nil
if mf . ManifestContent ( ) == iceberg . ManifestContentData {
dataManifests = append ( dataManifests , mf )
} else {
deleteManifests = append ( deleteManifests , mf )
}
specIDs [ mf . PartitionSpecID ( ) ] = struct { } { }
}
if len ( specIDs ) > 1 {
return "compaction skipped: multiple partition specs present (not yet supported)" , nil
// If delete manifests exist and apply_deletes is disabled (or not yet
// implemented for this code path), skip compaction to avoid producing
// incorrect results by dropping deletes.
if len ( deleteManifests ) > 0 && ! config . ApplyDeletes {
return "compaction skipped: delete manifests present and apply_deletes is disabled" , nil , nil
}
// Collect data file entries from data manifests
var allEntries [ ] iceberg . ManifestEntry
for _ , mf := range m anifests {
for _ , mf := range dataM anifests {
manifestData , err := loadFileByIcebergPath ( ctx , filerClient , bucketName , tablePath , mf . FilePath ( ) )
if err != nil {
return "" , fmt . Errorf ( "read manifest %s: %w" , mf . FilePath ( ) , err )
return "" , nil , fmt . Errorf ( "read manifest %s: %w" , mf . FilePath ( ) , err )
}
entries , err := iceberg . ReadManifest ( mf , bytes . NewReader ( manifestData ) , true )
if err != nil {
return "" , fmt . Errorf ( "parse manifest %s: %w" , mf . FilePath ( ) , err )
return "" , nil , fmt . Errorf ( "parse manifest %s: %w" , mf . FilePath ( ) , err )
}
allEntries = append ( allEntries , entries ... )
}
// Collect delete entries if we need to apply deletes
var positionDeletes map [ string ] [ ] int64
var eqDeleteGroups [ ] equalityDeleteGroup
if config . ApplyDeletes && len ( deleteManifests ) > 0 {
var allDeleteEntries [ ] iceberg . ManifestEntry
for _ , mf := range deleteManifests {
manifestData , err := loadFileByIcebergPath ( ctx , filerClient , bucketName , tablePath , mf . FilePath ( ) )
if err != nil {
return "" , nil , fmt . Errorf ( "read delete manifest %s: %w" , mf . FilePath ( ) , err )
}
entries , err := iceberg . ReadManifest ( mf , bytes . NewReader ( manifestData ) , true )
if err != nil {
return "" , nil , fmt . Errorf ( "parse delete manifest %s: %w" , mf . FilePath ( ) , err )
}
allDeleteEntries = append ( allDeleteEntries , entries ... )
}
// Separate position and equality deletes
var posDeleteEntries , eqDeleteEntries [ ] iceberg . ManifestEntry
for _ , entry := range allDeleteEntries {
switch entry . DataFile ( ) . ContentType ( ) {
case iceberg . EntryContentPosDeletes :
posDeleteEntries = append ( posDeleteEntries , entry )
case iceberg . EntryContentEqDeletes :
eqDeleteEntries = append ( eqDeleteEntries , entry )
}
}
if len ( posDeleteEntries ) > 0 {
positionDeletes , err = collectPositionDeletes ( ctx , filerClient , bucketName , tablePath , posDeleteEntries )
if err != nil {
return "" , nil , fmt . Errorf ( "collect position deletes: %w" , err )
}
}
if len ( eqDeleteEntries ) > 0 {
eqDeleteGroups , err = collectEqualityDeletes ( ctx , filerClient , bucketName , tablePath , eqDeleteEntries , meta . CurrentSchema ( ) )
if err != nil {
return "" , nil , fmt . Errorf ( "collect equality deletes: %w" , err )
}
}
}
// Build compaction bins: group small files by partition
// MinInputFiles is clamped by ParseConfig to [2, ...] so int conversion is safe.
bins := buildCompactionBins ( allEntries , config . TargetFileSizeBytes , int ( config . MinInputFiles ) )
if len ( bins ) == 0 {
return "no files eligible for compaction" , nil
return "no files eligible for compaction" , nil , nil
}
// Build a lookup from spec ID to PartitionSpec for per-bin manifest writing.
specByID := make ( map [ int ] iceberg . PartitionSpec )
for _ , ps := range meta . PartitionSpecs ( ) {
specByID [ ps . ID ( ) ] = ps
}
spec := meta . PartitionSpec ( )
schema := meta . CurrentSchema ( )
version := meta . Version ( )
snapshotID := currentSnap . SnapshotID
@ -103,6 +158,9 @@ func (h *Handler) compactDataFiles(
// Compute the snapshot ID for the commit up front so all manifest entries
// reference the same snapshot that will actually be committed.
newSnapID := time . Now ( ) . UnixMilli ( )
// Random suffix for artifact filenames to avoid collisions between
// concurrent compaction runs on different tables sharing a timestamp.
artifactSuffix := compactRandomSuffix ( )
// Process each bin: read source Parquet files, merge, write output
var newManifestEntries [ ] iceberg . ManifestEntry
@ -152,73 +210,87 @@ func (h *Handler) compactDataFiles(
for binIdx , bin := range bins {
select {
case <- ctx . Done ( ) :
return "" , ctx . Err ( )
return "" , nil , ctx . Err ( )
default :
}
mergedFileName := fmt . Sprintf ( "compact-%d-%d-%d.parquet" , snapshotID , newSnapID , binIdx )
mergedFileName := fmt . Sprintf ( "compact-%d-%d-%s-% d.parquet" , snapshotID , newSnapID , artifactSuffix , binIdx )
mergedFilePath := path . Join ( "data" , mergedFileName )
mergedData , recordCount , err := mergeParquetFiles ( ctx , filerClient , bucketName , tablePath , bin . Entries )
mergedData , recordCount , err := mergeParquetFiles ( ctx , filerClient , bucketName , tablePath , bin . Entries , positionDeletes , eqDeleteGroups , schema )
if err != nil {
glog . Warningf ( "iceberg compact: failed to merge bin %d (%d files): %v" , binIdx , len ( bin . Entries ) , err )
continu e
goto binDon e
}
// Write merged file to filer
if err := ensureFilerDir ( ctx , filerClient , dataDir ) ; err != nil {
return "" , fmt . Errorf ( "ensure data dir: %w" , err )
return "" , nil , fmt . Errorf ( "ensure data dir: %w" , err )
}
if err := saveFilerFile ( ctx , filerClient , dataDir , mergedFileName , mergedData ) ; err != nil {
return "" , fmt . Errorf ( "save merged file: %w" , err )
}
// Create new DataFile entry for the merged file
dfBuilder , err := iceberg . NewDataFileBuilder (
spec ,
iceberg . EntryContentData ,
mergedFilePath ,
iceberg . ParquetFile ,
bin . Partition ,
nil , nil ,
recordCount ,
int64 ( len ( mergedData ) ) ,
)
if err != nil {
glog . Warningf ( "iceberg compact: failed to build data file entry for bin %d: %v" , binIdx , err )
// Clean up the written file
_ = deleteFilerFile ( ctx , filerClient , dataDir , mergedFileName )
continue
return "" , nil , fmt . Errorf ( "save merged file: %w" , err )
}
writtenArtifacts = append ( writtenArtifacts , artifact { dir : dataDir , fileName : mergedFileName } )
newEntry := iceberg . NewManifestEntry (
iceberg . EntryStatusADDED ,
& newSnapID ,
nil , nil ,
dfBuilder . Build ( ) ,
)
newManifestEntries = append ( newManifestEntries , newEntry )
// Use the partition spec matching this bin's spec ID
{
binSpec , ok := specByID [ int ( bin . SpecID ) ]
if ! ok {
glog . Warningf ( "iceberg compact: spec %d not found for bin %d, skipping" , bin . SpecID , binIdx )
_ = deleteFilerFile ( ctx , filerClient , dataDir , mergedFileName )
goto binDone
}
// Create new DataFile entry for the merged file
dfBuilder , err := iceberg . NewDataFileBuilder (
binSpec ,
iceberg . EntryContentData ,
mergedFilePath ,
iceberg . ParquetFile ,
bin . Partition ,
nil , nil ,
recordCount ,
int64 ( len ( mergedData ) ) ,
)
if err != nil {
glog . Warningf ( "iceberg compact: failed to build data file entry for bin %d: %v" , binIdx , err )
_ = deleteFilerFile ( ctx , filerClient , dataDir , mergedFileName )
goto binDone
}
writtenArtifacts = append ( writtenArtifacts , artifact { dir : dataDir , fileName : mergedFileName } )
// Mark original entries as deleted
for _ , entry := range bin . Entries {
delEntry := iceberg . NewManifestEntry (
iceberg . EntryStatusDELETED ,
newEntry := iceberg . NewManifestEntry (
iceberg . EntryStatusADDED ,
& newSnapID ,
entrySeqNum ( entry ) , entryFileSeqNum ( entry ) ,
entry . DataFile ( ) ,
nil , nil ,
dfBuilder . Build ( ) ,
)
deletedManifestEntries = append ( deletedManifestEntries , delEntry )
newManifestEntries = append ( newManifestEntries , newEntry )
// Mark original entries as deleted
for _ , entry := range bin . Entries {
delEntry := iceberg . NewManifestEntry (
iceberg . EntryStatusDELETED ,
& newSnapID ,
entrySeqNum ( entry ) , entryFileSeqNum ( entry ) ,
entry . DataFile ( ) ,
)
deletedManifestEntries = append ( deletedManifestEntries , delEntry )
}
totalMerged += len ( bin . Entries )
}
totalMerged += len ( bin . Entries )
binDone :
if onProgress != nil {
onProgress ( binIdx , len ( bins ) )
}
}
if len ( newManifestEntries ) == 0 {
return "no bins successfully compacted" , nil
return "no bins successfully compacted" , nil , nil
}
// Build entries for the new manifest:
// Build entries for the new manifests :
// - ADDED entries for merged files
// - DELETED entries for original files
// - EXISTING entries for files that weren't compacted
@ -227,11 +299,31 @@ func (h *Handler) compactDataFiles(
compactedPaths [ entry . DataFile ( ) . FilePath ( ) ] = struct { } { }
}
var manifestEntries [ ] iceberg . ManifestEntry
manifestEntries = append ( manifestEntries , newManifestEntries ... )
manifestEntries = append ( manifestEntries , deletedManifestEntries ... )
// Group all manifest entries by spec ID for per-spec manifest writing.
type specEntries struct {
specID int32
entries [ ] iceberg . ManifestEntry
}
specEntriesMap := make ( map [ int32 ] * specEntries )
addToSpec := func ( specID int32 , entry iceberg . ManifestEntry ) {
se , ok := specEntriesMap [ specID ]
if ! ok {
se = & specEntries { specID : specID }
specEntriesMap [ specID ] = se
}
se . entries = append ( se . entries , entry )
}
// Keep existing entries that weren't compacted
// New and deleted entries carry the spec ID from their bin
for _ , entry := range newManifestEntries {
addToSpec ( entry . DataFile ( ) . SpecID ( ) , entry )
}
for _ , entry := range deletedManifestEntries {
addToSpec ( entry . DataFile ( ) . SpecID ( ) , entry )
}
// Existing entries that weren't compacted
for _ , entry := range allEntries {
if _ , compacted := compactedPaths [ entry . DataFile ( ) . FilePath ( ) ] ; ! compacted {
existingEntry := iceberg . NewManifestEntry (
@ -240,46 +332,87 @@ func (h *Handler) compactDataFiles(
entrySeqNum ( entry ) , entryFileSeqNum ( entry ) ,
entry . DataFile ( ) ,
)
manifestEntries = append ( manifestEntries , existingEntry )
}
}
// Write new manifest
var manifestBuf bytes . Buffer
manifestFileName := fmt . Sprintf ( "compact-%d.avro" , newSnapID )
newManifest , err := iceberg . WriteManifest (
path . Join ( "metadata" , manifestFileName ) ,
& manifestBuf ,
version ,
spec ,
schema ,
newSnapID ,
manifestEntries ,
)
if err != nil {
return "" , fmt . Errorf ( "write compact manifest: %w" , err )
addToSpec ( entry . DataFile ( ) . SpecID ( ) , existingEntry )
}
}
// Write one manifest per spec ID, iterating in sorted order for
// deterministic manifest list construction.
sortedSpecIDs := make ( [ ] int32 , 0 , len ( specEntriesMap ) )
for sid := range specEntriesMap {
sortedSpecIDs = append ( sortedSpecIDs , sid )
}
sort . Slice ( sortedSpecIDs , func ( i , j int ) bool { return sortedSpecIDs [ i ] < sortedSpecIDs [ j ] } )
var allManifests [ ] iceberg . ManifestFile
for _ , sid := range sortedSpecIDs {
se := specEntriesMap [ sid ]
ps , ok := specByID [ int ( se . specID ) ]
if ! ok {
return "" , nil , fmt . Errorf ( "partition spec %d not found in table metadata" , se . specID )
}
var manifestBuf bytes . Buffer
manifestFileName := fmt . Sprintf ( "compact-%d-%s-spec%d.avro" , newSnapID , artifactSuffix , se . specID )
newManifest , err := iceberg . WriteManifest (
path . Join ( "metadata" , manifestFileName ) ,
& manifestBuf ,
version ,
ps ,
schema ,
newSnapID ,
se . entries ,
)
if err != nil {
return "" , nil , fmt . Errorf ( "write compact manifest for spec %d: %w" , se . specID , err )
}
if err := saveFilerFile ( ctx , filerClient , metaDir , manifestFileName , manifestBuf . Bytes ( ) ) ; err != nil {
return "" , fmt . Errorf ( "save compact manifest: %w" , err )
if err := saveFilerFile ( ctx , filerClient , metaDir , manifestFileName , manifestBuf . Bytes ( ) ) ; err != nil {
return "" , nil , fmt . Errorf ( "save compact manifest for spec %d: %w" , se . specID , err )
}
writtenArtifacts = append ( writtenArtifacts , artifact { dir : metaDir , fileName : manifestFileName } )
allManifests = append ( allManifests , newManifest )
}
writtenArtifacts = append ( writtenArtifacts , artifact { dir : metaDir , fileName : manifestFileName } )
// Build manifest list with only the new manifest (the early abort at the
// top of this function guarantees no delete manifests are present).
allManifests := [ ] iceberg . ManifestFile { newManifest }
// Carry forward delete manifests only if deletes were NOT applied.
// When deletes were applied, they've been consumed during the merge.
// Position deletes reference specific data files — if all those files
// were compacted, the deletes are fully consumed. Equality deletes
// apply broadly, so they're only consumed if all data files were compacted.
if ! config . ApplyDeletes || ( len ( positionDeletes ) == 0 && len ( eqDeleteGroups ) == 0 ) {
for _ , mf := range deleteManifests {
allManifests = append ( allManifests , mf )
}
} else {
// Check if any non-compacted data files remain
hasUncompactedFiles := false
for _ , entry := range allEntries {
if _ , compacted := compactedPaths [ entry . DataFile ( ) . FilePath ( ) ] ; ! compacted {
hasUncompactedFiles = true
break
}
}
if hasUncompactedFiles {
// Some files weren't compacted — carry forward delete manifests
// since deletes may still apply to those files.
for _ , mf := range deleteManifests {
allManifests = append ( allManifests , mf )
}
}
// If all files were compacted, deletes are fully consumed — don't carry forward.
}
// Write new manifest list
var manifestListBuf bytes . Buffer
seqNum := currentSnap . SequenceNumber + 1
err = iceberg . WriteManifestList ( version , & manifestListBuf , newSnapID , & snapshotID , & seqNum , 0 , allManifests )
if err != nil {
return "" , fmt . Errorf ( "write compact manifest list: %w" , err )
return "" , nil , fmt . Errorf ( "write compact manifest list: %w" , err )
}
manifestListFileName := fmt . Sprintf ( "snap-%d.avro" , newSnapID )
manifestListFileName := fmt . Sprintf ( "snap-%d-%s .avro" , newSnapID , artifactSuffix )
if err := saveFilerFile ( ctx , filerClient , metaDir , manifestListFileName , manifestListBuf . Bytes ( ) ) ; err != nil {
return "" , fmt . Errorf ( "save compact manifest list: %w" , err )
return "" , nil , fmt . Errorf ( "save compact manifest list: %w" , err )
}
writtenArtifacts = append ( writtenArtifacts , artifact { dir : metaDir , fileName : manifestListFileName } )
@ -318,11 +451,17 @@ func (h *Handler) compactDataFiles(
return builder . SetSnapshotRef ( table . MainBranch , newSnapID , table . BranchRef )
} )
if err != nil {
return "" , fmt . Errorf ( "commit compaction: %w" , err )
return "" , nil , fmt . Errorf ( "commit compaction: %w" , err )
}
committed = true
return fmt . Sprintf ( "compacted %d files into %d (across %d bins)" , totalMerged , len ( newManifestEntries ) , len ( bins ) ) , nil
metrics := map [ string ] int64 {
MetricFilesMerged : int64 ( totalMerged ) ,
MetricFilesWritten : int64 ( len ( newManifestEntries ) ) ,
MetricBins : int64 ( len ( bins ) ) ,
MetricDurationMs : time . Since ( start ) . Milliseconds ( ) ,
}
return fmt . Sprintf ( "compacted %d files into %d (across %d bins)" , totalMerged , len ( newManifestEntries ) , len ( bins ) ) , metrics , nil
}
// buildCompactionBins groups small data files by partition for bin-packing.
@ -333,7 +472,8 @@ func buildCompactionBins(entries []iceberg.ManifestEntry, targetSize int64, minF
minFiles = 2
}
// Group entries by partition key
// Group entries by spec ID + partition key so that files from different
// partition specs are never mixed in the same compaction bin.
groups := make ( map [ string ] * compactionBin )
for _ , entry := range entries {
df := entry . DataFile ( )
@ -345,13 +485,15 @@ func buildCompactionBins(entries []iceberg.ManifestEntry, targetSize int64, minF
}
partKey := partitionKey ( df . Partition ( ) )
bin , ok := groups [ partKey ]
groupKey := fmt . Sprintf ( "spec%d\x00%s" , df . SpecID ( ) , partKey )
bin , ok := groups [ groupKey ]
if ! ok {
bin = & compactionBin {
PartitionKey : partKey ,
Partition : df . Partition ( ) ,
SpecID : df . SpecID ( ) ,
}
groups [ part Key ] = bin
groups [ grou pKey] = bin
}
bin . Entries = append ( bin . Entries , entry )
bin . TotalSize += df . FileSizeBytes ( )
@ -370,8 +512,11 @@ func buildCompactionBins(entries []iceberg.ManifestEntry, targetSize int64, minF
}
}
// Sort by partition key for deterministic order
// Sort by spec ID then partition key for deterministic order
sort . Slice ( result , func ( i , j int ) bool {
if result [ i ] . SpecID != result [ j ] . SpecID {
return result [ i ] . SpecID < result [ j ] . SpecID
}
return result [ i ] . PartitionKey < result [ j ] . PartitionKey
} )
@ -394,6 +539,7 @@ func splitOversizedBin(bin compactionBin, targetSize int64, minFiles int) []comp
current := compactionBin {
PartitionKey : bin . PartitionKey ,
Partition : bin . Partition ,
SpecID : bin . SpecID ,
}
for _ , entry := range sorted {
if current . TotalSize > 0 && current . TotalSize + entry . DataFile ( ) . FileSizeBytes ( ) > targetSize {
@ -401,6 +547,7 @@ func splitOversizedBin(bin compactionBin, targetSize int64, minFiles int) []comp
current = compactionBin {
PartitionKey : bin . PartitionKey ,
Partition : bin . Partition ,
SpecID : bin . SpecID ,
}
}
current . Entries = append ( current . Entries , entry )
@ -474,17 +621,251 @@ func partitionKey(partition map[int]any) string {
return strings . Join ( parts , "\x00" )
}
// collectPositionDeletes reads position delete Parquet files and returns a map
// from normalized data file path to sorted row positions that should be deleted.
// Paths are normalized so that absolute S3 URLs and relative paths match.
func collectPositionDeletes (
ctx context . Context ,
filerClient filer_pb . SeaweedFilerClient ,
bucketName , tablePath string ,
deleteEntries [ ] iceberg . ManifestEntry ,
) ( map [ string ] [ ] int64 , error ) {
result := make ( map [ string ] [ ] int64 )
for _ , entry := range deleteEntries {
if entry . DataFile ( ) . ContentType ( ) != iceberg . EntryContentPosDeletes {
continue
}
fileDeletes , err := readPositionDeleteFile ( ctx , filerClient , bucketName , tablePath , entry . DataFile ( ) . FilePath ( ) )
if err != nil {
return nil , fmt . Errorf ( "read position delete file %s: %w" , entry . DataFile ( ) . FilePath ( ) , err )
}
for filePath , positions := range fileDeletes {
normalized := normalizeIcebergPath ( filePath , bucketName , tablePath )
result [ normalized ] = append ( result [ normalized ] , positions ... )
}
}
// Sort positions for each file (binary search during filtering)
for filePath := range result {
sort . Slice ( result [ filePath ] , func ( i , j int ) bool {
return result [ filePath ] [ i ] < result [ filePath ] [ j ]
} )
}
return result , nil
}
// readPositionDeleteFile reads a position delete Parquet file and returns a map
// from data file path to row positions. The file must have columns "file_path"
// (string) and "pos" (int32 or int64).
func readPositionDeleteFile (
ctx context . Context ,
filerClient filer_pb . SeaweedFilerClient ,
bucketName , tablePath , filePath string ,
) ( map [ string ] [ ] int64 , error ) {
data , err := loadFileByIcebergPath ( ctx , filerClient , bucketName , tablePath , filePath )
if err != nil {
return nil , err
}
reader := parquet . NewReader ( bytes . NewReader ( data ) )
defer reader . Close ( )
pqSchema := reader . Schema ( )
filePathIdx := - 1
posIdx := - 1
for i , col := range pqSchema . Columns ( ) {
name := strings . Join ( col , "." )
switch name {
case "file_path" :
filePathIdx = i
case "pos" :
posIdx = i
}
}
if filePathIdx < 0 || posIdx < 0 {
return nil , fmt . Errorf ( "position delete file %s missing required columns (file_path=%d, pos=%d)" , filePath , filePathIdx , posIdx )
}
result := make ( map [ string ] [ ] int64 )
rows := make ( [ ] parquet . Row , 256 )
for {
select {
case <- ctx . Done ( ) :
return nil , ctx . Err ( )
default :
}
n , readErr := reader . ReadRows ( rows )
for i := 0 ; i < n ; i ++ {
row := rows [ i ]
fp := row [ filePathIdx ] . String ( )
pos := row [ posIdx ] . Int64 ( )
result [ fp ] = append ( result [ fp ] , pos )
}
if readErr != nil {
if readErr == io . EOF {
break
}
return nil , readErr
}
}
return result , nil
}
// equalityDeleteGroup holds a set of delete keys for a specific set of equality field IDs.
// Different equality delete files may use different field IDs, so deletes are grouped.
type equalityDeleteGroup struct {
FieldIDs [ ] int
Keys map [ string ] struct { }
}
// collectEqualityDeletes reads equality delete Parquet files and returns groups
// of delete keys, one per distinct set of equality field IDs. This correctly
// handles the case where different delete files use different equality columns.
func collectEqualityDeletes (
ctx context . Context ,
filerClient filer_pb . SeaweedFilerClient ,
bucketName , tablePath string ,
deleteEntries [ ] iceberg . ManifestEntry ,
schema * iceberg . Schema ,
) ( [ ] equalityDeleteGroup , error ) {
type groupState struct {
fieldIDs [ ] int
keys map [ string ] struct { }
}
groups := make ( map [ string ] * groupState )
for _ , entry := range deleteEntries {
if entry . DataFile ( ) . ContentType ( ) != iceberg . EntryContentEqDeletes {
continue
}
eqFieldIDs := entry . DataFile ( ) . EqualityFieldIDs ( )
if len ( eqFieldIDs ) == 0 {
continue
}
groupKey := fmt . Sprint ( eqFieldIDs )
gs , ok := groups [ groupKey ]
if ! ok {
gs = & groupState { fieldIDs : eqFieldIDs , keys : make ( map [ string ] struct { } ) }
groups [ groupKey ] = gs
}
keys , err := readEqualityDeleteFile ( ctx , filerClient , bucketName , tablePath , entry . DataFile ( ) . FilePath ( ) , eqFieldIDs , schema )
if err != nil {
return nil , fmt . Errorf ( "read equality delete file %s: %w" , entry . DataFile ( ) . FilePath ( ) , err )
}
for k := range keys {
gs . keys [ k ] = struct { } { }
}
}
result := make ( [ ] equalityDeleteGroup , 0 , len ( groups ) )
for _ , gs := range groups {
result = append ( result , equalityDeleteGroup { FieldIDs : gs . fieldIDs , Keys : gs . keys } )
}
return result , nil
}
// readEqualityDeleteFile reads an equality delete Parquet file and returns a set
// of composite keys built from the specified field IDs. The Iceberg schema is used
// to map field IDs to column names, which are then looked up in the Parquet schema.
func readEqualityDeleteFile (
ctx context . Context ,
filerClient filer_pb . SeaweedFilerClient ,
bucketName , tablePath , filePath string ,
fieldIDs [ ] int ,
icebergSchema * iceberg . Schema ,
) ( map [ string ] struct { } , error ) {
data , err := loadFileByIcebergPath ( ctx , filerClient , bucketName , tablePath , filePath )
if err != nil {
return nil , err
}
reader := parquet . NewReader ( bytes . NewReader ( data ) )
defer reader . Close ( )
colIndices , err := resolveEqualityColIndices ( reader . Schema ( ) , fieldIDs , icebergSchema )
if err != nil {
return nil , fmt . Errorf ( "resolve columns in %s: %w" , filePath , err )
}
result := make ( map [ string ] struct { } )
rows := make ( [ ] parquet . Row , 256 )
for {
select {
case <- ctx . Done ( ) :
return nil , ctx . Err ( )
default :
}
n , readErr := reader . ReadRows ( rows )
for i := 0 ; i < n ; i ++ {
key := buildEqualityKey ( rows [ i ] , colIndices )
result [ key ] = struct { } { }
}
if readErr != nil {
if readErr == io . EOF {
break
}
return nil , readErr
}
}
return result , nil
}
// buildEqualityKey builds a composite string key from specific column values
// in a row. Each value is serialized as "kind:length:value" to avoid ambiguity
// between types (e.g., int 123 vs string "123") and to prevent collisions from
// values containing separator characters.
func buildEqualityKey ( row parquet . Row , colIndices [ ] int ) string {
if len ( colIndices ) == 1 {
v := row [ colIndices [ 0 ] ]
s := v . String ( )
return fmt . Sprintf ( "%d:%d:%s" , v . Kind ( ) , len ( s ) , s )
}
var b strings . Builder
for _ , idx := range colIndices {
v := row [ idx ]
s := v . String ( )
fmt . Fprintf ( & b , "%d:%d:%s" , v . Kind ( ) , len ( s ) , s )
}
return b . String ( )
}
// resolveEqualityColIndices maps Iceberg field IDs to Parquet column indices.
func resolveEqualityColIndices ( pqSchema * parquet . Schema , fieldIDs [ ] int , icebergSchema * iceberg . Schema ) ( [ ] int , error ) {
if len ( fieldIDs ) == 0 {
return nil , nil
}
colNameToIdx := make ( map [ string ] int )
for i , col := range pqSchema . Columns ( ) {
colNameToIdx [ strings . Join ( col , "." ) ] = i
}
indices := make ( [ ] int , len ( fieldIDs ) )
for i , fid := range fieldIDs {
field , ok := icebergSchema . FindFieldByID ( fid )
if ! ok {
return nil , fmt . Errorf ( "field ID %d not found in iceberg schema" , fid )
}
idx , ok := colNameToIdx [ field . Name ]
if ! ok {
return nil , fmt . Errorf ( "column %q (field ID %d) not found in parquet schema" , field . Name , fid )
}
indices [ i ] = idx
}
return indices , nil
}
// mergeParquetFiles reads multiple small Parquet files and merges them into
// a single Parquet file. Files are processed one at a time: each source file
// is loaded, its rows are streamed into the output writer, and then its data
// is released before the next file is loaded. This keeps peak memory
// proportional to the size of a single input file plus the output buffer,
// rather than the sum of all inputs.
// a single Parquet file, optionally filtering out rows matching position or
// equality deletes. Files are processed one at a time to keep memory usage
// proportional to a single input file plus the output buffer.
func mergeParquetFiles (
ctx context . Context ,
filerClient filer_pb . SeaweedFilerClient ,
bucketName , tablePath string ,
entries [ ] iceberg . ManifestEntry ,
positionDeletes map [ string ] [ ] int64 ,
eqDeleteGroups [ ] equalityDeleteGroup ,
icebergSchema * iceberg . Schema ,
) ( [ ] byte , int64 , error ) {
if len ( entries ) == 0 {
return nil , 0 , fmt . Errorf ( "no entries to merge" )
@ -502,15 +883,42 @@ func mergeParquetFiles(
return nil , 0 , fmt . Errorf ( "no parquet schema found in %s" , entries [ 0 ] . DataFile ( ) . FilePath ( ) )
}
// Resolve equality delete column indices for each group.
type resolvedEqGroup struct {
colIndices [ ] int
keys map [ string ] struct { }
}
var resolvedEqGroups [ ] resolvedEqGroup
if len ( eqDeleteGroups ) > 0 && icebergSchema != nil {
for _ , g := range eqDeleteGroups {
indices , resolveErr := resolveEqualityColIndices ( parquetSchema , g . FieldIDs , icebergSchema )
if resolveErr != nil {
firstReader . Close ( )
return nil , 0 , fmt . Errorf ( "resolve equality columns: %w" , resolveErr )
}
resolvedEqGroups = append ( resolvedEqGroups , resolvedEqGroup { colIndices : indices , keys : g . Keys } )
}
}
var outputBuf bytes . Buffer
writer := parquet . NewWriter ( & outputBuf , parquetSchema )
// drainReader streams all rows from reader into writer, then closes reader.
// source identifies the input file for error messages.
var totalRows int64
rows := make ( [ ] parquet . Row , 256 )
hasEqDeletes := len ( resolvedEqGroups ) > 0
// drainReader streams rows from reader into writer, filtering out deleted
// rows. source is the data file path (used for error messages and
// position delete lookups).
drainReader := func ( reader * parquet . Reader , source string ) error {
defer reader . Close ( )
// Normalize source path so it matches the normalized keys in positionDeletes.
normalizedSource := normalizeIcebergPath ( source , bucketName , tablePath )
posDeletes := positionDeletes [ normalizedSource ]
posDeleteIdx := 0
var absolutePos int64
for {
select {
case <- ctx . Done ( ) :
@ -519,10 +927,52 @@ func mergeParquetFiles(
}
n , readErr := reader . ReadRows ( rows )
if n > 0 {
if _ , writeErr := writer . WriteRows ( rows [ : n ] ) ; writeErr != nil {
return fmt . Errorf ( "write rows from %s: %w" , source , writeErr )
// Filter rows if we have any deletes
if len ( posDeletes ) > 0 || hasEqDeletes {
writeIdx := 0
for i := 0 ; i < n ; i ++ {
rowPos := absolutePos + int64 ( i )
// Check position deletes (sorted, so advance index)
if len ( posDeletes ) > 0 {
for posDeleteIdx < len ( posDeletes ) && posDeletes [ posDeleteIdx ] < rowPos {
posDeleteIdx ++
}
if posDeleteIdx < len ( posDeletes ) && posDeletes [ posDeleteIdx ] == rowPos {
posDeleteIdx ++
continue // skip this row
}
}
// Check equality deletes — each group independently
deleted := false
for _ , g := range resolvedEqGroups {
key := buildEqualityKey ( rows [ i ] , g . colIndices )
if _ , ok := g . keys [ key ] ; ok {
deleted = true
break
}
}
if deleted {
continue // skip this row
}
rows [ writeIdx ] = rows [ i ]
writeIdx ++
}
absolutePos += int64 ( n )
if writeIdx > 0 {
if _ , writeErr := writer . WriteRows ( rows [ : writeIdx ] ) ; writeErr != nil {
return fmt . Errorf ( "write rows from %s: %w" , source , writeErr )
}
totalRows += int64 ( writeIdx )
}
} else {
if _ , writeErr := writer . WriteRows ( rows [ : n ] ) ; writeErr != nil {
return fmt . Errorf ( "write rows from %s: %w" , source , writeErr )
}
totalRows += int64 ( n )
}
totalRows += int64 ( n )
}
if readErr != nil {
if readErr == io . EOF {
@ -577,6 +1027,16 @@ func mergeParquetFiles(
return outputBuf . Bytes ( ) , totalRows , nil
}
// compactRandomSuffix returns a short random hex string for use in artifact
// filenames to prevent collisions between concurrent runs.
func compactRandomSuffix ( ) string {
b := make ( [ ] byte , 4 )
if _ , err := rand . Read ( b ) ; err != nil {
return fmt . Sprintf ( "%x" , time . Now ( ) . UnixNano ( ) & 0xFFFFFFFF )
}
return hex . EncodeToString ( b )
}
// schemasEqual compares two parquet schemas structurally.
func schemasEqual ( a , b * parquet . Schema ) bool {
if a == b {