Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 5 additions & 0 deletions table/scanner.go
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down
35 changes: 35 additions & 0 deletions table/scanner_internal_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,9 @@
package table

import (
"runtime"
"sync"
"sync/atomic"
"testing"

"github.com/apache/iceberg-go"
Expand Down Expand Up @@ -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)
}
Loading