From d07043777e18e189796211e4671e34d7d8a3a5df Mon Sep 17 00:00:00 2001 From: ferhat elmas Date: Sun, 25 Jan 2026 22:13:39 +0100 Subject: [PATCH] fix(table): handle race for key map getter Signed-off-by: ferhat elmas --- table/scanner.go | 5 +++++ table/scanner_internal_test.go | 35 ++++++++++++++++++++++++++++++++++ 2 files changed, 40 insertions(+) diff --git a/table/scanner.go b/table/scanner.go index 8dfc238e1..ee12bf9ef 100644 --- a/table/scanner.go +++ b/table/scanner.go @@ -55,6 +55,11 @@ func (k *keyDefaultMap[K, V]) Get(key K) V { k.mx.Lock() defer k.mx.Unlock() + // race check between RLock and Lock + if v, ok := k.data[key]; ok { + return v + } + v := k.defaultFactory(key) k.data[key] = v diff --git a/table/scanner_internal_test.go b/table/scanner_internal_test.go index f0813fc99..af08b687e 100644 --- a/table/scanner_internal_test.go +++ b/table/scanner_internal_test.go @@ -18,6 +18,9 @@ package table import ( + "runtime" + "sync" + "sync/atomic" "testing" "github.com/apache/iceberg-go" @@ -100,3 +103,35 @@ func TestMinSequenceNum(t *testing.T) { }) } } + +func TestKeyDefaultMapRaceCondition(t *testing.T) { + var factoryCallCount atomic.Int64 + factory := func(key string) int { + factoryCallCount.Add(1) + runtime.Gosched() // to widen the race window + + return 42 + } + + kdm := newKeyDefaultMap(factory) + + var wg sync.WaitGroup + start := make(chan struct{}) + + numGoroutines := 1000 + for range numGoroutines { + wg.Add(1) + go func() { + defer wg.Done() + <-start + _ = kdm.Get("same-key") + }() + } + + close(start) + wg.Wait() + + callCount := factoryCallCount.Load() + assert.Equal(t, int64(1), callCount, + "factory should be called exactly once per key, but was called %d times", callCount) +}