Skip to content

Goroutine leak in MapExec when worker errors occur β€” no context cancellation for write pipelineΒ #796

@akarpovspl

Description

@akarpovspl

Apache Iceberg version

main (development)

Please describe the bug 🐞

Goroutine leak in MapExec when all worker goroutines error β€” no context cancellation

Summary

MapExec in table/internal/utils.go uses a plain errgroup.Group without context cancellation. When all worker goroutines return errors while the feeder still has items to produce, the feeder goroutine blocks forever on a channel send, preventing the output channel from ever closing. This deadlocks the consumer (the caller's goroutine) and leaks the feeder goroutine.

MapExec also does not accept a context.Context, so parent cancellation cannot propagate to in-flight operations.

Affected Code

Location Role
table/internal/utils.go:523 MapExec β€” the buggy function
table/writer.go:131 writeFiles calls MapExec for unpartitioned writes
table/arrow_utils.go:1348 recordsToDataFiles calls writeFiles
table/snapshot_producers.go:235 overwriteFiles.existingManifestEntries also calls MapExec

Reproduction Scenario

  1. Begin an unpartitioned table write that produces many WriteTask items via binPackRecords (more than 2 * nWorkers items).
  2. The write goes through recordsToDataFiles β†’ writeFiles β†’ MapExec with N workers.
  3. All N workers encounter errors (e.g., filesystem full, permission denied, network failure) before the feeder has drained its input iterator.
  4. Each worker returns its error and stops consuming from the input channel ch.
  5. The feeder goroutine fills the ch buffer (capacity N), then blocks on ch <- v because no goroutine is reading from ch.
  6. defer close(out) in the feeder never executes.
  7. The consumer blocks on for v := range out (waiting for out to close), or the drain loop for range out {} blocks if early exit was triggered.

Result: The feeder goroutine leaks. The consumer (caller's goroutine) hangs indefinitely.

Root Cause Analysis

// table/internal/utils.go:523-574

func MapExec[T, S any](nWorkers int, slice iter.Seq[T], fn func(T) (S, error)) iter.Seq2[S, error] {
    var g errgroup.Group          // ← (1) no context: first worker error does NOT signal others
    ch := make(chan T, nWorkers)
    out := make(chan S, nWorkers)

    for range nWorkers {
        g.Go(func() error {
            for v := range ch {
                result, err := fn(v)
                if err != nil {
                    return err       // ← (2) worker exits; stops reading from ch
                }
                out <- result
            }
            return nil
        })
    }

    var err error
    go func() {
        defer close(out)         // ← (4) never reached if feeder is stuck
        for v := range slice {
            ch <- v              // ← (3) blocks when ch buffer is full and all workers exited
        }
        close(ch)
        err = g.Wait()
    }()

    return func(yield func(S, error) bool) {
        defer func() {
            for range out {      // ← (5) drain also blocks forever (out never closes)
            }
        }()

        for v := range out {     // ← (5) blocks forever (out never closes)
            if !yield(v, nil) {
                return
            }
        }

        if err != nil {
            var z S
            yield(z, err)
        }
    }
}

Step-by-step for nWorkers=2, items [A, B, C, D, E]:

  1. Feeder sends A, B to ch (fills buffer). Workers 1 and 2 each pick one item.
  2. Feeder sends C to ch (slot freed by worker pick-up). Worker 1 errors on A, exits.
  3. Feeder sends D to ch (slot freed by worker 2). Worker 2 errors on B, exits. Both workers have returned.
  4. ch contains [C, D] β€” full buffer, no readers.
  5. Feeder tries ch <- E. Blocks forever.
  6. close(out) never fires β†’ consumer blocks on for v := range out forever.

Contrast with Correct Pattern

readAllDeleteFiles in table/arrow_scanner.go:73 correctly uses context-aware errgroup:

g, ctx := errgroup.WithContext(ctx)
g.SetLimit(concurrency)

When one goroutine fails, the derived context is cancelled, and other goroutines can observe the cancellation and exit promptly. Any code that selects on ctx.Done() can unblock.

Suggested Fix

  1. Add a context.Context parameter to MapExec.
  2. Use errgroup.WithContext(ctx) so that worker errors cancel the derived context.
  3. Use select with ctx.Done() in both the feeder and workers to unblock on cancellation.
  4. Update all callers (writeFiles, existingManifestEntries) to pass their context.
func MapExec[T, S any](ctx context.Context, nWorkers int, slice iter.Seq[T], fn func(T) (S, error)) iter.Seq2[S, error] {
    if nWorkers <= 0 {
        nWorkers = runtime.GOMAXPROCS(0)
    }

    g, ctx := errgroup.WithContext(ctx)
    ch := make(chan T, nWorkers)
    out := make(chan S, nWorkers)

    for range nWorkers {
        g.Go(func() error {
            for v := range ch {
                result, err := fn(v)
                if err != nil {
                    return err
                }
                select {
                case out <- result:
                case <-ctx.Done():
                    return context.Cause(ctx)
                }
            }
            return nil
        })
    }

    var err error
    go func() {
        defer close(out)
        for v := range slice {
            select {
            case ch <- v:
            case <-ctx.Done():
                // Context cancelled (worker errored or parent cancelled).
                // Returning from range-over-func body without calling yield
                // stops the iterator, so we just need to exit this loop.
                // With range-over-func, a bare "return" from this closure
                // signals the iterator to stop.
                close(ch)
                err = g.Wait()
                return
            }
        }
        close(ch)
        err = g.Wait()
    }()

    return func(yield func(S, error) bool) {
        defer func() {
            for range out {
            }
        }()

        for v := range out {
            if !yield(v, nil) {
                return
            }
        }

        if err != nil {
            var z S
            yield(z, err)
        }
    }
}

Key point: in the feeder goroutine, we cannot use break inside a select to exit a for loop β€” break only breaks the select statement. Use return (the feeder is an anonymous goroutine, so returning exits it) with appropriate cleanup.

Secondary Issue: AllManifests errgroup without context

Table.AllManifests in table/table.go:141 also uses a plain errgroup.Group{}. However, unlike MapExec, this does not cause a goroutine leak or deadlock:

  • A separate goroutine calls g.Wait() and then close(ch) (via defer), which causes MakeSequencedChan to close its output, and the consumer's select loop exits.

The real issue here is data correctness: when a goroutine for snapshot index i errors without sending to ch, MakeSequencedChan never sees index i. Any successfully-processed items with index > i are buffered in the internal priority queue but never emitted (because isNext requires consecutive indices). Those results are silently dropped when the source channel closes. The consumer may return only a partial result set without indicating which snapshots were skipped.

Adding errgroup.WithContext would still be an improvement here β€” it would short-circuit remaining goroutines on first failure rather than letting them all run to completion.

Existing Test Coverage

TestMapExecFinish in table/internal/utils_test.go:75 only tests the happy path (all items succeed). There is no test for the error path. A test that makes all workers return errors with a large input set would reproduce the deadlock (the test would hang).

Environment

  • iceberg-go at current main branch
  • Go 1.22+ (iter/range-over-func)

Metadata

Metadata

Assignees

No one assigned

    Labels

    No labels
    No labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions