This is an automated email from the ASF dual-hosted git repository.

zeroshade pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/iceberg-go.git


The following commit(s) were added to refs/heads/main by this push:
     new 0bd8477f fix(table): handle race for key map getter (#699)
0bd8477f is described below

commit 0bd8477fd960174bb35bb405bccab7d47af0e988
Author: ferhat elmas <[email protected]>
AuthorDate: Mon Jan 26 20:31:59 2026 +0100

    fix(table): handle race for key map getter (#699)
    
    * wasted resources (unnecessary calls to factory)
    * references can be different between goroutines that could cause
    inconsistency
    * if factory has side effects, it might happen multiple times
    
    Signed-off-by: ferhat elmas <[email protected]>
---
 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 8dfc238e..ee12bf9e 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 f0813fc9..af08b687 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)
+}

Reply via email to