Skip to content

Conversation

@jensneuse
Copy link
Member

@jensneuse jensneuse commented Oct 25, 2025

  • memory management improved through arenas
  • moved request deduplication from transport layer to engine
  • UniqueRequestID is no longer needed, we pre-compute a hash for the headers outside of the Engine and can supply it via HeadersBuilder interface through the RequestContext to the engine

Summary by CodeRabbit

  • New Features

    • Single-flight deduplication for inbound GraphQL and subgraph requests.
    • Subscription triggers include source metadata and propagate HTTP headers to data sources.
  • Improvements

    • Arena-based memory management for reduced allocations and better performance.
    • Data-source calls are header-aware and return payloads directly.
  • Refactor

    • Removed parallel-list fetch path.
  • Chores

    • Updated dependency constraints and added arena pooling dependency.

✏️ Tip: You can customize this high-level summary in your review settings.

@coderabbitai
Copy link
Contributor

coderabbitai bot commented Oct 25, 2025

Walkthrough

Refactors datasource APIs to accept HTTP headers and return []byte, integrates arena.Arena across resolver/json builders, adds sharded single‑flight deduplication for inbound/subgraph requests, removes xxhash-based subscription UniqueRequestID logic, and removes ParallelListItemFetch support. (50 words)

Changes

Cohort / File(s) Summary
Go module
v2/go.mod
Updated astjson to v1.0.0 and added github.com/wundergraph/go-arena v1.1.0.
Datasource API & HTTP client
v2/pkg/engine/datasource/...
v2/pkg/engine/datasource/httpclient/nethttpclient.go, .../staticdatasource/static_datasource.go, .../introspection_datasource/source.go
Load/LoadWithFiles signatures changed to accept http.Header and return (data []byte, err error); httpclient Do/DoMultipartForm/makeHTTPRequest now return []byte, add ContentLengthHeader and per-context size-hint helpers; multipart/request-building flows return assembled bytes.
gRPC datasource & JSON builder
v2/pkg/engine/datasource/grpc_datasource/...
DataSource gains arena.Pool; per-call arena acquire/release added; Load methods updated to headers+byte returns; jsonBuilder now holds jsonArena (arena.Arena) and builder/marshal functions use builder-held arena.
GraphQL subscription & client
v2/pkg/engine/datasource/graphql_datasource/*
Removed xxhash-based UniqueRequestID and SSE-specific hash logic; Start/AsyncStart now accept and propagate http.Header; tests adjusted; GraphQLSubscriptionTrigger extended with SourceName and SourceID.
Resolver, Loader & arena integration
v2/pkg/engine/resolve/*, v2/pkg/fastjsonext/*, v2/pkg/astvisitor/visitor.go
Introduced arena.Arena broadly: Resolver/Loader/Resolvable/fastjsonext use arenas; added arena pools, arena-aware parse/merge APIs; Response/Loader storage switched to []byte and arena-aware helpers added; NewResolvable signature updated to accept arena/context.
Single‑flight deduplication
v2/pkg/engine/resolve/subgraph_request_singleflight.go, .../inbound_request_singleflight.go, .../subgraph_request_singleflight_test.go
Added sharded SubgraphRequestSingleFlight and InboundRequestSingleFlight with GetOrCreate/Finish semantics, per-fetch size hints and rolling-window aggregation; tests added for leader/follower and size-hint behavior.
Loader/Resolver wiring & headers
v2/pkg/engine/resolve/loader.go, v2/pkg/engine/resolve/resolve.go, v2/pkg/engine/resolve/context.go
Loader/Resolver gained arena and single‑flight fields; Context adds SubgraphHeadersBuilder and VariablesHash; helpers for headers-for-subgraph and load-by-context added; response assembly flows adapted to returned []byte.
Input templating & writer abstraction
v2/pkg/engine/resolve/inputtemplate.go
Introduced InputTemplateWriter interface to replace *bytes.Buffer across template rendering APIs.
Fetch model simplification
v2/pkg/engine/resolve/fetch.go, v2/pkg/engine/postprocess/create_concrete_single_fetch_types.go, v2/pkg/engine/resolve/fetchtree.go
Removed ParallelListItemFetch, FetchKindParallelListItem, and all parallel-list handling and creation logic.
Variables parsing & small fixes
v2/pkg/astnormalization/uploads/upload_finder.go, v2/pkg/variablesvalidation/variablesvalidation.go, v2/pkg/engine/resolve/tainted_objects_test.go
Replaced astjson.ParseBytesWithoutCache with astjson.ParseBytes in multiple places.
Mocks, tests & fixtures
many v2/pkg/engine/**/*_test.go, v2/pkg/fastjsonext/fastjsonext_test.go, v2/pkg/engine/datasource/introspection_datasource/fixtures/*, execution/engine/testdata/*
Updated tests and mocks to new Load/Start signatures (headers + []byte returns), NewResolvable call sites, arena-aware expectations, and minor golden-file newline changes; added subgraph singleflight tests.

Sequence Diagram(s)

sequenceDiagram
    participant Client
    participant Resolver
    participant SF as SubgraphSingleFlight
    participant DataSource

    Client->>Resolver: GraphQL request (headers, variables)
    Resolver->>SF: GetOrCreateItem(fetchItem, input, extraKey)
    alt follower (shared)
        SF-->>Resolver: existing item (shared)
        Resolver->>Resolver: wait on item.loaded
    else leader (new)
        SF-->>Resolver: new item (leader)
        Resolver->>DataSource: Load(ctx, headers, input)
        DataSource-->>Resolver: (data []byte, err)
        Resolver->>SF: Finish(item) / FinishOk(item, data)
    end
    Resolver-->>Client: Response (assembled with arena)
Loading
sequenceDiagram
    participant App
    participant Resolver
    participant ArenaPool
    participant Loader
    participant DataSource

    App->>Resolver: ArenaResolveGraphQLResponse(ctx, response)
    Resolver->>ArenaPool: Acquire()
    ArenaPool-->>Resolver: arena
    Resolver->>Loader: Init with arena
    Loader->>DataSource: Load(ctx, headers, input)
    DataSource-->>Loader: (data []byte, err)
    Loader->>Loader: Parse & Merge (with arena)
    Loader->>Resolver: assembled response bytes
    Resolver->>ArenaPool: Release(arena)
    Resolver-->>App: Write response
Loading

Estimated code review effort

🎯 5 (Critical) | ⏱️ ~120 minutes

Areas needing extra attention:

  • Arena lifecycle: correctness of Acquire/Release across Resolver, Loader, Resolvable, jsonBuilder to avoid use-after-free.
  • Concurrency and correctness of SubgraphRequestSingleFlight and InboundRequestSingleFlight (sharding, leader/follower semantics, channel close/removal).
  • Consistent propagation of http.Header through Load/Start/AsyncStart call sites, tests, and subscription wiring.
  • Migration from writer-based outputs to returned []byte across codebase and mocks (error paths and streaming assumptions).
  • Ensure removal of ParallelListItemFetch left no dangling references in planning/postprocessing.

Possibly related PRs

Pre-merge checks and finishing touches

✅ Passed checks (2 passed)
Check name Status Explanation
Description Check ✅ Passed Check skipped - CodeRabbit’s high-level summary is enabled.
Title check ✅ Passed The title 'feat: improve memory management & request deduplication' accurately reflects the main changes: arena-based memory management improvements and request deduplication infrastructure (SubgraphRequestSingleFlight and InboundRequestSingleFlight).
✨ Finishing touches
  • 📝 Generate docstrings
🧪 Generate unit tests (beta)
  • Create PR with unit tests
  • Post copyable unit tests in a comment
  • Commit unit tests in branch feat/improve-memory-usage-with-arenas

Comment @coderabbitai help to get the list of available commands and usage tips.

Copy link
Contributor

@coderabbitai coderabbitai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actionable comments posted: 15

Caution

Some comments are outside the diff and can’t be posted inline due to platform limitations.

⚠️ Outside diff range comments (6)
v2/pkg/engine/resolve/inputtemplate.go (3)

161-161: Fix unchecked error to resolve pipeline failure.

The errcheck linter flags this line because WriteString can return an error. With the interface abstraction, implementations might fail, unlike *bytes.Buffer which never returns errors.

Apply this diff to handle the error:

-	preparedInput.WriteString(value[0])
-	return nil
+	_, err := preparedInput.WriteString(value[0])
+	return err

168-168: Handle WriteString error for consistency.

Similar to line 161, this WriteString call ignores potential errors. Although not flagged by the pipeline (possibly because the linter stopped at the first error), it should be handled consistently.

Apply this diff to handle the error:

-		preparedInput.WriteString(value[j])
+		if _, err := preparedInput.WriteString(value[j]); err != nil {
+			return err
+		}

39-51: Update caller type compatibility in loader.go

The signature change to InputTemplateWriter introduces type mismatches at both call sites. The callers in loader.go still initialize preparedInput as *bytes.Buffer and pass it to SetInputUndefinedVariables, which now requires InputTemplateWriter:

  • Line 1340 & 1386: preparedInput := bytes.NewBuffer(nil) passed to function expecting InputTemplateWriter
  • Line 1428 & 1500: preparedInput := bytes.NewBuffer(make([]byte, 0, 64)) passed to function expecting InputTemplateWriter

Either revert the signature change to *bytes.Buffer or update the callers in loader.go to use InputTemplateWriter.

v2/pkg/engine/resolve/loader_test.go (1)

1029-1049: Fix benchmark assertion to match new response format.

The benchmark expects a response with an "errors" field, but the new response format no longer includes an empty errors array by default. The pipeline failure indicates this assertion is failing.

Update the expected output to match the actual response format:

-	expected := `{"errors":[],"data":{"topProducts":[{"name":"Table","__typename":"Product","upc":"1","reviews":[{"body":"Love Table!","author":{"__typename":"User","id":"1","name":"user-1"}},{"body":"Prefer other Table.","author":{"__typename":"User","id":"2","name":"user-2"}}],"stock":8},{"name":"Couch","__typename":"Product","upc":"2","reviews":[{"body":"Couch Too expensive.","author":{"__typename":"User","id":"1","name":"user-1"}}],"stock":2},{"name":"Chair","__typename":"Product","upc":"3","reviews":[{"body":"Chair Could be better.","author":{"__typename":"User","id":"2","name":"user-2"}}],"stock":5}]}}`
+	expected := `{"data":{"topProducts":[{"name":"Table","__typename":"Product","upc":"1","reviews":[{"body":"Love Table!","author":{"__typename":"User","id":"1","name":"user-1"}},{"body":"Prefer other Table.","author":{"__typename":"User","id":"2","name":"user-2"}}],"stock":8},{"name":"Couch","__typename":"Product","upc":"2","reviews":[{"body":"Couch Too expensive.","author":{"__typename":"User","id":"1","name":"user-1"}}],"stock":2},{"name":"Chair","__typename":"Product","upc":"3","reviews":[{"body":"Chair Could be better.","author":{"__typename":"User","id":"2","name":"user-2"}}],"stock":5}]}}`
v2/pkg/engine/resolve/resolvable.go (1)

109-121: Non-nil arena precondition across parse/merge/error paths

These sites assume r.astjsonArena is non-nil (ParseBytesWithArena, MergeValues, SetArrayItem, ArrayValue, AppendError*). This is fine if callers set the arena first; otherwise it’s crash-prone. The fix is in resolve.go (ensure arena is acquired and injected before Init), see my comment there.

Also applies to: 470-479, 862-863, 1198-1212, 1286-1295, 1298-1312

v2/pkg/engine/datasource/grpc_datasource/json_builder.go (1)

252-257: Avoid mutating message.Fields via slice aliasing.

validFields := message.Fields followed by append(...) can mutate the underlying slice, causing duplicated fields across calls.

Apply:

- validFields := message.Fields
+ validFields := append(RPCFields(nil), message.Fields...)
 if message.IsOneOf() {
-  validFields = append(validFields, message.FieldSelectionSet.SelectFieldsForTypes(
+  validFields = append(validFields, message.FieldSelectionSet.SelectFieldsForTypes(
      message.SelectValidTypes(string(data.Type().Descriptor().Name())))...)
 }

Replace RPCFields with the actual slice type alias (e.g., []RPCField) if different.

🧹 Nitpick comments (32)
v2/pkg/engine/resolve/inputtemplate.go (1)

47-47: Consider handling Write errors consistently.

These Write calls ignore potential errors. While not flagged by the pipeline, handling errors consistently would improve robustness, especially since the interface abstraction now allows for implementations that might fail.

Consider handling errors from all Write calls similar to the WriteString fixes. For example, at line 84:

-		_, _ = preparedInput.Write(segment.Data)
+		if _, err = preparedInput.Write(segment.Data); err != nil {
+			return err
+		}

Apply similar patterns to lines 47, 106, 123, 144, and 166 if error handling is desired.

Also applies to: 84-84, 106-106, 123-123, 144-144, 166-166

v2/pkg/engine/resolve/context.go (2)

39-48: Clarify SubgraphHeadersBuilder contract

Add a short doc that the returned uint64 is the dedup scope key (stable within a request, distinct across requests/users) and that headers must be treated as immutable.

-type SubgraphHeadersBuilder interface {
-    HeadersForSubgraph(subgraphName string) (http.Header, uint64)
-}
+// SubgraphHeadersBuilder provides per-subgraph request headers and a dedup scope key.
+// The returned header must be treated as immutable by callers.
+// The returned uint64 should uniquely scope deduplication (e.g., request ID),
+// so concurrent operations with different auth headers do not deduplicate together.
+type SubgraphHeadersBuilder interface {
+    HeadersForSubgraph(subgraphName string) (http.Header, uint64)
+}

Confirm all call sites of HeadersForSubgraphRequest use the returned uint64 as the SingleFlight extraKey.


211-224: Hygiene: reset Request.ID and SubgraphHeadersBuilder on Free()

Minor cleanup to avoid leaking state when Contexts are pooled/reused.

 func (c *Context) Free() {
   c.ctx = nil
   c.Variables = nil
   c.Files = nil
   c.Request.Header = nil
+  c.Request.ID = 0
   c.RenameTypeNames = nil
   c.RemapVariables = nil
   c.TracingOptions.DisableAll()
   c.Extensions = nil
   c.subgraphErrors = nil
   c.authorizer = nil
   c.LoaderHooks = nil
+  c.SubgraphHeadersBuilder = nil
 }
v2/pkg/engine/resolve/singleflight.go (3)

16-23: Remove unused cleanup channel

cleanup chan func() is never used; drop it to reduce noise.

 type SingleFlight struct {
     mu      *sync.RWMutex
     items   map[uint64]*SingleFlightItem
     sizes   map[uint64]*fetchSize
     xxPool  *sync.Pool
-    cleanup chan func()
 }
@@
 func NewSingleFlight() *SingleFlight {
     return &SingleFlight{
         items: make(map[uint64]*SingleFlightItem),
         sizes: make(map[uint64]*fetchSize),
         mu:    new(sync.RWMutex),
         xxPool: &sync.Pool{
             New: func() any {
                 return xxhash.New()
             },
         },
-        cleanup: make(chan func()),
     }
 }

Also applies to: 29-41


118-136: Size hint rollover: consider EMA for smoother adaptation (optional)

Current reset at 50 samples turns the moving average into ~50% weight on the last sample. An exponential moving average would be smoother under traffic variance.

If desired, replace the rollover with avg = alpha*avg + (1-alpha)*len(response) stored as fixed‑point to avoid floats.


75-83: Hash extraKey into the digest instead of adding it to reduce collision risk

Currently, sfKey is computed as h.Sum64() + extraKey, which can collide (e.g., 100+50 and 75+75 both yield 150). Since extraKey comes from request headers via HeadersForSubgraph, it should be mixed into the hash to ensure distinct headers produce distinct keys.

@@
-import (
-    "sync"
-
-    "github.com/cespare/xxhash/v2"
-)
+import (
+    "encoding/binary"
+    "sync"
+
+    "github.com/cespare/xxhash/v2"
+)
@@
-func (s *SingleFlight) sfKey(h *xxhash.Digest, fetchItem *FetchItem, input []byte, extraKey uint64) uint64 {
+func (s *SingleFlight) sfKey(h *xxhash.Digest, fetchItem *FetchItem, input []byte, extraKey uint64) uint64 {
     if fetchItem != nil && fetchItem.Fetch != nil {
         info := fetchItem.Fetch.FetchInfo()
         if info != nil {
             _, _ = h.WriteString(info.DataSourceID)
             _, _ = h.WriteString(":")
         }
     }
     _, _ = h.Write(input)
-    return h.Sum64() + extraKey
+    var ek [8]byte
+    binary.LittleEndian.PutUint64(ek[:], extraKey)
+    _, _ = h.Write(ek[:])
+    return h.Sum64()
 }

Also applies to: 85-96

v2/pkg/engine/datasource/introspection_datasource/source_test.go (1)

31-41: Return-based Load usage looks good; simplify newline trim.
The switch to returning responseData is correct. You can simplify the newline removal from json.Indent with TrimSuffix for clarity.

-// Trim the trailing newline that json.Indent adds
-responseBytes := actualResponse.Bytes()
-if len(responseBytes) > 0 && responseBytes[len(responseBytes)-1] == '\n' {
-  responseBytes = responseBytes[:len(responseBytes)-1]
-}
+// Trim the trailing newline that json.Indent adds
+responseBytes := bytes.TrimSuffix(actualResponse.Bytes(), []byte{'\n'})
v2/pkg/variablesvalidation/variablesvalidation.go (1)

101-101: Normalize empty/null variables to "{}" before parsing.
To match GraphQL semantics and mirror UploadFinder, guard nil/empty/"null" variables so ParseBytes never errors on absent variables.

 func (v *VariablesValidator) Validate(operation, definition *ast.Document, variables []byte) error {
   v.visitor.definition = definition
   v.visitor.operation = operation
-  v.visitor.variables, v.visitor.err = astjson.ParseBytes(variables)
+  // Normalize absent variables to {}
+  if variables == nil || bytes.Equal(variables, []byte("null")) || bytes.Equal(variables, []byte("")) {
+    variables = []byte("{}")
+  }
+  v.visitor.variables, v.visitor.err = astjson.ParseBytes(variables)
   if v.visitor.err != nil {
     return v.visitor.err
   }

Please confirm callers never pass nil/empty variables inadvertently. If they do, add/keep this guard to prevent regressions.

v2/pkg/engine/resolve/resolvable_test.go (1)

15-15: Constructor updates LGTM; consider a tiny test helper to DRY.
All call sites now pass the arena argument (nil) correctly. Optionally add a newTestResolvable(opts) helper to cut duplication.

func newTestResolvable(opts ResolvableOptions) *Resolvable { return NewResolvable(nil, opts) }

Also applies to: 87-87, 160-160, 234-234, 262-262, 337-337, 373-373, 443-443, 473-473, 504-504, 552-552, 626-626, 656-656, 690-690, 722-722, 758-758, 841-841, 926-926, 953-953, 976-976, 998-998, 1021-1021, 1049-1049, 1147-1147, 1245-1245, 1345-1345

v2/pkg/engine/resolve/variables_renderer.go (1)

350-356: Use sync.Pool.New to remove the nil-branch; ensure Reset clears arena.
Minor micro-optimization: initialize the pool with New to simplify getResolvable.

-var (
-  _graphQLVariableResolveRendererPool = &sync.Pool{}
-)
+var (
+  _graphQLVariableResolveRendererPool = &sync.Pool{
+    New: func() any { return NewResolvable(nil, ResolvableOptions{}) },
+  }
+)

 func (g *GraphQLVariableResolveRenderer) getResolvable() *Resolvable {
-  v := _graphQLVariableResolveRendererPool.Get()
-  if v == nil {
-    return NewResolvable(nil, ResolvableOptions{})
-  }
-  return v.(*Resolvable)
+  return _graphQLVariableResolveRendererPool.Get().(*Resolvable)
 }

Please confirm Resolvable.Reset releases any arena-backed state (if ever set on this path), so pooled instances don’t retain large allocations between uses.

v2/pkg/engine/datasource/staticdatasource/static_datasource.go (1)

78-80: Consider returning an error instead of panicking.

The panic for unimplemented functionality could cause runtime crashes if this method is accidentally called. Consider returning a descriptive error instead:

-func (Source) LoadWithFiles(ctx context.Context, headers http.Header, input []byte, files []*httpclient.FileUpload) (data []byte, err error) {
-	panic("not implemented")
-}
+func (Source) LoadWithFiles(ctx context.Context, headers http.Header, input []byte, files []*httpclient.FileUpload) (data []byte, err error) {
+	return nil, fmt.Errorf("static data source does not support file uploads")
+}

Note: You'll need to add "fmt" to the imports if you adopt this change.

v2/pkg/engine/datasource/graphql_datasource/graphql_datasource_test.go (1)

8455-8455: Updated Start signature: ensure nil-safe header handling

Calls now use Start(ctx, headers, options, updater) with headers=nil. Verify the implementation treats nil as empty and does not mutate the provided map. Consider adding one test that passes non-empty headers and asserts they reach the upstream.

Also applies to: 8461-8461, 8474-8474, 8486-8486, 8504-8504, 8527-8527, 8591-8591, 8611-8611, 8635-8635

v2/pkg/engine/datasource/graphql_datasource/graphql_datasource.go (2)

1957-1964: Avoid clobbering configured headers; prefer merge with precedence.

Overwriting options.Header loses statically configured subscription headers. Merge instead, letting passed headers override duplicates.

Apply:

 err := json.Unmarshal(input, &options)
 if err != nil {
   return err
 }
- options.Header = headers
+ // merge configured headers with passed headers; passed headers take precedence
+ if options.Header == nil {
+   options.Header = make(http.Header)
+ }
+ for k, v := range headers {
+   options.Header[k] = v
+ }

1977-1984: Same header-merge concern as AsyncStart.

Use the same merge strategy here to preserve configured headers.

- options.Header = headers
+ if options.Header == nil {
+   options.Header = make(http.Header)
+ }
+ for k, v := range headers {
+   options.Header[k] = v
+ }
v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go (1)

154-156: Do not panic; return a proper error for unsupported file uploads.

Panics in datasource paths can crash the process. Return an error or GraphQL error JSON.

-func (d *DataSource) LoadWithFiles(ctx context.Context, headers http.Header, input []byte, files []*httpclient.FileUpload) (data []byte, err error) {
-  panic("unimplemented")
-}
+func (d *DataSource) LoadWithFiles(ctx context.Context, headers http.Header, input []byte, files []*httpclient.FileUpload) (data []byte, err error) {
+  // gRPC transport does not support multipart uploads
+  return nil, fmt.Errorf("gRPC datasource: file uploads are not supported")
+}
v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go (1)

237-239: Remove leftover debug printing.

Avoid noisy stdout in tests.

- bytes := output
- fmt.Println(string(bytes))
+ bytes := output
v2/pkg/engine/datasource/pubsub_datasource/pubsub_kafka.go (2)

59-61: Avoid panic; return unsupported operation error.

Never panic in datasource code paths.

-func (s *KafkaPublishDataSource) LoadWithFiles(ctx context.Context, headers http.Header, input []byte, files []*httpclient.FileUpload) (data []byte, err error) {
-  panic("not implemented")
-}
+func (s *KafkaPublishDataSource) LoadWithFiles(ctx context.Context, headers http.Header, input []byte, files []*httpclient.FileUpload) (data []byte, err error) {
+  return nil, fmt.Errorf("kafka publish: file uploads are not supported")
+}

46-57: Choose one error strategy: return error or JSON status, not both.

Returning non-nil error and a JSON body can cause double-handling upstream. Prefer nil error with explicit JSON or return only error.

Option A (error-only):

- if err := s.pubSub.Publish(ctx, publishConfiguration); err != nil {
-   return []byte(`{"success": false}`), err
- }
- return []byte(`{"success": true}`), nil
+ if err := s.pubSub.Publish(ctx, publishConfiguration); err != nil {
+   return nil, err
+ }
+ return []byte(`{"success": true}`), nil

Option B (JSON-only, nil error):

- if err := s.pubSub.Publish(ctx, publishConfiguration); err != nil {
-   return []byte(`{"success": false}`), err
- }
+ if err := s.pubSub.Publish(ctx, publishConfiguration); err != nil {
+   return []byte(`{"success": false","error":"publish failed"}`), nil
+ }

Pick consistently across datasources.

Please confirm the engine’s expectation for datasource error handling so we align consistently.

v2/pkg/engine/resolve/resolvable.go (1)

81-100: Reset clears the arena; ensure it is set before next Init/Resolve

Reset sets r.astjsonArena = nil. Any subsequent Init/Resolve must set the arena first, or calls like astjson.ObjectValue/ParseBytesWithArena will likely panic or mis-allocate. Add a comment or a guard, and ensure all call sites set the arena before use.
Would you like me to add a SetArena(a arena.Arena) helper and guard Init when arena is nil?

v2/pkg/engine/resolve/resolve.go (3)

1089-1114: Set arena in subscription SkipLoader path for consistency

SkipLoader still constructs Resolvable with nil arena. It often works, but error/value-completion paths would allocate via arena. Acquire/assign an arena as in ResolveGraphQLResponse to avoid surprises.

Example:

- t := newTools(r.options, r.allowedErrorExtensionFields, r.allowedErrorFields, r.sf)
+ t := newTools(r.options, r.allowedErrorExtensionFields, r.allowedErrorFields, r.sf)
+ resolveArena := r.resolveArenaPool.Acquire(ctx.Request.ID)
+ t.loader.jsonArena = resolveArena.Arena
+ t.resolvable.astjsonArena = resolveArena.Arena
+ defer r.resolveArenaPool.Release(ctx.Request.ID, resolveArena)

1199-1224: Also set arena in Async subscription SkipLoader path

Same rationale as above; apply the same acquire/assign/release here.


1168-1174: Improve uniqueID mixing: write headersHash into hasher instead of adding

Adding 64-bit hashes can increase collisions. Prefer feeding headersHash bytes into xxhash and use Sum64().

Suggested change (add import "encoding/binary"):

+ import "encoding/binary"
@@
- _, _ = xxh.Write(input)
- // the hash for subgraph headers is pre-computed
- // we can just add it to the input hash to get a unique id
- uniqueID := xxh.Sum64() + headersHash
+ _, _ = xxh.Write(input)
+ var hb [8]byte
+ binary.LittleEndian.PutUint64(hb[:], headersHash)
+ _, _ = xxh.Write(hb[:])
+ uniqueID := xxh.Sum64()

Also applies to: 1226-1233

v2/pkg/engine/datasource/pubsub_datasource/pubsub_nats.go (2)

43-51: Headers currently unused

Start receives headers but the underlying NatsPubSub.Subscribe doesn’t. If headers are intentionally ignored for NATS, add a short comment; otherwise consider plumbing usage or dropping the param in this source.


79-93: LGTM: request path returns data slice and propagates error

Readability nit: rename subscriptionConfiguration -> requestConfiguration. Optional.

v2/pkg/engine/resolve/resolve_test.go (1)

112-136: Remove or use unused parameter enableSingleFlight.

The enableSingleFlight bool is unused in testFn; either wire it to options or remove it to avoid confusion.

v2/pkg/engine/datasource/httpclient/nethttpclient.go (4)

315-325: Confirm FileUpload lifetime; avoid deleting user files.

The defer removes every opened file (os.Remove). If FileUpload.Path() can point to non-temporary user files, this deletes user data.

Option: only delete when FileUpload is marked temporary, or move cleanup behind an explicit flag.

- if err = os.Remove(file.Name()); err != nil {
+ if fileShouldBeDeleted(file) { // e.g., a field or method on FileUpload
+   if err = os.Remove(file.Name()); err != nil {
       return
- }
+   }
+ }

Please confirm the contract of FileUpload.Path().


232-241: Close decompressor readers after use to free resources.

gzip/deflate readers implement io.ReadCloser. Close them after ReadFrom to release resources early.

 out := buffer(ctx)
-_, err = out.ReadFrom(respReader)
+_, err = out.ReadFrom(respReader)
 if err != nil {
   return nil, err
 }
+if rc, ok := respReader.(io.ReadCloser); ok {
+  _ = rc.Close()
+}

209-216: Prefer a single Accept-Encoding value.

Set “gzip, deflate” once instead of two separate headers; clearer and avoids duplicate values.

-request.Header.Set(AcceptEncodingHeader, EncodingGzip)
-request.Header.Add(AcceptEncodingHeader, EncodingDeflate)
+request.Header.Set(AcceptEncodingHeader, EncodingGzip+", "+EncodingDeflate)

238-265: Trace injection assumes JSON object; guard or fallback for non-objects.

jsonparser.Set will fail for non-object bodies (arrays, scalars). Consider detecting object first; otherwise return original data and put trace into ResponseContext only.

v2/pkg/engine/datasource/grpc_datasource/json_builder.go (2)

406-415: Clarify nesting bound check.

if level > md.NestingLevel likely intends >=; using >= avoids an unnecessary extra frame and is easier to reason about.

- if level > md.NestingLevel {
+ if level >= md.NestingLevel {
   return current, nil
 }

114-121: Consider exposing a Reset to reuse jsonBuilder arenas safely.

If jsonBuilder instances are reused, add a Reset method to re-init jsonArena (or allocate per call) to avoid unbounded growth.

v2/pkg/engine/resolve/loader.go (1)

432-446: Consider using arena for array creation.

The function now accepts an arena parameter and uses it for SetArrayItem, but the array itself is created with MustParseBytes (line 441) without the arena. For consistency and to fully leverage arena benefits, consider:

-	arr := astjson.MustParseBytes([]byte(`[]`))
+	arr := astjson.ArrayValue(a)
	for i, item := range items {
		arr.SetArrayItem(a, i, item)
	}
📜 Review details

Configuration used: Path: .coderabbit.yaml

Review profile: CHILL

Plan: Pro

📥 Commits

Reviewing files that changed from the base of the PR and between 7c4acc9 and 60b5c3b.

⛔ Files ignored due to path filters (2)
  • go.work.sum is excluded by !**/*.sum
  • v2/go.sum is excluded by !**/*.sum
📒 Files selected for processing (45)
  • v2/go.mod (1 hunks)
  • v2/pkg/astnormalization/uploads/upload_finder.go (1 hunks)
  • v2/pkg/engine/datasource/graphql_datasource/graphql_datasource.go (3 hunks)
  • v2/pkg/engine/datasource/graphql_datasource/graphql_datasource_test.go (14 hunks)
  • v2/pkg/engine/datasource/graphql_datasource/graphql_subscription_client.go (0 hunks)
  • v2/pkg/engine/datasource/graphql_datasource/graphql_subscription_client_test.go (0 hunks)
  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go (5 hunks)
  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go (25 hunks)
  • v2/pkg/engine/datasource/grpc_datasource/json_builder.go (15 hunks)
  • v2/pkg/engine/datasource/httpclient/httpclient_test.go (2 hunks)
  • v2/pkg/engine/datasource/httpclient/nethttpclient.go (9 hunks)
  • v2/pkg/engine/datasource/introspection_datasource/fixtures/schema_introspection.golden (1 hunks)
  • v2/pkg/engine/datasource/introspection_datasource/fixtures/schema_introspection_with_custom_root_operation_types.golden (1 hunks)
  • v2/pkg/engine/datasource/introspection_datasource/fixtures/type_introspection.golden (1 hunks)
  • v2/pkg/engine/datasource/introspection_datasource/source.go (3 hunks)
  • v2/pkg/engine/datasource/introspection_datasource/source_test.go (1 hunks)
  • v2/pkg/engine/datasource/pubsub_datasource/pubsub_datasource_test.go (4 hunks)
  • v2/pkg/engine/datasource/pubsub_datasource/pubsub_kafka.go (3 hunks)
  • v2/pkg/engine/datasource/pubsub_datasource/pubsub_nats.go (3 hunks)
  • v2/pkg/engine/datasource/staticdatasource/static_datasource.go (2 hunks)
  • v2/pkg/engine/plan/planner_test.go (2 hunks)
  • v2/pkg/engine/plan/visitor.go (1 hunks)
  • v2/pkg/engine/resolve/arena.go (1 hunks)
  • v2/pkg/engine/resolve/authorization_test.go (3 hunks)
  • v2/pkg/engine/resolve/context.go (2 hunks)
  • v2/pkg/engine/resolve/datasource.go (1 hunks)
  • v2/pkg/engine/resolve/event_loop_test.go (2 hunks)
  • v2/pkg/engine/resolve/inputtemplate.go (7 hunks)
  • v2/pkg/engine/resolve/loader.go (50 hunks)
  • v2/pkg/engine/resolve/loader_hooks_test.go (14 hunks)
  • v2/pkg/engine/resolve/loader_test.go (14 hunks)
  • v2/pkg/engine/resolve/resolvable.go (14 hunks)
  • v2/pkg/engine/resolve/resolvable_custom_field_renderer_test.go (2 hunks)
  • v2/pkg/engine/resolve/resolvable_test.go (26 hunks)
  • v2/pkg/engine/resolve/resolve.go (20 hunks)
  • v2/pkg/engine/resolve/resolve_federation_test.go (21 hunks)
  • v2/pkg/engine/resolve/resolve_mock_test.go (3 hunks)
  • v2/pkg/engine/resolve/resolve_test.go (29 hunks)
  • v2/pkg/engine/resolve/response.go (1 hunks)
  • v2/pkg/engine/resolve/singleflight.go (1 hunks)
  • v2/pkg/engine/resolve/tainted_objects_test.go (3 hunks)
  • v2/pkg/engine/resolve/variables_renderer.go (1 hunks)
  • v2/pkg/fastjsonext/fastjsonext.go (1 hunks)
  • v2/pkg/fastjsonext/fastjsonext_test.go (1 hunks)
  • v2/pkg/variablesvalidation/variablesvalidation.go (1 hunks)
💤 Files with no reviewable changes (2)
  • v2/pkg/engine/datasource/graphql_datasource/graphql_subscription_client.go
  • v2/pkg/engine/datasource/graphql_datasource/graphql_subscription_client_test.go
🧰 Additional context used
🧬 Code graph analysis (30)
v2/pkg/engine/resolve/variables_renderer.go (1)
v2/pkg/engine/resolve/resolvable.go (2)
  • NewResolvable (71-79)
  • ResolvableOptions (64-69)
v2/pkg/engine/resolve/context.go (1)
v2/pkg/engine/plan/planner.go (1)
  • IncludeQueryPlanInResponse (92-96)
v2/pkg/engine/resolve/resolvable_custom_field_renderer_test.go (1)
v2/pkg/engine/resolve/resolvable.go (2)
  • NewResolvable (71-79)
  • ResolvableOptions (64-69)
v2/pkg/engine/resolve/loader_test.go (1)
v2/pkg/engine/resolve/resolvable.go (2)
  • NewResolvable (71-79)
  • ResolvableOptions (64-69)
v2/pkg/engine/datasource/introspection_datasource/source.go (2)
v2/pkg/engine/datasource/staticdatasource/static_datasource.go (3)
  • Source (72-72)
  • Source (74-76)
  • Source (78-80)
v2/pkg/engine/datasource/httpclient/file.go (1)
  • FileUpload (3-7)
v2/pkg/engine/resolve/resolve_federation_test.go (2)
v2/pkg/engine/resolve/context.go (1)
  • Context (16-37)
v2/pkg/engine/resolve/resolve_mock_test.go (1)
  • NewMockDataSource (28-32)
v2/pkg/engine/resolve/response.go (4)
v2/pkg/engine/resolve/inputtemplate.go (1)
  • InputTemplate (31-37)
v2/pkg/engine/resolve/variables.go (1)
  • Variables (27-27)
v2/pkg/engine/resolve/datasource.go (1)
  • SubscriptionDataSource (15-19)
v2/pkg/engine/resolve/fetch.go (2)
  • PostProcessingConfiguration (116-132)
  • QueryPlan (251-254)
v2/pkg/engine/datasource/graphql_datasource/graphql_datasource.go (3)
v2/pkg/engine/datasource/staticdatasource/static_datasource.go (3)
  • Source (72-72)
  • Source (74-76)
  • Source (78-80)
v2/pkg/engine/datasource/httpclient/file.go (1)
  • FileUpload (3-7)
v2/pkg/engine/datasource/httpclient/nethttpclient.go (2)
  • DoMultipartForm (272-328)
  • Do (267-270)
v2/pkg/engine/resolve/resolve_mock_test.go (2)
v2/pkg/engine/resolve/context.go (1)
  • Context (16-37)
v2/pkg/engine/datasource/httpclient/file.go (1)
  • FileUpload (3-7)
v2/pkg/engine/resolve/resolvable_test.go (1)
v2/pkg/engine/resolve/resolvable.go (2)
  • NewResolvable (71-79)
  • ResolvableOptions (64-69)
v2/pkg/engine/datasource/graphql_datasource/graphql_datasource_test.go (3)
v2/pkg/engine/resolve/context.go (1)
  • NewContext (168-175)
v2/pkg/engine/datasource/graphql_datasource/graphql_datasource.go (1)
  • SubscriptionSource (1953-1955)
v2/pkg/engine/datasource/httpclient/file.go (2)
  • FileUpload (3-7)
  • NewFileUpload (9-15)
v2/pkg/engine/resolve/resolvable.go (2)
v2/pkg/engine/resolve/resolve.go (1)
  • New (181-245)
v2/pkg/fastjsonext/fastjsonext.go (2)
  • AppendErrorToArray (8-15)
  • AppendErrorWithExtensionsCodeToArray (17-27)
v2/pkg/engine/plan/planner_test.go (2)
v2/pkg/engine/resolve/context.go (1)
  • Context (16-37)
v2/pkg/engine/datasource/httpclient/file.go (1)
  • FileUpload (3-7)
v2/pkg/engine/resolve/resolve_test.go (10)
v2/pkg/engine/resolve/context.go (2)
  • Context (16-37)
  • ExecutionOptions (50-56)
v2/pkg/engine/datasource/httpclient/nethttpclient.go (1)
  • Do (267-270)
v2/pkg/engine/resolve/resolve_mock_test.go (1)
  • NewMockDataSource (28-32)
v2/pkg/engine/resolve/response.go (2)
  • GraphQLResponse (35-43)
  • GraphQLResponseInfo (45-47)
v2/pkg/engine/resolve/fetchtree.go (6)
  • Single (59-67)
  • SingleWithPath (69-82)
  • Sequence (26-31)
  • Parallel (33-38)
  • ArrayPath (52-57)
  • ObjectPath (40-45)
v2/pkg/engine/resolve/fetch.go (4)
  • SingleFetch (91-99)
  • SingleFetch (153-155)
  • FetchConfiguration (270-302)
  • PostProcessingConfiguration (116-132)
v2/pkg/engine/resolve/datasource.go (1)
  • DataSource (10-13)
v2/pkg/engine/resolve/inputtemplate.go (5)
  • InputTemplate (31-37)
  • TemplateSegment (22-29)
  • SegmentType (15-15)
  • StaticSegmentType (18-18)
  • VariableSegmentType (19-19)
v2/pkg/engine/resolve/variables.go (4)
  • VariableKind (7-7)
  • ContextVariableKind (10-10)
  • Variables (27-27)
  • ResolvableObjectVariableKind (13-13)
v2/pkg/engine/resolve/subscription_filter.go (2)
  • SubscriptionFilter (16-21)
  • SubscriptionFieldFilter (23-26)
v2/pkg/engine/resolve/singleflight.go (2)
v2/pkg/engine/resolve/resolve.go (1)
  • New (181-245)
v2/pkg/engine/resolve/fetch.go (3)
  • FetchItem (29-34)
  • Fetch (20-27)
  • FetchInfo (376-397)
v2/pkg/engine/datasource/pubsub_datasource/pubsub_kafka.go (3)
v2/pkg/engine/resolve/context.go (1)
  • Context (16-37)
v2/pkg/engine/resolve/resolve.go (1)
  • SubscriptionUpdater (1383-1390)
v2/pkg/engine/datasource/pubsub_datasource/kafka_event_manager.go (1)
  • KafkaPublishEventConfiguration (17-21)
v2/pkg/engine/resolve/loader.go (5)
v2/pkg/engine/resolve/singleflight.go (1)
  • SingleFlight (16-22)
v2/pkg/engine/resolve/fetch.go (5)
  • FetchItemPathElement (78-82)
  • FetchItem (29-34)
  • DataSourceLoadTrace (405-419)
  • Fetch (20-27)
  • FetchInfo (376-397)
v2/pkg/engine/resolve/inputtemplate.go (1)
  • SetInputUndefinedVariables (39-51)
v2/pkg/pool/hash64.go (1)
  • Hash64 (10-16)
v2/pkg/engine/datasource/httpclient/nethttpclient.go (1)
  • WithHTTPClientSizeHint (139-141)
v2/pkg/engine/resolve/inputtemplate.go (1)
v2/pkg/engine/resolve/context.go (1)
  • Context (16-37)
v2/pkg/engine/resolve/datasource.go (3)
v2/pkg/engine/resolve/context.go (1)
  • Context (16-37)
v2/pkg/engine/datasource/httpclient/file.go (1)
  • FileUpload (3-7)
v2/pkg/engine/resolve/resolve.go (1)
  • SubscriptionUpdater (1383-1390)
v2/pkg/engine/resolve/event_loop_test.go (2)
v2/pkg/engine/resolve/context.go (1)
  • Context (16-37)
v2/pkg/engine/resolve/resolve.go (1)
  • SubscriptionUpdater (1383-1390)
v2/pkg/engine/datasource/httpclient/nethttpclient.go (2)
v2/pkg/engine/resolve/context.go (1)
  • Context (16-37)
v2/pkg/engine/datasource/httpclient/file.go (1)
  • FileUpload (3-7)
v2/pkg/engine/datasource/staticdatasource/static_datasource.go (4)
v2/pkg/engine/datasource/graphql_datasource/graphql_datasource.go (1)
  • Source (1829-1831)
v2/pkg/engine/datasource/introspection_datasource/source.go (1)
  • Source (18-20)
v2/pkg/engine/resolve/context.go (1)
  • Context (16-37)
v2/pkg/engine/datasource/httpclient/file.go (1)
  • FileUpload (3-7)
v2/pkg/fastjsonext/fastjsonext_test.go (1)
v2/pkg/fastjsonext/fastjsonext.go (3)
  • AppendErrorToArray (8-15)
  • PathElement (29-32)
  • CreateErrorObjectWithPath (34-50)
v2/pkg/engine/resolve/resolve.go (6)
v2/pkg/engine/resolve/arena.go (2)
  • ArenaPool (13-17)
  • NewArenaPool (30-34)
v2/pkg/engine/resolve/singleflight.go (2)
  • SingleFlight (16-22)
  • NewSingleFlight (29-41)
v2/pkg/engine/resolve/resolvable.go (2)
  • NewResolvable (71-79)
  • ResolvableOptions (64-69)
v2/pkg/engine/resolve/context.go (4)
  • Context (16-37)
  • Request (163-166)
  • ExecutionOptions (50-56)
  • SubgraphHeadersBuilder (39-41)
v2/pkg/engine/resolve/response.go (3)
  • GraphQLResponse (35-43)
  • GraphQLSubscription (12-16)
  • SubscriptionResponseWriter (68-74)
v2/pkg/pool/hash64.go (1)
  • Hash64 (10-16)
v2/pkg/engine/datasource/grpc_datasource/json_builder.go (1)
v2/pkg/engine/datasource/grpc_datasource/execution_plan.go (3)
  • RPCMessage (71-85)
  • ListMetadata (190-195)
  • LevelInfo (198-201)
v2/pkg/engine/resolve/loader_hooks_test.go (1)
v2/pkg/engine/resolve/context.go (1)
  • Context (16-37)
v2/pkg/engine/datasource/pubsub_datasource/pubsub_nats.go (4)
v2/pkg/engine/resolve/context.go (2)
  • Context (16-37)
  • Request (163-166)
v2/pkg/engine/resolve/resolve.go (1)
  • SubscriptionUpdater (1383-1390)
v2/pkg/engine/datasource/pubsub_datasource/nats_event_manager.go (1)
  • NatsPublishAndRequestEventConfiguration (31-35)
v2/pkg/engine/datasource/httpclient/file.go (1)
  • FileUpload (3-7)
v2/pkg/engine/resolve/authorization_test.go (2)
v2/pkg/engine/resolve/context.go (1)
  • Context (16-37)
v2/pkg/engine/resolve/resolve_mock_test.go (1)
  • NewMockDataSource (28-32)
v2/pkg/engine/datasource/httpclient/httpclient_test.go (1)
v2/pkg/engine/datasource/httpclient/nethttpclient.go (1)
  • Do (267-270)
v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go (2)
v2/pkg/engine/resolve/datasource.go (1)
  • DataSource (10-13)
v2/pkg/engine/datasource/httpclient/file.go (1)
  • FileUpload (3-7)
🪛 GitHub Actions: v2-ci
v2/pkg/engine/resolve/loader_test.go

[error] 1046-1046: BenchmarkLoader_LoadGraphQLResponseData failed: expected a response containing an errors field, but got a response without 'errors' field. Deep comparison mismatch detected in test.

v2/pkg/engine/resolve/inputtemplate.go

[error] 161-161: golangci-lint: Error return value of preparedInput.WriteString is not checked (errcheck).

🪛 GitHub Check: Linters (1.25)
v2/pkg/engine/resolve/resolvable.go

[failure] 14-14:
File is not properly formatted (gci)

v2/pkg/engine/resolve/loader.go

[failure] 20-20:
File is not properly formatted (gci)

v2/pkg/engine/resolve/resolve.go

[failure] 15-15:
File is not properly formatted (gci)

🔇 Additional comments (52)
v2/pkg/engine/datasource/introspection_datasource/fixtures/schema_introspection_with_custom_root_operation_types.golden (1)

514-514: LGTM - Formatting-only change.

The trailing newline adjustment has no functional impact on the fixture data.

v2/pkg/engine/datasource/introspection_datasource/fixtures/schema_introspection.golden (1)

366-366: LGTM - Formatting-only change.

The trailing newline adjustment has no functional impact on the fixture data.

v2/pkg/engine/datasource/introspection_datasource/fixtures/type_introspection.golden (1)

59-59: LGTM - Formatting-only change.

The trailing newline adjustment has no functional impact on the fixture data.

v2/pkg/engine/resolve/inputtemplate.go (3)

7-7: LGTM!

The io import is necessary for the InputTemplateWriter interface definition.


58-63: LGTM! Clean abstraction for arena-based memory management.

The interface is well-designed and appropriately scoped, including only the methods needed for the rendering flow. This abstraction enables the arena-based memory management improvements mentioned in the PR objectives while maintaining compatibility with *bytes.Buffer.


65-65: All callers are compatible with the interface change.

The exported methods Render and RenderAndCollectUndefinedVariables have been correctly updated to accept InputTemplateWriter. Since bytes.Buffer implements the required interface (io.Writer, io.StringWriter, Reset(), Bytes()), all existing callers—in loader.go, resolve.go, subscription_filter.go, and tests—pass compatible types without requiring updates.

v2/pkg/engine/resolve/tainted_objects_test.go (1)

73-75: LGTM: switched to astjson.ParseBytes

The change is straightforward; parsing behavior in tests remains correct.

Also applies to: 97-99, 323-327

v2/pkg/engine/plan/visitor.go (1)

1293-1296: Populate SourceName/ID on subscription trigger

Good addition; aligns tests and surfaces subgraph identity on triggers.

Confirm objectFetchConfiguration always sets both sourceName and sourceID for subscription fetches; if either can be empty, consider defaulting SourceName to sourceID (and vice versa) for consistency.

v2/pkg/engine/resolve/response.go (1)

19-26: LGTM: GraphQLSubscriptionTrigger includes SourceName/SourceID

Public surface extended as intended; no behavioral changes.

v2/pkg/engine/datasource/pubsub_datasource/pubsub_datasource_test.go (1)

427-429: Tests updated for SourceName/SourceID

Expectations match the new trigger fields and datasource id/name.

Also applies to: 492-494, 539-541, 592-594

v2/pkg/engine/resolve/resolve_federation_test.go (1)

19-28: Mocks updated for new Load signature and response []byte

Good updates: include headers param and return []byte. Assertions validate inputs properly.

Also applies to: 23-27

v2/pkg/engine/resolve/context.go (1)

164-166: Request.ID type change verified — all consumers compatible

The type change from prior type to uint64 is verified as safe:

  • Pooling/dedup: Both ArenaPool.Acquire() and ArenaPool.Release() have signatures expecting uint64; all call sites pass ctx.Request.ID directly—fully compatible.
  • Logging: No code logs Request.ID.
  • Tracing: RequestData (the struct included in trace output) contains no ID field; Request.ID is not part of trace output.
  • JSON marshaling: No JSON marshaling of Request or its ID field.
v2/go.mod (1)

31-33: All concerns verified—no issues found.

The repository correctly uses Go 1.25 across both go.mod directive and CI workflows (matrix values in v2.yml and execution.yml both specify go: [ "1.25" ]). The setup-go@v5 action is compatible with this version. The astjson v1.0.0 and go-arena v1.0.0 dependencies are properly listed in the require section, go.sum is present and synchronized (226 entries), and no deprecated astjson APIs are in use. The module is tidy.

v2/pkg/astnormalization/uploads/upload_finder.go (1)

77-77: ParseBytes migration verified—behavior parity and concurrency characteristics confirmed.

Edge case handling (nil, "", "null" → "{}") is explicit at lines 74–76, ensuring normalized input regardless of ParseBytes implementation. Variables are accessed only within the same FindUploads invocation, with no shared Parser instance—each call is independent. The migration is safe and complete (no ParseBytesWithoutCache calls remain in the codebase).

v2/pkg/engine/resolve/authorization_test.go (2)

512-519: LGTM! Mock signature correctly updated.

The mock expectation properly reflects the new DataSource Load signature with HTTP headers and direct byte slice return. The DoAndReturn handler is correctly structured to return ([]byte, error).


817-824: LGTM! Consistent mock pattern.

All mock data source setups in this test file follow the same pattern with the updated Load signature. The test data continues to return valid GraphQL responses wrapped in the expected format.

v2/pkg/engine/resolve/resolvable_custom_field_renderer_test.go (1)

443-443: LGTM! Constructor signature updated correctly.

The NewResolvable call now includes the arena.Arena parameter (nil) as the first argument, aligning with the updated constructor signature in the broader PR.

v2/pkg/fastjsonext/fastjsonext_test.go (1)

24-29: LGTM! Arena parameter migration applied consistently.

The test correctly passes nil for the arena parameter to AppendErrorToArray, aligning with the PR's migration from *astjson.Arena to arena.Arena interface.

v2/pkg/engine/resolve/event_loop_test.go (1)

74-85: LGTM! Subscription Start signature updated.

The Start method now correctly accepts HTTP headers as the second parameter, consistent with the PR's objective to propagate headers through the data flow. The method implementation remains functionally correct.

v2/pkg/engine/resolve/loader_test.go (3)

20-34: LGTM! Mock responses correctly structured.

The mock data sources now return responses wrapped under a top-level "data" field, consistent with the GraphQL response format and the broader changes in this PR.


290-300: LGTM! Test setup correctly updated.

Both the NewResolvable constructor call and the expected output assertion have been updated to match the new patterns introduced in the PR.


1524-1530: LGTM! Error path rewriting correctly updated.

The rewriteErrorPaths function calls now pass nil as the first parameter (arena), consistent with the arena-based memory management changes in this PR.

v2/pkg/engine/datasource/httpclient/httpclient_test.go (1)

82-85: LGTM! Test correctly updated for new Do signature.

The test now properly handles the new Do signature that returns ([]byte, error) directly instead of writing to a buffer. The assertion compares the returned bytes with the expected output.

v2/pkg/engine/resolve/loader_hooks_test.go (1)

52-56: LGTM! Mock expectations correctly updated.

The mock Load method now expects three parameters (context, headers, input) and the DoAndReturn handler correctly returns ([]byte, error), aligning with the updated DataSource interface.

v2/pkg/engine/datasource/introspection_datasource/source.go (2)

22-33: LGTM! Load method signature correctly updated.

The Load method now:

  • Accepts headers http.Header as the second parameter
  • Returns (data []byte, err error) instead of writing to a buffer
  • Uses the new singleTypeBytes helper for type-specific responses

All error paths correctly return (nil, err) on failure.


61-68: LGTM! Helper method properly implements byte-returning pattern.

The new singleTypeBytes helper correctly mirrors the previous singleType logic but returns bytes directly instead of writing to an io.Writer. The nil type handling properly returns the null byte slice.

v2/pkg/engine/plan/planner_test.go (3)

7-7: LGTM - Import added for new API.

The net/http import is correctly added to support the new http.Header parameter in the updated DataSource method signatures.


1078-1080: LGTM - Test mock updated to new API.

The Load method signature correctly reflects the new DataSource API that accepts HTTP headers and returns data directly instead of writing to a buffer. Returning nil, nil is appropriate for a test fake.


1082-1084: LGTM - Test mock updated consistently.

The LoadWithFiles method signature correctly mirrors the Load method changes with HTTP headers and direct return values. The test fake implementation is appropriate.

v2/pkg/engine/datasource/staticdatasource/static_datasource.go (2)

5-5: LGTM - Import added for new API.

The net/http import correctly supports the updated method signatures.


74-76: LGTM - Static source correctly returns input.

The implementation appropriately returns the input bytes directly, which is the expected behavior for a static data source.

v2/pkg/engine/resolve/resolve_mock_test.go (1)

1-67: LGTM - Auto-generated mock updated correctly.

This is an auto-generated mock file (by MockGen) that has been correctly regenerated to match the new DataSource interface signatures with HTTP headers and direct return values. No manual review concerns.

v2/pkg/fastjsonext/fastjsonext.go (5)

5-5: LGTM - Arena dependency added for memory pooling.

The go-arena import supports the PR's objective of improving memory management through arena-based pooling.


8-15: LGTM - Arena-based error appending refactored correctly.

The function signature and implementation correctly adopt the new arena-centric API, passing the arena context through value creation and mutation calls.


17-27: LGTM - Consistent arena usage for error with extensions.

The function correctly uses arena-based value construction throughout, maintaining consistency with the new API pattern.


34-50: LGTM - Error object creation properly arena-aware.

The function correctly constructs error objects using arena-based value creation, ensuring memory efficiency and proper lifecycle management.


52-59: No issues found - nil arena usage is intentional and correct.

After thorough verification:

  • The PrintGraphQLResponse function uses out.Set(nil, ...) to set fields on a value created by astjson.MustParse
  • This nil arena pattern is consistent with the codebase: the codebase explicitly uses ParseBytesWithArena(nil, data) in loader_test.go, demonstrating nil arena is an established, intentional pattern
  • Tests pass successfully across multiple scenarios (loader_test.go lines 297, 759, 1044, 1431), confirming the function works correctly
  • The function serves as a terminal operation that composes pre-constructed values and serializes them—it does not manage arena lifetimes like the building functions (AppendErrorToArray, CreateErrorObjectWithPath)
  • The design is appropriate: building functions receive and manage explicit arena parameters, while terminal functions like PrintGraphQLResponse use nil arena for simpler composition and serialization
v2/pkg/engine/datasource/graphql_datasource/graphql_datasource_test.go (2)

4023-4025: Include of SourceName/SourceID on subscription trigger looks right

Good addition for tracing/observability and source attribution. Please just confirm the values match the configured DataSource ID in the surrounding plan to avoid misleading logs/metrics.

Also applies to: 4066-4068


8773-8776: Load now returns bytes — assertions look good

The switch to capturing data from Load and asserting on the exact JSON payload is correct and matches the new API.

Also applies to: 8795-8798

v2/pkg/engine/datasource/graphql_datasource/graphql_datasource.go (1)

1909-1917: Good: headers plumbed and return-bytes API adhered to.

The methods correctly compact variables, forward headers to httpclient, and return data bytes.

v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go (1)

404-409: Assertion aligns with error-as-data policy; keep consistent with Load behavior.

No change required; just ensure datasource consistently returns nil error in all error paths (see datasource comment).

If you adopt the merge-error change, re-run this test to confirm it still passes.

v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go (1)

101-129: ****

The repository declares Go 1.25 in its go.mod files. Since Go 1.22+, loop variables are scoped per-iteration by default, automatically preventing the closure capture issue the review comment warns about. The suggested rebinding refactor is unnecessary—the original code is correct and does not have a closure capture bug. No changes are required.

Likely an incorrect or invalid review comment.

v2/pkg/engine/resolve/resolve.go (2)

5-20: Fix import ordering to satisfy gci

Linters report “File is not properly formatted (gci)”. Run gci/fmt over this file (and module-wide) to fix grouping/order.

Example:

  • standard lib
  • third-party
  • project-local

306-349: ArenaResolveGraphQLResponse looks solid

Acquire/assign/release of arenas and buffered write via arena buffer are correct.

v2/pkg/engine/resolve/loader.go (8)

360-373: Good use of arena allocation for items.

The arena-based allocation in selectItemsForPath properly uses the loader's arena for memory management, which aligns with the PR's memory management improvements.


547-549: Proper arena allocation before parsing.

The code correctly allocates a slice copy in the arena before parsing, ensuring the parsed data is arena-managed throughout its lifecycle.


1576-1592: Operation type context propagation looks good.

The new context key pattern for operation type propagation is implemented correctly with a safe default fallback.


1594-1603: Headers extraction method looks correct.

The headersForSubgraphRequest method properly handles nil checks and extracts headers for subgraph requests, with the extraKey used for request deduplication.


1656-1666: Direct load implementation is correct.

The loadByContextDirect method properly calls the updated DataSource interface methods with headers, aligning with the PR's API changes.


1806-1814: Execution now properly uses loadByContext.

The refactored execution flow correctly delegates to loadByContext, which encapsulates SingleFlight deduplication logic while respecting loader hooks.


791-791: Consistent arena usage for error object creation.

All error object creation now uses ParseWithArena, ensuring arena-based memory management is applied consistently throughout the error handling paths.

Also applies to: 1018-1028, 1065-1065, 1079-1079, 1091-1091, 1109-1109, 1144-1144, 1150-1150, 1160-1160, 1166-1166, 1186-1186, 1191-1191, 1198-1198, 1203-1203, 1210-1210, 1214-1214


185-186: ****

The initialization concern is unfounded. Both jsonArena and sf are properly initialized before use:

  • sf is assigned in newTools() (line 265 in resolve.go) from the Resolver's sf field, which is created with NewSingleFlight() at Resolver initialization.
  • jsonArena is assigned from the arena pool in resolve.go (lines 319, 485) immediately before the loader is used, after which LoadGraphQLResponseData is called.

The fields don't need cleanup in Free() because they're lifecycle-managed by their respective owners (Resolver for sf, arena pool for jsonArena), not by the Loader.

Likely an incorrect or invalid review comment.

Copy link
Contributor

@coderabbitai coderabbitai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actionable comments posted: 0

♻️ Duplicate comments (6)
v2/pkg/engine/datasource/httpclient/nethttpclient.go (3)

163-165: Avoid mutating/capturing caller's headers; clone before use.

Direct assignment request.Header = baseHeaders aliases the map, allowing subsequent Add/Set calls to mutate the caller's map, which can cause race conditions across concurrent requests.

Apply this diff:

 if baseHeaders != nil {
-   request.Header = baseHeaders
+   request.Header = baseHeaders.Clone()
 }

219-223: Set Request.ContentLength field instead of the header.

Manually setting the Content-Length header is ignored or overwritten by net/http. The request.ContentLength field should be set instead.

Apply this diff:

 if contentLength > 0 {
-   // always set the Content-Length Header so that chunking can be avoided
-   // and other parties can more efficiently parse
-   request.Header.Set(ContentLengthHeader, fmt.Sprintf("%d", contentLength))
+   request.ContentLength = int64(contentLength)
 }

Note: Remove the header manipulation; net/http will populate the header from the field. For streaming bodies with unknown length, leave ContentLength at its default (-1).


314-319: Check os.Open error before appending; prevent nil deref in cleanup.

temporaryFile is appended to tempFiles before checking the error from os.Open. If the open fails, tempFiles will contain a nil entry, causing the defer cleanup at lines 329-339 to panic when attempting to close or remove the file.

Apply this diff:

 temporaryFile, err := os.Open(file.Path())
-tempFiles = append(tempFiles, temporaryFile)
 if err != nil {
    return nil, err
 }
+tempFiles = append(tempFiles, temporaryFile)
 formValues[key] = bufio.NewReader(temporaryFile)
v2/pkg/engine/resolve/loader.go (1)

20-20: Fix import ordering per static analysis.

The gci linter indicates this file is not properly formatted. Ensure imports are grouped correctly (standard library, external, internal).

Run the following to fix:

#!/bin/bash
# Format the file with gci
gci write --skip-generated -s standard -s default -s "prefix(github.com/wundergraph/graphql-go-tools)" v2/pkg/engine/resolve/loader.go
v2/pkg/engine/resolve/resolvable.go (1)

14-14: Fix import ordering per static analysis.

The gci linter indicates this file is not properly formatted. Ensure imports are grouped correctly (standard library, external, internal).

Run the following to fix:

#!/bin/bash
# Format the file with gci
gci write --skip-generated -s standard -s default -s "prefix(github.com/wundergraph/graphql-go-tools)" v2/pkg/engine/resolve/resolvable.go
v2/pkg/engine/resolve/resolve.go (1)

15-15: Fix import ordering per static analysis.

The gci linter indicates this file is not properly formatted. Ensure imports are grouped correctly (standard library, external, internal).

Run the following to fix:

#!/bin/bash
# Format the file with gci
gci write --skip-generated -s standard -s default -s "prefix(github.com/wundergraph/graphql-go-tools)" v2/pkg/engine/resolve/resolve.go
🧹 Nitpick comments (3)
v2/pkg/engine/datasource/graphql_datasource/graphql_subscription_client_test.go (1)

2440-2440: LGTM - discarding return values is acceptable here.

Explicitly discarding both return values from fmt.Fprintf is a common pattern in test mock handlers. Since this test focuses on WebSocket upgrade failure handling rather than the mock server's write success, this is acceptable.

If you want to be more defensive, consider checking the error:

-				_, _ = fmt.Fprintf(w, `{"error": "WebSocket upgrade failed", "status": %d}`, tc.statusCode)
+				_, err := fmt.Fprintf(w, `{"error": "WebSocket upgrade failed", "status": %d}`, tc.statusCode)
+				require.NoError(t, err)
v2/pkg/engine/datasource/httpclient/nethttpclient.go (1)

329-339: Consider completing cleanup even on error.

The early return on Close() or Remove() errors prevents cleanup of remaining files, potentially leaking file descriptors. While minor compared to the nil entry issue at lines 314-319, consider logging errors and continuing the cleanup loop.

Example approach:

 defer func() {
    multipartBody.Close()
    for _, file := range tempFiles {
-       if err := file.Close(); err != nil {
-           return
-       }
-       if err = os.Remove(file.Name()); err != nil {
-           return
-       }
+       if closeErr := file.Close(); closeErr != nil {
+           // log closeErr but continue cleanup
+       }
+       if removeErr := os.Remove(file.Name()); removeErr != nil {
+           // log removeErr but continue cleanup
+       }
    }
 }()
v2/pkg/engine/resolve/loader.go (1)

1636-1686: Consider defensive copy when caching single-flight responses.

Line 1684 stores res.out directly into the shared cache without copying. While mergeResult (lines 556-558) makes a defensive copy before parsing, the cached item.response slice could be problematic if:

  • The DataSource.Load() implementation reuses the backing array across calls, or
  • Concurrent readers access the cached response while the backing array is modified elsewhere.

A defensive copy at line 1684 would eliminate this risk:

-	item.response = res.out
+	item.response = append([]byte(nil), res.out...)

This aligns with the defensive copy pattern already used at line 547-548 (in earlier code) and lines 556-558.

📜 Review details

Configuration used: Path: .coderabbit.yaml

Review profile: CHILL

Plan: Pro

📥 Commits

Reviewing files that changed from the base of the PR and between 60b5c3b and 3fb0272.

📒 Files selected for processing (6)
  • v2/pkg/engine/datasource/graphql_datasource/graphql_subscription_client_test.go (1 hunks)
  • v2/pkg/engine/datasource/httpclient/nethttpclient.go (9 hunks)
  • v2/pkg/engine/resolve/context.go (2 hunks)
  • v2/pkg/engine/resolve/loader.go (50 hunks)
  • v2/pkg/engine/resolve/resolvable.go (14 hunks)
  • v2/pkg/engine/resolve/resolve.go (20 hunks)
🚧 Files skipped from review as they are similar to previous changes (1)
  • v2/pkg/engine/resolve/context.go
🧰 Additional context used
🧬 Code graph analysis (4)
v2/pkg/engine/resolve/resolve.go (7)
v2/pkg/engine/resolve/arena.go (2)
  • ArenaPool (13-17)
  • NewArenaPool (30-34)
v2/pkg/engine/resolve/singleflight.go (2)
  • SingleFlight (16-22)
  • NewSingleFlight (29-41)
v2/pkg/engine/resolve/resolvable.go (2)
  • NewResolvable (72-80)
  • ResolvableOptions (65-70)
v2/pkg/engine/resolve/context.go (4)
  • Context (16-37)
  • Request (170-173)
  • ExecutionOptions (57-63)
  • SubgraphHeadersBuilder (43-47)
v2/pkg/engine/resolve/response.go (3)
  • GraphQLResponse (35-43)
  • GraphQLSubscription (12-16)
  • SubscriptionResponseWriter (68-74)
v2/pkg/engine/datasource/staticdatasource/static_datasource.go (3)
  • Source (72-72)
  • Source (74-76)
  • Source (78-80)
v2/pkg/pool/hash64.go (1)
  • Hash64 (10-16)
v2/pkg/engine/resolve/loader.go (5)
v2/pkg/engine/resolve/singleflight.go (1)
  • SingleFlight (16-22)
v2/pkg/engine/resolve/fetch.go (5)
  • FetchItemPathElement (78-82)
  • FetchItem (29-34)
  • DataSourceLoadTrace (405-419)
  • Fetch (20-27)
  • FetchInfo (376-397)
v2/pkg/engine/resolve/context.go (3)
  • RateLimitOptions (129-142)
  • Context (16-37)
  • ExecutionOptions (57-63)
v2/pkg/engine/resolve/inputtemplate.go (1)
  • SetInputUndefinedVariables (39-51)
v2/pkg/pool/hash64.go (1)
  • Hash64 (10-16)
v2/pkg/engine/resolve/resolvable.go (2)
v2/pkg/engine/resolve/resolve.go (1)
  • New (187-251)
v2/pkg/fastjsonext/fastjsonext.go (2)
  • AppendErrorToArray (8-15)
  • AppendErrorWithExtensionsCodeToArray (17-27)
v2/pkg/engine/datasource/httpclient/nethttpclient.go (2)
v2/pkg/engine/resolve/context.go (1)
  • Context (16-37)
v2/pkg/engine/datasource/httpclient/file.go (1)
  • FileUpload (3-7)
🪛 GitHub Check: Linters (1.25)
v2/pkg/engine/resolve/resolve.go

[failure] 15-15:
File is not properly formatted (gci)

v2/pkg/engine/resolve/loader.go

[failure] 20-20:
File is not properly formatted (gci)

v2/pkg/engine/resolve/resolvable.go

[failure] 14-14:
File is not properly formatted (gci)

🔇 Additional comments (19)
v2/pkg/engine/datasource/httpclient/nethttpclient.go (6)

30-30: LGTM: constant addition.

The ContentLengthHeader constant is properly defined and aligns with other header constants in the file.


133-154: LGTM: well-designed size hint mechanism.

The context-based size hint with a custom key type prevents collisions, and the buffer sizing logic balances memory efficiency with allocation overhead.


240-250: LGTM: thoughtful buffer lifecycle management.

The comment explaining the GC-based approach over pooling is valuable documentation. The buffer creation leverages the size hint mechanism to reduce allocations efficiently.


252-278: LGTM: clean trace extension handling.

The conditional trace embedding is well-structured, and error propagation follows the consistent (nil, err) pattern throughout.


281-284: LGTM: signature change aligns with PR objectives.

The updated signature to return []byte directly and accept baseHeaders matches the PR's goal of improving memory management and moving request deduplication into the engine.


341-341: LGTM: appropriate contentLength for multipart.

Passing contentLength=0 for the multipart body is correct since the body is a pipe reader with unknown length at request creation time.

v2/pkg/engine/resolve/loader.go (5)

186-196: Arena and single-flight integration looks good.

The documentation clearly explains the arena's thread-safety constraints and lifecycle. The warning about tying parsed JSON's underlying bytes to the arena lifecycle is crucial for avoiding segfaults.


369-383: Arena-aware helper functions are well-structured.

The consistent pattern of accepting arena.Arena as the first parameter and using arena-backed allocations throughout selectItemsForPath, selectItems, and itemsData ensures memory is managed correctly.

Also applies to: 401-439, 441-455


556-558: Defensive copy before parsing is essential.

This pattern correctly ties the lifecycle of the parsed JSON to the arena-allocated slice, preventing segfaults as noted in the comments at lines 190-193.


752-831: Arena-aware error handling is consistently implemented.

The lazy initialization via l.resolvable.ensureErrorsInitialized() (lines 788, 828, etc.) is a good optimization, and arena-backed object creation throughout error rendering paths is correct.

Also applies to: 873-1044


1606-1624: New context helper looks good.

GetOperationTypeFromContext provides a clean way to access the operation type from context, useful for the transport layer to make decisions (e.g., disabling single-flight for mutations).

v2/pkg/engine/resolve/resolvable.go (3)

72-80: Constructor signature updated for arena support.

Accepting arena.Arena as a parameter is correct. Callers must ensure they provide a valid arena before calling methods that use it (like Init), or rely on the new ArenaResolveGraphQLResponse path in resolve.go.


240-245: Lazy error initialization is an excellent optimization.

This pattern avoids allocating the errors array upfront, which can significantly reduce memory usage when most operations complete without errors. The consistent use of ensureErrorsInitialized() throughout the file (lines 773, 1215, 1286, 1292, 1298) ensures correctness.


110-128: Arena integration in Init methods looks correct.

Both Init and InitSubscription properly use arena-backed object creation and parsing. The explicit comments about not initializing errors (lines 115-116, 134-135) helpfully document the lazy initialization strategy.

Also applies to: 130-166

v2/pkg/engine/resolve/resolve.go (5)

75-85: Arena pools and single-flight initialization look excellent.

The separation of resolveArenaPool and responseBufferPool is well-reasoned (as explained in the comments), and both pools along with the single-flight cache are properly initialized.

Also applies to: 239-241


313-356: ArenaResolveGraphQLResponse has excellent arena lifecycle management.

This new method demonstrates proper arena handling:

  • Acquires arenas before use (lines 325, 343)
  • Sets arenas on both loader and resolvable (lines 326-327)
  • Releases arenas on all error paths (lines 331, 338, 347-349)
  • Releases arenas after successful completion (lines 352, 354)

The use of arena-backed buffers (line 344) for response writing is also correct.


488-530: Subscription update arena handling is correct.

The pattern of acquiring the arena (line 490), setting it on tools (lines 491-492), and releasing on all exit paths (lines 495, 507, 519, 530) is consistent and safe.


1079-1084: Header propagation helper is clean and straightforward.

The triggerHeaders method correctly delegates to SubgraphHeadersBuilder when available and provides sensible defaults.


253-275: Clarify the misleading "we set the arena manually" comment and verify nil arena handling in ResolveGraphQLResponse.

The code shows both ResolveGraphQLResponse and ArenaResolveGraphQLResponse exist in parallel, but the comment at line 255 is inaccurate. ResolveGraphQLResponse never sets the arena before calling Init (line 293)—only ArenaResolveGraphQLResponse does. The data parameter difference between them (ResolveGraphQLResponse passes data bytes, ArenaResolveGraphQLResponse passes nil) may influence arena behavior, but without examining the astjson library's nil-arena handling, it's unclear if this is an intentional design or a latent issue.

Either:

  • Update the comment to clarify that newTools intentionally creates Resolvable with nil arena for the non-arena code path, and verify tests cover this path
  • Or verify that astjson functions (ObjectValue, ParseBytesWithArena, MergeValues) safely handle nil arenas

Copy link
Contributor

@coderabbitai coderabbitai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actionable comments posted: 1

♻️ Duplicate comments (1)
v2/pkg/engine/datasource/httpclient/nethttpclient.go (1)

314-318: Critical: os.Open error still checked after append - can cause nil dereference.

The past review comment on this issue was not addressed. When os.Open fails, temporaryFile is nil but still appended to tempFiles. The defer cleanup at line 332 will then panic when calling file.Close() on the nil entry.

Apply this diff to fix:

  temporaryFile, err := os.Open(file.Path())
- tempFiles = append(tempFiles, temporaryFile)
  if err != nil {
      return nil, err
  }
+ tempFiles = append(tempFiles, temporaryFile)
📜 Review details

Configuration used: Path: .coderabbit.yaml

Review profile: CHILL

Plan: Pro

📥 Commits

Reviewing files that changed from the base of the PR and between 3fb0272 and ce83a7b.

📒 Files selected for processing (5)
  • v2/pkg/engine/datasource/httpclient/nethttpclient.go (9 hunks)
  • v2/pkg/engine/resolve/inputtemplate.go (8 hunks)
  • v2/pkg/engine/resolve/loader.go (50 hunks)
  • v2/pkg/engine/resolve/loader_test.go (14 hunks)
  • v2/pkg/engine/resolve/resolvable.go (14 hunks)
🧰 Additional context used
🧬 Code graph analysis (5)
v2/pkg/engine/resolve/resolvable.go (2)
v2/pkg/engine/resolve/resolve.go (1)
  • New (187-251)
v2/pkg/fastjsonext/fastjsonext.go (2)
  • AppendErrorToArray (8-15)
  • AppendErrorWithExtensionsCodeToArray (17-27)
v2/pkg/engine/resolve/inputtemplate.go (2)
v2/pkg/engine/resolve/context.go (1)
  • Context (16-37)
v2/pkg/lexer/literal/literal.go (1)
  • COMMA (23-23)
v2/pkg/engine/resolve/loader_test.go (2)
v2/pkg/engine/resolve/resolvable.go (2)
  • NewResolvable (72-80)
  • ResolvableOptions (65-70)
v2/pkg/engine/resolve/loader.go (1)
  • Loader (159-196)
v2/pkg/engine/resolve/loader.go (5)
v2/pkg/engine/datasource/httpclient/nethttpclient.go (2)
  • ResponseContext (72-76)
  • WithHTTPClientSizeHint (142-144)
v2/pkg/engine/resolve/fetch.go (11)
  • SingleFetch (91-99)
  • SingleFetch (153-155)
  • BatchEntityFetch (160-169)
  • BatchEntityFetch (194-196)
  • EntityFetch (200-209)
  • EntityFetch (226-228)
  • FetchItemPathElement (78-82)
  • FetchItem (29-34)
  • DataSourceLoadTrace (405-419)
  • Fetch (20-27)
  • FetchInfo (376-397)
v2/pkg/engine/resolve/context.go (3)
  • RateLimitOptions (129-142)
  • Context (16-37)
  • ExecutionOptions (57-63)
v2/pkg/engine/resolve/inputtemplate.go (1)
  • SetInputUndefinedVariables (39-51)
v2/pkg/pool/hash64.go (1)
  • Hash64 (10-16)
v2/pkg/engine/datasource/httpclient/nethttpclient.go (2)
v2/pkg/engine/resolve/context.go (1)
  • Context (16-37)
v2/pkg/engine/datasource/httpclient/file.go (1)
  • FileUpload (3-7)
🪛 GitHub Actions: v2-ci
v2/pkg/engine/resolve/loader.go

[error] 23-23: golangci-lint: File is not properly formatted (gci).

🪛 GitHub Check: Linters (1.25)
v2/pkg/engine/resolve/resolvable.go

[failure] 16-16:
File is not properly formatted (gci)

v2/pkg/engine/resolve/loader.go

[failure] 23-23:
File is not properly formatted (gci)

🔇 Additional comments (12)
v2/pkg/engine/resolve/resolvable.go (3)

239-244: LGTM: Well-designed lazy initialization helper.

The ensureErrorsInitialized() helper cleanly implements the lazy initialization pattern for the errors array, supporting the memory optimization goal mentioned in the PR objectives. The consistent usage throughout the file (lines 772, 1214, 1285, 1291, 1297) ensures errors are only allocated when needed.


113-115: Effective memory optimization through lazy initialization.

Setting r.errors = nil instead of eager initialization is a good optimization that defers allocation until errors actually occur. The inline comments clearly document this design decision, and the ensureErrorsInitialized() helper ensures safe usage throughout the codebase.

Also applies to: 133-134, 172-173


71-79: Intentional API change for arena integration.

The constructor signature change from NewResolvable(context.Context, ResolvableOptions) to NewResolvable(arena.Arena, ResolvableOptions) is a breaking change that aligns with the PR's arena-based memory management objectives. All test call sites have been correctly updated to pass nil or an appropriate arena instance.

v2/pkg/engine/resolve/loader_test.go (1)

290-290: Test updates correctly reflect API changes.

All test instantiations of NewResolvable have been properly updated to pass nil as the first arena parameter, consistent with the new constructor signature. This approach is appropriate for tests that don't require specific arena lifecycle management.

Also applies to: 379-379, 470-470, 752-752, 1027-1027, 1128-1128, 1424-1424

v2/pkg/engine/resolve/inputtemplate.go (2)

58-63: Well-designed abstraction with InputTemplateWriter interface.

The new InputTemplateWriter interface cleanly abstracts buffer operations (io.Writer, io.StringWriter, Reset(), Bytes()) and enables controlled reuse across the template rendering pipeline. This design improves testability and flexibility.


161-174: Error handling improvement for write operations.

Previously, write errors in renderHeaderVariable were silently ignored using _, _ = preparedInput.Write(...). The updated code now properly checks and propagates these errors (lines 161-163, 168-174), which improves robustness and makes potential issues visible to callers.

v2/pkg/engine/resolve/loader.go (3)

185-195: Excellent documentation of arena thread-safety constraints.

The detailed comments on jsonArena clearly explain critical constraints:

  • Not thread-safe (use only from main goroutine)
  • Lifecycle managed by Resolver (don't Reset or Release)
  • Critical requirement: underlying bytes must be arena-allocated to tie lifecycles

This documentation will help prevent common arena misuse patterns that lead to segfaults.


552-557: Critical arena lifecycle management pattern.

Lines 555-557 implement a crucial pattern: allocating a new slice on the arena and copying res.out before parsing with ParseBytesWithArena. This ensures the parsed AST and the underlying bytes share the same arena lifecycle, preventing segfaults when the arena is released. As noted in the arena comments (line 191), this pattern is essential for correct arena usage.


1635-1685: SingleFlight integration properly handles mutations and context.

The loadByContext function correctly:

  • Disables deduplication for mutations (line 1644) to avoid side-effect issues
  • Respects the DisableRequestDeduplication flag (line 1645)
  • Handles context cancellation via select with item.loaded and ctx.Done() (lines 1657-1661)
  • Provides size hints to the HTTP client (line 1672)
  • Properly stores shared responses (line 1667, 1683)

The integration aligns with the PR's request deduplication objectives.

v2/pkg/engine/datasource/httpclient/nethttpclient.go (3)

133-154: LGTM: Buffer size hint mechanism is well-designed.

The context-based size hint and the buffer function provide an efficient way to pre-allocate buffers based on expected response sizes, reducing allocations and copies. The 64-byte default for cases without a hint is a reasonable middle ground.


240-278: LGTM: Response buffering and trace handling are well-implemented.

The buffer allocation strategy is clearly documented, and the trace injection using jsonparser.Set is correct. Error handling is consistent throughout.


281-283: LGTM: Do function correctly updated to new signature.

The function now properly accepts baseHeaders and returns data directly, aligning with the PR objectives. Passing len(body) as contentLength is correct for regular HTTP requests.

Copy link
Contributor

@coderabbitai coderabbitai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actionable comments posted: 3

♻️ Duplicate comments (2)
v2/pkg/engine/datasource/graphql_datasource/graphql_datasource_test.go (1)

8857-8862: Close temp files to avoid FD leaks in tests

Defer closing files created with os.CreateTemp to prevent descriptor leaks and flaky CI.

Apply in both subtests:

@@
- f, err := os.CreateTemp(dir, fileName)
+ f, err := os.CreateTemp(dir, fileName)
   assert.NoError(t, err)
+  defer func() { _ = f.Close() }()
@@
- f1, err := os.CreateTemp(dir, file1Name)
+ f1, err := os.CreateTemp(dir, file1Name)
   assert.NoError(t, err)
+  defer func() { _ = f1.Close() }()
@@
- f2, err := os.CreateTemp(dir, file2Name)
+ f2, err := os.CreateTemp(dir, file2Name)
   assert.NoError(t, err)
+  defer func() { _ = f2.Close() }()

Also applies to: 8915-8924

v2/pkg/engine/resolve/subgraph_request_singleflight.go (1)

156-169: Mix extraKey into the SF hash via bytes, not arithmetic addition.

computeSFKey still does:

_, _ = h.Write(input)
return h.Sum64() + extraKey // extraKey is the pre-generated hash for the headers

Adding extraKey arithmetically can fold distinct (bodyHash, extraKey) pairs into the same final value and weakens the distribution guarantees you get from xxhash.

As previously suggested in an earlier review, encode extraKey into 8 bytes and feed it into the digest so it participates in the hash properly:

func (s *SubgraphRequestSingleFlight) computeSFKeyWithDigest(h *xxhash.Digest, fetchItem *FetchItem, input []byte, extraKey uint64) uint64 {
    if fetchItem != nil && fetchItem.Fetch != nil {
        if info := fetchItem.Fetch.FetchInfo(); info != nil {
            _, _ = h.WriteString(info.DataSourceID)
            _, _ = h.WriteString(":")
        }
    }
    _, _ = h.Write(input)

    if extraKey != 0 {
        var b [8]byte
        binary.LittleEndian.PutUint64(b[:], extraKey)
        _, _ = h.Write(b[:])
    }

    return h.Sum64()
}

This keeps the strong SF key semantics even when header hashes are mixed in.

🧹 Nitpick comments (1)
v2/pkg/engine/datasource/graphql_datasource/graphql_datasource_test.go (1)

8456-8456: Remove no-op defers on ctx.Context().Done()

Done() returns a channel; deferring it has no effect. Remove or use context.WithCancel and defer cancel() when you need lifecycle control.

- defer ctx.Context().Done()
+ // no-op removed; use a cancelable context if needed:
+ // c, cancel := context.WithCancel(ctx.Context()); defer cancel()

Also applies to: 8468-8468, 8508-8508, 8574-8574, 8616-8616

📜 Review details

Configuration used: Path: .coderabbit.yaml

Review profile: CHILL

Plan: Pro

📥 Commits

Reviewing files that changed from the base of the PR and between d8f04ca and 57b8680.

📒 Files selected for processing (6)
  • v2/pkg/engine/datasource/graphql_datasource/graphql_datasource_test.go (14 hunks)
  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go (28 hunks)
  • v2/pkg/engine/resolve/inbound_request_singleflight.go (1 hunks)
  • v2/pkg/engine/resolve/loader.go (48 hunks)
  • v2/pkg/engine/resolve/subgraph_request_singleflight.go (1 hunks)
  • v2/pkg/engine/resolve/subgraph_request_singleflight_test.go (1 hunks)
🚧 Files skipped from review as they are similar to previous changes (1)
  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go
🧰 Additional context used
🧠 Learnings (5)
📚 Learning: 2025-09-19T14:50:19.528Z
Learnt from: endigma
Repo: wundergraph/graphql-go-tools PR: 1298
File: v2/pkg/engine/resolve/resolve.go:999-1011
Timestamp: 2025-09-19T14:50:19.528Z
Learning: In the graphql-go-tools resolver, when handling backpressure for async unsubscribe operations, the team prefers using individual goroutines over bounded deferrer implementations. The rationale is that bounded implementations can also overflow, and the preferred solution for overload is scaling the deployment rather than internal buffering. The current approach prevents deadlocks at the cost of goroutine creation, which is an acceptable tradeoff.

Applied to files:

  • v2/pkg/engine/resolve/inbound_request_singleflight.go
  • v2/pkg/engine/datasource/graphql_datasource/graphql_datasource_test.go
  • v2/pkg/engine/resolve/loader.go
📚 Learning: 2025-09-19T14:51:33.724Z
Learnt from: endigma
Repo: wundergraph/graphql-go-tools PR: 1298
File: v2/pkg/engine/resolve/resolve.go:339-347
Timestamp: 2025-09-19T14:51:33.724Z
Learning: In the graphql-go-tools resolver, the event loop follows a strict single-threaded design where all subscription lifecycle management (add/remove from triggers) must happen through the events channel to the main processEvents() goroutine. Worker goroutines should not directly modify subscription state or call unsubscribe methods, as this would break the single-threaded event loop architecture and create race conditions.

Applied to files:

  • v2/pkg/engine/datasource/graphql_datasource/graphql_datasource_test.go
📚 Learning: 2025-08-29T09:35:47.969Z
Learnt from: ysmolski
Repo: wundergraph/graphql-go-tools PR: 1282
File: v2/pkg/engine/plan/visitor.go:5-5
Timestamp: 2025-08-29T09:35:47.969Z
Learning: The wundergraph/graphql-go-tools project does not support Go versions < 1.23, so compatibility concerns for features available in Go 1.21+ (like cmp.Or) should not be raised.

Applied to files:

  • v2/pkg/engine/resolve/loader.go
📚 Learning: 2025-07-02T15:28:02.122Z
Learnt from: SkArchon
Repo: wundergraph/graphql-go-tools PR: 1203
File: v2/pkg/engine/resolve/loader.go:63-67
Timestamp: 2025-07-02T15:28:02.122Z
Learning: In the graphql-go-tools codebase, result structs are consistently initialized with non-nil bytes.Buffer instances, making additional nil checks for res.out unnecessary defensive programming.

Applied to files:

  • v2/pkg/engine/resolve/loader.go
📚 Learning: 2025-08-08T09:43:07.433Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1262
File: v2/pkg/engine/datasource/grpc_datasource/json_builder.go:0-0
Timestamp: 2025-08-08T09:43:07.433Z
Learning: In v2/pkg/engine/datasource/grpc_datasource/json_builder.go, mergeEntities intentionally uses the loop index when calling indexMap.getResultIndex because the index map is type-aware, making per-type counters unnecessary under the current assumptions. Avoid suggesting per-type ordinal counters for this path in future reviews.

Applied to files:

  • v2/pkg/engine/resolve/loader.go
🧬 Code graph analysis (5)
v2/pkg/engine/resolve/subgraph_request_singleflight.go (2)
v2/pkg/engine/resolve/resolve.go (1)
  • New (191-256)
v2/pkg/engine/resolve/fetch.go (2)
  • Fetch (19-26)
  • FetchInfo (345-366)
v2/pkg/engine/resolve/inbound_request_singleflight.go (2)
v2/pkg/engine/resolve/context.go (4)
  • Context (16-38)
  • ExecutionOptions (60-78)
  • Request (185-188)
  • SubgraphHeadersBuilder (44-50)
v2/pkg/engine/resolve/response.go (1)
  • GraphQLResponse (36-44)
v2/pkg/engine/resolve/subgraph_request_singleflight_test.go (2)
v2/pkg/engine/resolve/fetch.go (6)
  • FetchInfo (345-366)
  • FetchKind (11-11)
  • FetchKindSingle (14-14)
  • FetchDependencies (110-113)
  • Fetch (19-26)
  • GraphCoordinate (368-372)
v2/pkg/engine/resolve/subgraph_request_singleflight.go (1)
  • NewSingleFlight (52-65)
v2/pkg/engine/datasource/graphql_datasource/graphql_datasource_test.go (3)
v2/pkg/engine/resolve/context.go (1)
  • NewContext (190-197)
v2/pkg/engine/datasource/graphql_datasource/graphql_datasource.go (1)
  • SubscriptionSource (1953-1955)
v2/pkg/engine/datasource/httpclient/file.go (2)
  • FileUpload (3-7)
  • NewFileUpload (9-15)
v2/pkg/engine/resolve/loader.go (6)
v2/pkg/engine/resolve/fetch.go (8)
  • PostProcessingConfiguration (115-131)
  • FetchItemPathElement (77-81)
  • FetchItem (28-33)
  • DataSourceLoadTrace (374-388)
  • BatchEntityFetch (159-168)
  • BatchEntityFetch (193-195)
  • Fetch (19-26)
  • FetchInfo (345-366)
v2/pkg/engine/resolve/subgraph_request_singleflight.go (1)
  • SubgraphRequestSingleFlight (12-15)
v2/pkg/engine/resolve/context.go (2)
  • Context (16-38)
  • ExecutionOptions (60-78)
v2/pkg/engine/resolve/inputtemplate.go (1)
  • SetInputUndefinedVariables (39-51)
v2/pkg/engine/resolve/resolve.go (1)
  • New (191-256)
v2/pkg/engine/datasource/httpclient/nethttpclient.go (1)
  • WithHTTPClientSizeHint (142-144)
⏰ Context from checks skipped due to timeout of 90000ms. You can increase the timeout in your CodeRabbit configuration to a maximum of 15 minutes (900000ms). (4)
  • GitHub Check: Build and test (go 1.25 / windows-latest)
  • GitHub Check: Build and test (go 1.25 / ubuntu-latest)
  • GitHub Check: Build and test (go 1.25 / windows-latest)
  • GitHub Check: Build and test (go 1.25 / ubuntu-latest)
🔇 Additional comments (7)
v2/pkg/engine/datasource/graphql_datasource/graphql_datasource_test.go (4)

4011-4013: LGTM: populate SourceName/SourceID on subscription triggers

Consistently tying triggers to the DS id improves traceability. No issues spotted.

Also applies to: 4054-4056


8761-8764: LGTM: Load now returns []byte; assertions updated

Return contract and assertions look correct for null/undefined variables handling.

Also applies to: 8783-8786


8870-8872: LGTM: LoadWithFiles return value and assertions updated

Switch to returning []byte and asserting on empty body is correct for these handlers.

Also applies to: 8927-8931


8443-8443: Nil headers passed to Start: verified as nil-safe ✓

The Start() method implementation shows that nil headers are handled safely:

  1. Direct assignment (safe): Line options.Header = headers simply assigns the nil value to a struct field—no dereference occurs here.

  2. Defensive nil-guard downstream: In nethttpclient.go:167, the code checks if headers != nil before calling jsonparser.ObjectEach(). This prevents any unsafe access of nil headers in the transport stack.

  3. No unsafe dereferences found: Throughout the verified call path, nil headers are never dereferenced without a preceding nil check.

The code follows Go best practices for nil-safety. The concern about nil dereference is not substantiated by the codebase—the implementation properly guards against nil headers through defensive nil-checks in downstream consumers.

v2/pkg/engine/resolve/subgraph_request_singleflight_test.go (1)

9-209: Solid coverage of subgraph single‑flight semantics.

The stubs plus the four tests exercise all the important behaviors: leader/follower distinction, size‑hint propagation, FetchKey vs SFKey separation, and nil/missing FetchInfo cases. The rolling‑window test also closely matches the logic in Finish and guards against regressions in the size‑hint algorithm.

v2/pkg/engine/resolve/loader.go (2)

51-73: ResponseInfo responseBody conversion looks correct.

Storing the raw subgraph body as []byte and exposing it via GetResponseBody() keeps the internal field private while still allowing hooks/transport code to inspect the payload. Since newResponseInfo only assigns res.out and consumers get a fresh string on each call, there’s no mutation risk here.


331-345: Subgraph single-flight integration in Loader is well‑threaded.

  • selectItemsForPath and selectItems use arena‑allocated slices without sharing jsonArena across goroutines, respecting the “not thread safe” comment.
  • loadByContext cleanly wires SubgraphRequestSingleFlight in: it derives headers + header hash, checks singleFlightAllowed, uses size hints via WithHTTPClientSizeHint, and correctly distinguishes leader vs follower.
  • Leader/follower stats in singleFlightStats are set once in loadByContext and exported via executeSourceLoad to DataSourceLoadTrace as SingleFlightUsed / SingleFlightSharedResponse, which matches the intended tracing semantics.

Given downstream code only reads from res.out and item.response is treated as immutable, this integration looks safe and matches the broader design changes in the PR.

Also applies to: 363-401, 419-427, 115-187, 1652-1700

@jensneuse jensneuse requested a review from endigma November 18, 2025 14:49
Copy link
Contributor

@coderabbitai coderabbitai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actionable comments posted: 0

Caution

Some comments are outside the diff and can’t be posted inline due to platform limitations.

⚠️ Outside diff range comments (1)
v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go (1)

120-163: Fix goroutine closure over loop variables (index, serviceCall, builder).

errGrp.Go closes over index, serviceCall, and builder from the for index, serviceCall := range serviceCalls loop. In Go this reuses the same variables, so all goroutines can race on the last values, corrupting results[index] and sharing the wrong builder/arena.

Capture per‑iteration copies before spawning the goroutine:

-	for index, serviceCall := range serviceCalls {
-		item := d.acquirePoolItem(input, index)
-		poolItems = append(poolItems, item)
-		builder := newJSONBuilder(item.Arena, d.mapping, variables)
-		errGrp.Go(func() error {
+	for index, serviceCall := range serviceCalls {
+		item := d.acquirePoolItem(input, index)
+		poolItems = append(poolItems, item)
+		builder := newJSONBuilder(item.Arena, d.mapping, variables)
+
+		idx := index
+		call := serviceCall
+		b := builder
+
+		errGrp.Go(func() error {
 			// Invoke the gRPC method - this will populate serviceCall.Output
-
-			err := d.cc.Invoke(errGrpCtx, serviceCall.MethodFullName(), serviceCall.Input, serviceCall.Output)
+			err := d.cc.Invoke(errGrpCtx, call.MethodFullName(), call.Input, call.Output)
 			if err != nil {
 				return err
 			}
 
-			response, err := builder.marshalResponseJSON(&serviceCall.RPC.Response, serviceCall.Output)
+			response, err := b.marshalResponseJSON(&call.RPC.Response, call.Output)
 			if err != nil {
 				return err
 			}
@@
-			if serviceCall.RPC.Kind == CallKindEntity {
-				err = builder.validateFederatedResponse(response)
+			if call.RPC.Kind == CallKindEntity {
+				err = b.validateFederatedResponse(response)
 				if err != nil {
 					return err
 				}
 			}
 
-			results[index] = resultData{
-				kind:         serviceCall.RPC.Kind,
+			results[idx] = resultData{
+				kind:         call.RPC.Kind,
 				response:     response,
-				responsePath: serviceCall.RPC.ResponsePath,
+				responsePath: call.RPC.ResponsePath,
 			}
 
 			return nil
 		})
 	}
♻️ Duplicate comments (2)
v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go (1)

172-188: Align merge error handling with GraphQL‑error‑as‑data convention.

On merge failure you set data = builder.writeErrorBytes(err) but still return a non‑nil err. Upstream, Loader treats non‑nil errors as transport failures, while in other branches (e.g. errGrp.Wait) you already return error JSON with a nil error.

To keep behavior consistent and avoid double‑reporting, treat merge failures like other GraphQL errors by marking the operation as failed but returning nil error:

-		for _, result := range results {
+		for _, result := range results {
 			switch result.kind {
 			case CallKindResolve:
 				err = builder.mergeWithPath(root, result.response, result.responsePath)
 			default:
 				root, err = builder.mergeValues(root, result.response)
 			}
 			if err != nil {
-				data = builder.writeErrorBytes(err)
-				return err
+				data = builder.writeErrorBytes(err)
+				failed = true
+				return nil
 			}
 		}
@@
-	}); err != nil || failed {
-		return data, err
-	}
+	}); err != nil || failed {
+		return data, err
+	}

This mirrors the earlier errGrp.Wait() branch and preserves the “GraphQL error JSON + nil Go error” contract.

v2/pkg/engine/resolve/resolve.go (1)

1100-1116: Critical: prepareTrigger still collapses subscriptions to triggerID=0 without a headers builder.

When ctx.SubgraphHeadersBuilder is nil, prepareTrigger returns (nil, 0). That means all subscriptions share triggerID=0, so different GraphQL subscriptions on the same resolver instance will be multiplexed onto a single trigger, causing cross‑subscription collisions (subscriptions with different inputs sharing the same upstream trigger and lifecycle).

The earlier fix proposal was to always hash the input, and only additionally mix in a header hash. You can restore that behavior by making hashing unconditional:

-func (r *Resolver) prepareTrigger(ctx *Context, sourceName string, input []byte) (headers http.Header, triggerID uint64) {
-	if ctx.SubgraphHeadersBuilder != nil {
-		header, headerHash := ctx.SubgraphHeadersBuilder.HeadersForSubgraph(sourceName)
-		keyGen := pool.Hash64.Get()
-		_, _ = keyGen.Write(input)
-		var b [8]byte
-		binary.LittleEndian.PutUint64(b[:], headerHash)
-		_, _ = keyGen.Write(b[:])
-		triggerID = keyGen.Sum64()
-		pool.Hash64.Put(keyGen)
-		return header, triggerID
-	}
-	return nil, 0
-}
+func (r *Resolver) prepareTrigger(ctx *Context, sourceName string, input []byte) (headers http.Header, triggerID uint64) {
+	var (
+		header     http.Header
+		headerHash uint64
+	)
+	if ctx.SubgraphHeadersBuilder != nil {
+		header, headerHash = ctx.SubgraphHeadersBuilder.HeadersForSubgraph(sourceName)
+	}
+	keyGen := pool.Hash64.Get()
+	_, _ = keyGen.Write(input)
+	var b [8]byte
+	binary.LittleEndian.PutUint64(b[:], headerHash)
+	_, _ = keyGen.Write(b[:])
+	triggerID = keyGen.Sum64()
+	pool.Hash64.Put(keyGen)
+	return header, triggerID
+}

This ensures triggerID is unique per (input, headers) even when no SubgraphHeadersBuilder is configured, preserving correct trigger separation.

Also applies to: 1155-1219, 1223-1283

🧹 Nitpick comments (2)
v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go (2)

194-203: Consider documenting Acquire key semantics for arena pool usage.

acquirePoolItem hashes input plus index into a uint64 key for d.pool.Acquire(key). Given you also use per‑request arenas elsewhere keyed only by request ID, a short comment here about intended reuse (per‑fetch vs per‑call vs per‑request) would help future maintainers avoid accidental key collisions or over‑sharing arenas across workloads.

No functional issue, just clarity.


211-213: Replace panic in LoadWithFiles with a safe error for interface callers.

LoadWithFiles now has the correct signature but still panics. Even if gRPC/file‑upload is “impossible” today, panicking from a DataSource implementation turns accidental use into a hard crash.

Consider returning a GraphQL error payload and nil Go error instead of panicking, e.g.:

func (d *DataSource) LoadWithFiles(ctx context.Context, headers http.Header, input []byte, files []*httpclient.FileUpload) ([]byte, error) {
	builder := newJSONBuilder(nil, d.mapping, gjson.Result{})
	return builder.writeErrorBytes(fmt.Errorf("gRPC datasource does not support file uploads")), nil
}
📜 Review details

Configuration used: Path: .coderabbit.yaml

Review profile: CHILL

Plan: Pro

📥 Commits

Reviewing files that changed from the base of the PR and between 57b8680 and 2ccc28c.

⛔ Files ignored due to path filters (1)
  • v2/go.sum is excluded by !**/*.sum
📒 Files selected for processing (6)
  • v2/go.mod (1 hunks)
  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go (7 hunks)
  • v2/pkg/engine/datasource/httpclient/nethttpclient.go (7 hunks)
  • v2/pkg/engine/resolve/loader.go (48 hunks)
  • v2/pkg/engine/resolve/loader_test.go (15 hunks)
  • v2/pkg/engine/resolve/resolve.go (21 hunks)
🚧 Files skipped from review as they are similar to previous changes (2)
  • v2/go.mod
  • v2/pkg/engine/resolve/loader_test.go
🧰 Additional context used
🧠 Learnings (6)
📚 Learning: 2025-09-19T14:50:19.528Z
Learnt from: endigma
Repo: wundergraph/graphql-go-tools PR: 1298
File: v2/pkg/engine/resolve/resolve.go:999-1011
Timestamp: 2025-09-19T14:50:19.528Z
Learning: In the graphql-go-tools resolver, when handling backpressure for async unsubscribe operations, the team prefers using individual goroutines over bounded deferrer implementations. The rationale is that bounded implementations can also overflow, and the preferred solution for overload is scaling the deployment rather than internal buffering. The current approach prevents deadlocks at the cost of goroutine creation, which is an acceptable tradeoff.

Applied to files:

  • v2/pkg/engine/resolve/resolve.go
  • v2/pkg/engine/resolve/loader.go
📚 Learning: 2025-09-19T14:51:33.724Z
Learnt from: endigma
Repo: wundergraph/graphql-go-tools PR: 1298
File: v2/pkg/engine/resolve/resolve.go:339-347
Timestamp: 2025-09-19T14:51:33.724Z
Learning: In the graphql-go-tools resolver, the event loop follows a strict single-threaded design where all subscription lifecycle management (add/remove from triggers) must happen through the events channel to the main processEvents() goroutine. Worker goroutines should not directly modify subscription state or call unsubscribe methods, as this would break the single-threaded event loop architecture and create race conditions.

Applied to files:

  • v2/pkg/engine/resolve/resolve.go
📚 Learning: 2025-07-02T15:28:02.122Z
Learnt from: SkArchon
Repo: wundergraph/graphql-go-tools PR: 1203
File: v2/pkg/engine/resolve/loader.go:63-67
Timestamp: 2025-07-02T15:28:02.122Z
Learning: In the graphql-go-tools codebase, result structs are consistently initialized with non-nil bytes.Buffer instances, making additional nil checks for res.out unnecessary defensive programming.

Applied to files:

  • v2/pkg/engine/resolve/resolve.go
  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go
  • v2/pkg/engine/resolve/loader.go
📚 Learning: 2025-10-16T13:05:19.838Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1323
File: v2/pkg/engine/datasource/grpc_datasource/compiler.go:683-702
Timestamp: 2025-10-16T13:05:19.838Z
Learning: In GraphQL field resolver context resolution (v2/pkg/engine/datasource/grpc_datasource/compiler.go), when traversing paths in resolveContextDataForPath, the code can safely assume that intermediate path segments will only be messages or lists, never scalars. This is because field resolvers are only defined on GraphQL object types, not scalar types, so the parent function must return either a message or a list. This invariant is enforced by the GraphQL type system design.

Applied to files:

  • v2/pkg/engine/resolve/resolve.go
📚 Learning: 2025-08-08T09:43:07.433Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1262
File: v2/pkg/engine/datasource/grpc_datasource/json_builder.go:0-0
Timestamp: 2025-08-08T09:43:07.433Z
Learning: In v2/pkg/engine/datasource/grpc_datasource/json_builder.go, mergeEntities intentionally uses the loop index when calling indexMap.getResultIndex because the index map is type-aware, making per-type counters unnecessary under the current assumptions. Avoid suggesting per-type ordinal counters for this path in future reviews.

Applied to files:

  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go
  • v2/pkg/engine/resolve/loader.go
📚 Learning: 2025-08-29T09:35:47.969Z
Learnt from: ysmolski
Repo: wundergraph/graphql-go-tools PR: 1282
File: v2/pkg/engine/plan/visitor.go:5-5
Timestamp: 2025-08-29T09:35:47.969Z
Learning: The wundergraph/graphql-go-tools project does not support Go versions < 1.23, so compatibility concerns for features available in Go 1.21+ (like cmp.Or) should not be raised.

Applied to files:

  • v2/pkg/engine/resolve/loader.go
🧬 Code graph analysis (4)
v2/pkg/engine/resolve/resolve.go (7)
v2/pkg/engine/resolve/subgraph_request_singleflight.go (2)
  • SubgraphRequestSingleFlight (12-15)
  • NewSingleFlight (52-65)
v2/pkg/engine/resolve/inbound_request_singleflight.go (2)
  • InboundRequestSingleFlight (14-16)
  • NewRequestSingleFlight (26-34)
v2/pkg/engine/resolve/resolvable.go (2)
  • NewResolvable (72-80)
  • ResolvableOptions (65-70)
v2/pkg/engine/resolve/context.go (4)
  • Context (16-38)
  • Request (185-188)
  • ExecutionOptions (60-78)
  • SubgraphHeadersBuilder (44-50)
v2/pkg/engine/resolve/response.go (3)
  • GraphQLResponse (36-44)
  • GraphQLSubscription (12-16)
  • SubscriptionResponseWriter (82-88)
v2/pkg/engine/datasource/staticdatasource/static_datasource.go (3)
  • Source (72-72)
  • Source (74-76)
  • Source (78-80)
v2/pkg/pool/hash64.go (1)
  • Hash64 (10-16)
v2/pkg/engine/datasource/httpclient/nethttpclient.go (2)
v2/pkg/engine/resolve/context.go (1)
  • Context (16-38)
v2/pkg/engine/datasource/httpclient/file.go (1)
  • FileUpload (3-7)
v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go (3)
v2/pkg/engine/resolve/datasource.go (1)
  • DataSource (10-13)
v2/pkg/engine/resolve/resolve.go (1)
  • New (191-256)
v2/pkg/engine/datasource/httpclient/file.go (1)
  • FileUpload (3-7)
v2/pkg/engine/resolve/loader.go (5)
v2/pkg/engine/resolve/fetch.go (8)
  • PostProcessingConfiguration (115-131)
  • FetchItemPathElement (77-81)
  • FetchItem (28-33)
  • DataSourceLoadTrace (374-388)
  • BatchEntityFetch (159-168)
  • BatchEntityFetch (193-195)
  • Fetch (19-26)
  • FetchInfo (345-366)
v2/pkg/engine/resolve/context.go (2)
  • Context (16-38)
  • ExecutionOptions (60-78)
v2/pkg/engine/resolve/inputtemplate.go (1)
  • SetInputUndefinedVariables (39-51)
v2/pkg/engine/resolve/resolve.go (1)
  • New (191-256)
v2/pkg/engine/datasource/httpclient/nethttpclient.go (1)
  • WithHTTPClientSizeHint (142-144)
🔇 Additional comments (14)
v2/pkg/engine/datasource/httpclient/nethttpclient.go (3)

133-154: Size‑hinted buffering helper looks good.

WithHTTPClientSizeHint + buffer(ctx) nicely centralize response buffer sizing and fall back to a small (64‑byte) default. The simple int hint in context keeps it cheap and unobtrusive.


156-279: makeHTTPRequest/Do: consistent error and tracing behavior.

The refactored makeHTTPRequest cleanly:

  • Builds the request with optional baseHeaders, JSON headers, and query params.
  • Streams/decompresses into a single buffer sized via buffer(ctx).
  • Returns []byte or, when tracing, wraps in an extensions.trace object.

Error paths all return (nil, err) and ensure response.Body is closed. This matches the new Do contract and integrates cleanly with the arenas higher up.


286-343: Multipart temp‑file and body cleanup is now safe.

The new DoMultipartForm:

  • Returns early on len(files) == 0 before any resources are opened.
  • Defers cleanup of all tempFiles and moves the os.Open error check before appending, avoiding nil entries.
  • Defers closing multipartBody after successful creation.

This eliminates the previous nil‑deref/potential leak hazards while preserving behavior.

v2/pkg/engine/resolve/resolve.go (3)

77-90: Arena pools and single‑flight fields are wired correctly.

Initializing resolveArenaPool/responseBufferPool via arena.NewArenaPool() and both single‑flight structures in New keeps arenas and deduplication clearly scoped on the Resolver. The fields are only accessed through methods that manage Acquire/Release, so there’s no obvious leak or race at this layer.

Also applies to: 229-247


258-280: ArenaResolveGraphQLResponse: resource and single‑flight lifecycle looks sound.

The arena‑based response path:

  • Checks inboundRequestSingleFlight first and directly writes cached inflight.Data for followers.
  • For leaders, acquires resolveArena and responseArena from separate pools, and releases them on all error and success paths.
  • Calls FinishErr/FinishOk exactly once per leader, after having data or an error.

As long as FinishOk/FinishErr are nil‑safe on inflight when dedup is disabled, this is a clean integration of arena buffering and inbound request dedup.

Please ensure InboundRequestSingleFlight.FinishOk/FinishErr safely handle a nil inflight item when deduplication is disabled.

Also applies to: 318-382


500-571: executeSubscriptionUpdate: arena usage is balanced across all paths.

The per‑update resolveArena := r.resolveArenaPool.Acquire(resolveCtx.Request.ID) is released on:

  • InitSubscription error,
  • LoadGraphQLResponseData error,
  • Resolve error,
  • and after a successful resolve before flushing.

This keeps subscription updates arena‑backed without leaks or double releases.

v2/pkg/engine/resolve/loader.go (8)

51-73: ResponseInfo/result changes align with new []byte‑returning sources.

Storing the subgraph body as responseBody []byte and exposing it via GetResponseBody() cleanly adapts the loader hooks to the new (data []byte, error) DataSource contract. Wiring responseBody: res.out in newResponseInfo matches how executeSourceLoad sets res.out, so hook consumers see the exact bytes used for merging.

Also applies to: 95-132


173-188: jsonArena usage and item selection look consistent with arena rules.

  • Loader.jsonArena is documented as single‑goroutine, resolver‑supplied arena, and used only from the main loader goroutine.
  • selectItemsForPath and selectItems allocate slices via arena.AllocateSlice and grow them with arena.SliceAppend using l.jsonArena.
  • Downstream merging/parsing functions (MergeValuesWithPath etc.) take the same arena, tying value lifetimes correctly.

This keeps data selection and merging arena‑aware without crossing goroutines.

Also applies to: 332-346, 364-402


404-418: itemsData and renderErrorsInvalidInput nicely avoid unsafe arena usage.

  • itemsData now builds arrays via astjson.MustParseBytes and SetArrayItem(nil, ...), avoiding resolvable.astjsonArena to prevent races when used from parallel fetches.
  • renderErrorsInvalidInput returns a concrete error payload and loadSingleFetch writes it into res.out while returning nil error. This matches the new DataSource semantics where GraphQL‑level errors are expressed in the payload, not as Go errors.

Also applies to: 645-665


454-638: mergeResult: arena‑backed parsing and merging look correct.

Key points:

  • Before ParseBytesWithArena, you copy res.out into an arena‑allocated slice, satisfying the “bytes live on same arena” requirement.
  • All subsequent merges (MergeValuesWithPath) use l.jsonArena, and you handle both single-item and batched cases, including res.batchStats buckets, with clear error handling (ErrMergeResult).
  • Tainted indices are propagated to taintedObjs consistently.

This is a solid arena‑aware rewrite of the merge logic.


667-768: Error rendering paths now consistently use jsonArena and ensure errors slice init.

Across:

  • mergeErrors,
  • optionallyEnsureExtensionErrorCode,
  • optionallyAttachServiceNameToErrorExtension,
  • setSubgraphStatusCode,
  • addApolloRouterCompatibilityError,
  • renderErrorsFailedDeps,
  • renderErrorsFailedToFetch,
  • renderErrorsStatusFallback,
  • renderAuthorizationRejectedErrors,
  • renderRateLimitRejectedErrors,

you now:

  • Parse small JSON snippets with astjson.ParseWithArena(l.jsonArena) (or MergeWithPath) and
  • Call l.resolvable.ensureErrorsInitialized() before appending.

This keeps all error objects arena‑backed and avoids the uninitialized slice footgun the comments call out.

Also applies to: 798-849, 895-943, 945-973, 990-1071, 1087-1131, 1133-1181


1247-1286: Batch/entity fetch paths and batchEntityTools pooling look safe.

  • loadSingleFetch and loadEntityFetch now rely on res.out for error payloads and use SetInputUndefinedVariables before calling executeSourceLoad, which matches the new HTTP client contract.
  • batchEntityTools and _batchEntityToolPool encapsulate per‑batch arenas and hash maps; reset clears both before pooling.
  • loadBatchEntityFetch:
    • Uses res.tools = batchEntityToolPool.Get(len(items)) and arena buffers for header/body.
    • Keeps batchStats only on the local arena, then copies it into a heap‑allocated res.batchStats before returning.
    • Ensures all tools are eventually returned via defer batchEntityToolPool.Put(res.tools) in callers.

This design prevents arena lifetime issues and avoids cross‑request sharing of batch metadata.

Also applies to: 1288-1365, 1409-1536


1575-1590: single‑flight integration and tracing fields are wired coherently.

  • singleFlightStats tracks used/shared flags, which executeSourceLoad reports into DataSourceLoadTrace.
  • loaderContextKey + operationTypeContextKey let transports distinguish query vs mutation from context.
  • headersForSubgraphRequest and singleFlightAllowed encode the dedup policy: only per‑subgraph queries are eligible, and dedup can be disabled per‑request.
  • loadByContext/loadByContextDirect:
    • Attach operation type into the context.
    • Use SubgraphHeadersBuilder to supply headers and extra key material.
    • Short‑circuit for shared results, or perform the actual load, recording res.out and caching item.response.

Combined with WithHTTPClientSizeHint to pass item.sizeHint down, this is a clean single‑flight integration that preserves tracing semantics.

Also applies to: 1592-1644, 1646-1694, 1696-1706, 1708-1885


1887-1905: compactJSON: non‑arena dedup path matches arena safety comments.

compactJSON compacts into a heap buffer, parses with astjson.ParseBytes, deduplicates object keys recursively, and marshals back. The explicit comment about not using arena here matches the earlier rules about only parsing arena‑backed bytes with ParseBytesWithArena, so this helper stays on the safe side and off the arena.

Comment on lines 57 to 69
func (s *NatsPublishDataSource) Load(ctx context.Context, headers http.Header, input []byte) (data []byte, err error) {
var publishConfiguration NatsPublishAndRequestEventConfiguration
err := json.Unmarshal(input, &publishConfiguration)
err = json.Unmarshal(input, &publishConfiguration)
if err != nil {
return err
return nil, err
}

if err := s.pubSub.Publish(ctx, publishConfiguration); err != nil {
_, err = io.WriteString(out, `{"success": false}`)
return err
return []byte(`{"success": false}`), err
}

_, err = io.WriteString(out, `{"success": true}`)
return err
return []byte(`{"success": true}`), nil
}
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Makes sense @jensneuse

# Conflicts:
#	v2/pkg/engine/datasource/graphql_datasource/graphql_datasource.go
#	v2/pkg/engine/datasource/pubsub_datasource/pubsub_datasource_test.go
#	v2/pkg/engine/datasource/pubsub_datasource/pubsub_kafka.go
#	v2/pkg/engine/datasource/pubsub_datasource/pubsub_nats.go
#	v2/pkg/engine/resolve/resolve.go
#	v2/pkg/engine/resolve/resolve_test.go
@jensneuse jensneuse requested a review from StarpTech November 24, 2025 09:12
Copy link
Collaborator

@StarpTech StarpTech left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

Copy link
Contributor

@coderabbitai coderabbitai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actionable comments posted: 1

♻️ Duplicate comments (5)
v2/pkg/engine/resolve/resolve_test.go (1)

349-355: Prefer DoAndReturn instead of Do(...).Return(...) for this gomock expectation.

Here the Do callback returns ([]byte, error) but gomock ignores those values and instead uses the ones from Return(...). Using DoAndReturn keeps the behavior in a single place and avoids accidental divergence between the callback and the declared return values.

You can simplify this to:

- mockDataSource.EXPECT().
-   Load(gomock.Any(), gomock.Any(), []byte(`{"id":1}`)).
-   Do(func(ctx context.Context, headers http.Header, input []byte) ([]byte, error) {
-     return []byte(`{"name":"Jens"}`), nil
-   }).
-   Return([]byte(`{"name":"Jens"}`), nil)
+ mockDataSource.EXPECT().
+   Load(gomock.Any(), gomock.Any(), []byte(`{"id":1}`)).
+   DoAndReturn(func(ctx context.Context, headers http.Header, input []byte) ([]byte, error) {
+     return []byte(`{"name":"Jens"}`), nil
+   })

This matches how other expectations in this file are written.

v2/pkg/engine/resolve/inbound_request_singleflight.go (1)

36-44: Fix single-flight follower race and shared Err write to avoid inconsistent results and data races.

The current leader/follower protocol has two concurrency problems:

  1. Race on HasFollowers → followers can see Data == nil on success.

    • A follower can execute Load and obtain req before the leader calls FinishOk, but lose the race on Mu so that FinishOk reads HasFollowers == false, skips copying data, closes Done, and only afterwards the follower sets HasFollowers = true.
    • In that case the follower wakes from <-req.Done with req.Err == nil but req.Data == nil, violating the expectation that successful deduplicated requests expose a valid response slice.
  2. Data race on Err between followers and FinishErr.

    • In the follower path, on ctx.ctx.Done() you assign inflightRequest.Err = ctx.ctx.Err() without synchronization, while FinishErr can concurrently assign req.Err = err. This is an unsynchronized multi-writer to the same field and will be flagged by go test -race.

Given that the response bytes are immutable and only copied once per in‑flight request, it’s simpler and safer to always copy data in FinishOk (before closing Done) and to avoid mutating Err in followers.

A possible fix:

 type InflightRequest struct {
   Done chan struct{}
   Data []byte
   Err  error
   ID   uint64
-
-  HasFollowers bool
-  Mu           sync.Mutex
 }

 func (r *InboundRequestSingleFlight) GetOrCreate(ctx *Context, response *GraphQLResponse) (*InflightRequest, error) {
@@
   shard := r.shardFor(key)
   req, shared := shard.m.Load(key)
   if shared {
-    inflightRequest := req.(*InflightRequest)
-    inflightRequest.Mu.Lock()
-    inflightRequest.HasFollowers = true
-    inflightRequest.Mu.Unlock()
+    inflightRequest := req.(*InflightRequest)
     select {
     case <-inflightRequest.Done:
       if inflightRequest.Err != nil {
         return nil, inflightRequest.Err
       }
       return inflightRequest, nil
     case <-ctx.ctx.Done():
-      inflightRequest.Err = ctx.ctx.Err()
-      return nil, inflightRequest.Err
+      // Don’t mutate shared Err; just return caller’s context error.
+      return nil, ctx.ctx.Err()
     }
   }
@@
 func (r *InboundRequestSingleFlight) FinishOk(req *InflightRequest, data []byte) {
   if req == nil {
     return
   }
   shard := r.shardFor(req.ID)
   shard.m.Delete(req.ID)
-  req.Mu.Lock()
-  hasFollowers := req.HasFollowers
-  req.Mu.Unlock()
-  if hasFollowers {
-    // optimization to only copy when we actually have to
-    req.Data = make([]byte, len(data))
-    copy(req.Data, data)
-  }
+  // Always copy so any followers see a valid immutable slice.
+  if data != nil {
+    buf := make([]byte, len(data))
+    copy(buf, data)
+    req.Data = buf
+  }
   close(req.Done)
 }

This removes the racy optimization, guarantees that any successful follower sees a valid Data slice, and eliminates the unsynchronized Err write from the follower path.

Also applies to: 73-99, 101-126

v2/pkg/engine/resolve/datasource.go (1)

10-12: Document headers immutability and []byte ownership for all datasource interfaces

The new headers http.Header and []byte return contracts are not documented on DataSource, SubscriptionDataSource, or AsyncSubscriptionDataSource. Given the arena-based optimizations and potential buffer reuse, it’s important to spell out that implementers:

  • Must treat headers as read‑only (no mutation).
  • Must return a caller‑owned []byte (no sharing of mutable internal buffers / arena-backed slices).

Adding concise comments at the interface level will prevent subtle corruption bugs in future datasource implementations.

 type DataSource interface {
-	Load(ctx context.Context, headers http.Header, input []byte) (data []byte, err error)
-	LoadWithFiles(ctx context.Context, headers http.Header, input []byte, files []*httpclient.FileUpload) (data []byte, err error)
+	// Load executes a request to the data source using the provided immutable headers and input.
+	// Implementations must not mutate headers and must return a caller-owned byte slice
+	// (do not share internal or arena-backed buffers that may be reused).
+	Load(ctx context.Context, headers http.Header, input []byte) (data []byte, err error)
+	// LoadWithFiles behaves like Load but also includes file uploads; the same immutability
+	// and ownership rules apply.
+	LoadWithFiles(ctx context.Context, headers http.Header, input []byte, files []*httpclient.FileUpload) (data []byte, err error)
 }
 
 type SubscriptionDataSource interface {
-	// Start is called when a new subscription is created. It establishes the connection to the data source.
-	// The updater is used to send updates to the client. Deduplication of the request must be done before calling this method.
-	Start(ctx *Context, headers http.Header, input []byte, updater SubscriptionUpdater) error
+	// Start is called when a new subscription is created. It establishes the connection to the data source.
+	// The updater is used to send updates to the client. Deduplication of the request must be done before calling this method.
+	// headers must be treated as immutable; implementations must not mutate them.
+	Start(ctx *Context, headers http.Header, input []byte, updater SubscriptionUpdater) error
 }
 
 type AsyncSubscriptionDataSource interface {
-	AsyncStart(ctx *Context, id uint64, headers http.Header, input []byte, updater SubscriptionUpdater) error
+	// AsyncStart starts a subscription asynchronously for the given id.
+	// headers must be treated as immutable; implementations must not mutate them.
+	AsyncStart(ctx *Context, id uint64, headers http.Header, input []byte, updater SubscriptionUpdater) error
+	// AsyncStop stops the subscription with the given id.
 	AsyncStop(id uint64)
 }

Also applies to: 15-23

v2/pkg/engine/datasource/graphql_datasource/graphql_datasource_test.go (1)

8882-8897: Close temp files created with os.CreateTemp in file upload tests

The temp files created in both subtests (f, f1, f2) are never closed, which can leak file descriptors in longer test runs/CI, even though t.TempDir cleans up paths on disk.

You can keep using os.WriteFile, but still close the original descriptors right after creation:

@@
-		f, err := os.CreateTemp(dir, fileName)
-		assert.NoError(t, err)
+		f, err := os.CreateTemp(dir, fileName)
+		assert.NoError(t, err)
+		defer func() { _ = f.Close() }()
@@
-		f1, err := os.CreateTemp(dir, file1Name)
-		assert.NoError(t, err)
+		f1, err := os.CreateTemp(dir, file1Name)
+		assert.NoError(t, err)
+		defer func() { _ = f1.Close() }()
@@
-		f2, err := os.CreateTemp(dir, file2Name)
-		assert.NoError(t, err)
+		f2, err := os.CreateTemp(dir, file2Name)
+		assert.NoError(t, err)
+		defer func() { _ = f2.Close() }()

The new LoadWithFiles calls and expectations (got being an empty slice) otherwise look fine given the server already validates the multipart body via verifyMultipartRequest.

Also applies to: 8927-8957

v2/pkg/engine/resolve/subgraph_request_singleflight.go (1)

138-188: Critical: xxhash digest pool misuse causes resource leak and inefficiency.

The digest pool is incorrectly used in computeKeys and computeSFKey:

  1. computeKeys (lines 138-146): Gets a digest h from the pool but never uses it—just resets and returns it. Meanwhile, computeSFKey and computeFetchKey each independently get their own digests, making h completely redundant.

  2. computeSFKey (lines 150-161): Gets a digest from the pool (line 151) but never returns it with Put() or uses defer. This is a resource leak—each call permanently removes a digest from the pool.

  3. computeFetchKey (lines 166-188): Correctly uses defer s.xxPool.Put(h) on line 168. This is the pattern that should be used throughout.

Additionally, as noted in past reviews, line 160 adds extraKey arithmetically (h.Sum64() + extraKey), which can cause hash collisions. The safer approach is to write extraKey bytes into the hasher before calling Sum64().

Apply this diff to fix the pool usage and hash mixing:

 func (s *SubgraphRequestSingleFlight) computeKeys(fetchItem *FetchItem, input []byte, extraKey uint64) (sfKey, fetchKey uint64) {
 	h := s.xxPool.Get().(*xxhash.Digest)
-	sfKey = s.computeSFKey(fetchItem, input, extraKey)
-	h.Reset()
-	fetchKey = s.computeFetchKey(fetchItem)
-	h.Reset()
-	s.xxPool.Put(h)
+	defer s.xxPool.Put(h)
+
+	h.Reset()
+	sfKey = s.computeSFKeyWithDigest(h, fetchItem, input, extraKey)
+
+	h.Reset()
+	fetchKey = s.computeFetchKeyWithDigest(h, fetchItem)
+
 	return sfKey, fetchKey
 }
 
-func (s *SubgraphRequestSingleFlight) computeSFKey(fetchItem *FetchItem, input []byte, extraKey uint64) uint64 {
-	h := s.xxPool.Get().(*xxhash.Digest)
+func (s *SubgraphRequestSingleFlight) computeSFKeyWithDigest(h *xxhash.Digest, fetchItem *FetchItem, input []byte, extraKey uint64) uint64 {
 	if fetchItem != nil && fetchItem.Fetch != nil {
 		info := fetchItem.Fetch.FetchInfo()
 		if info != nil {
 			_, _ = h.WriteString(info.DataSourceID)
 			_, _ = h.WriteString(":")
 		}
 	}
 	_, _ = h.Write(input)
-	return h.Sum64() + extraKey // extraKey in this case is the pre-generated hash for the headers
+	// Mix header hash into digest instead of adding arithmetically
+	var b [8]byte
+	binary.LittleEndian.PutUint64(b[:], extraKey)
+	_, _ = h.Write(b[:])
+	return h.Sum64()
 }
 
-func (s *SubgraphRequestSingleFlight) computeFetchKey(fetchItem *FetchItem) uint64 {
-	h := s.xxPool.Get().(*xxhash.Digest)
-	defer s.xxPool.Put(h)
+func (s *SubgraphRequestSingleFlight) computeFetchKeyWithDigest(h *xxhash.Digest, fetchItem *FetchItem) uint64 {
 	if fetchItem == nil || fetchItem.Fetch == nil {
 		return 0
 	}
 	info := fetchItem.Fetch.FetchInfo()
 	if info == nil {
 		return 0
 	}
 	_, _ = h.WriteString(info.DataSourceID)
 	_, _ = h.Write(pipe)
 	for i := range info.RootFields {
 		if i != 0 {
 			_, _ = h.Write(comma)
 		}
 		_, _ = h.WriteString(info.RootFields[i].TypeName)
 		_, _ = h.Write(dot)
 		_, _ = h.WriteString(info.RootFields[i].FieldName)
 	}
-	sum := h.Sum64()
-	return sum
+	return h.Sum64()
 }

Don't forget to add "encoding/binary" to the imports at the top of the file.

🧹 Nitpick comments (1)
v2/pkg/engine/datasource/graphql_datasource/graphql_datasource_test.go (1)

8466-8542: Updated SubscriptionSource.Start calls match new signature

Passing nil for the new second parameter and keeping the existing options/updater wiring preserves the intent of these tests (invalid input, client error, normal chat flows) under the new Start API. Consider adding a dedicated test later that exercises non‑nil headers if you want coverage for header/dedup behavior, but it’s not required for this PR.

Also applies to: 8596-8650

📜 Review details

Configuration used: Path: .coderabbit.yaml

Review profile: CHILL

Plan: Pro

📥 Commits

Reviewing files that changed from the base of the PR and between 2ccc28c and 9e6c198.

📒 Files selected for processing (8)
  • v2/pkg/engine/datasource/graphql_datasource/graphql_datasource.go (3 hunks)
  • v2/pkg/engine/datasource/graphql_datasource/graphql_datasource_test.go (14 hunks)
  • v2/pkg/engine/datasource/httpclient/nethttpclient.go (7 hunks)
  • v2/pkg/engine/resolve/datasource.go (1 hunks)
  • v2/pkg/engine/resolve/inbound_request_singleflight.go (1 hunks)
  • v2/pkg/engine/resolve/resolve.go (21 hunks)
  • v2/pkg/engine/resolve/resolve_test.go (29 hunks)
  • v2/pkg/engine/resolve/subgraph_request_singleflight.go (1 hunks)
🧰 Additional context used
🧠 Learnings (11)
📓 Common learnings
Learnt from: endigma
Repo: wundergraph/graphql-go-tools PR: 1298
File: v2/pkg/engine/resolve/resolve.go:999-1011
Timestamp: 2025-09-19T14:50:19.528Z
Learning: In the graphql-go-tools resolver, when handling backpressure for async unsubscribe operations, the team prefers using individual goroutines over bounded deferrer implementations. The rationale is that bounded implementations can also overflow, and the preferred solution for overload is scaling the deployment rather than internal buffering. The current approach prevents deadlocks at the cost of goroutine creation, which is an acceptable tradeoff.
📚 Learning: 2025-11-19T10:53:06.342Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1341
File: v2/pkg/engine/datasource/grpc_datasource/execution_plan.go:1039-1097
Timestamp: 2025-11-19T10:53:06.342Z
Learning: In v2/pkg/engine/datasource/grpc_datasource field resolver response handling, the `resolveRequiredFields` function intentionally uses two distinct approaches: for simple GraphQL object types it populates `message.Fields`, while for composite types (interface/union) it exclusively uses `message.FieldSelectionSet` with fragment-based selections. This differs from `buildFieldMessage` (regular queries) because field resolver responses returning composite types must align with protobuf oneOf structure, where all selections—including common interface fields—are handled through fragment selections built by `buildCompositeField`. The two approaches cannot be mixed in field resolver responses.

Applied to files:

  • v2/pkg/engine/resolve/resolve.go
  • v2/pkg/engine/resolve/resolve_test.go
  • v2/pkg/engine/datasource/graphql_datasource/graphql_datasource_test.go
📚 Learning: 2025-09-19T14:50:19.528Z
Learnt from: endigma
Repo: wundergraph/graphql-go-tools PR: 1298
File: v2/pkg/engine/resolve/resolve.go:999-1011
Timestamp: 2025-09-19T14:50:19.528Z
Learning: In the graphql-go-tools resolver, when handling backpressure for async unsubscribe operations, the team prefers using individual goroutines over bounded deferrer implementations. The rationale is that bounded implementations can also overflow, and the preferred solution for overload is scaling the deployment rather than internal buffering. The current approach prevents deadlocks at the cost of goroutine creation, which is an acceptable tradeoff.

Applied to files:

  • v2/pkg/engine/resolve/resolve.go
  • v2/pkg/engine/resolve/inbound_request_singleflight.go
  • v2/pkg/engine/resolve/resolve_test.go
  • v2/pkg/engine/datasource/graphql_datasource/graphql_datasource_test.go
📚 Learning: 2025-09-19T14:51:33.724Z
Learnt from: endigma
Repo: wundergraph/graphql-go-tools PR: 1298
File: v2/pkg/engine/resolve/resolve.go:339-347
Timestamp: 2025-09-19T14:51:33.724Z
Learning: In the graphql-go-tools resolver, the event loop follows a strict single-threaded design where all subscription lifecycle management (add/remove from triggers) must happen through the events channel to the main processEvents() goroutine. Worker goroutines should not directly modify subscription state or call unsubscribe methods, as this would break the single-threaded event loop architecture and create race conditions.

Applied to files:

  • v2/pkg/engine/resolve/resolve.go
  • v2/pkg/engine/resolve/inbound_request_singleflight.go
  • v2/pkg/engine/datasource/graphql_datasource/graphql_datasource_test.go
📚 Learning: 2025-11-19T09:42:17.644Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1341
File: v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go:406-429
Timestamp: 2025-11-19T09:42:17.644Z
Learning: In the wundergraph/graphql-go-tools gRPC datasource implementation (v2/pkg/engine/datasource/grpc_datasource), field resolvers must have arguments. The system does not currently support defining field resolvers without arguments. This invariant ensures that the `parentCallID` increment in `enterFieldResolver` is always matched by a decrement in `LeaveField` (which checks `r.operation.FieldHasArguments(ref)`).

Applied to files:

  • v2/pkg/engine/resolve/resolve.go
  • v2/pkg/engine/resolve/inbound_request_singleflight.go
  • v2/pkg/engine/resolve/resolve_test.go
📚 Learning: 2025-07-02T15:28:02.122Z
Learnt from: SkArchon
Repo: wundergraph/graphql-go-tools PR: 1203
File: v2/pkg/engine/resolve/loader.go:63-67
Timestamp: 2025-07-02T15:28:02.122Z
Learning: In the graphql-go-tools codebase, result structs are consistently initialized with non-nil bytes.Buffer instances, making additional nil checks for res.out unnecessary defensive programming.

Applied to files:

  • v2/pkg/engine/resolve/resolve.go
  • v2/pkg/engine/resolve/inbound_request_singleflight.go
  • v2/pkg/engine/resolve/resolve_test.go
📚 Learning: 2025-10-16T13:05:19.838Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1323
File: v2/pkg/engine/datasource/grpc_datasource/compiler.go:683-702
Timestamp: 2025-10-16T13:05:19.838Z
Learning: In GraphQL field resolver context resolution (v2/pkg/engine/datasource/grpc_datasource/compiler.go), when traversing paths in resolveContextDataForPath, the code can safely assume that intermediate path segments will only be messages or lists, never scalars. This is because field resolvers are only defined on GraphQL object types, not scalar types, so the parent function must return either a message or a list. This invariant is enforced by the GraphQL type system design.

Applied to files:

  • v2/pkg/engine/resolve/resolve.go
  • v2/pkg/engine/resolve/resolve_test.go
📚 Learning: 2025-08-29T09:35:47.969Z
Learnt from: ysmolski
Repo: wundergraph/graphql-go-tools PR: 1282
File: v2/pkg/engine/plan/visitor.go:5-5
Timestamp: 2025-08-29T09:35:47.969Z
Learning: The wundergraph/graphql-go-tools project does not support Go versions < 1.23, so compatibility concerns for features available in Go 1.21+ (like cmp.Or) should not be raised.

Applied to files:

  • v2/pkg/engine/resolve/inbound_request_singleflight.go
📚 Learning: 2025-10-15T13:34:15.892Z
Learnt from: ysmolski
Repo: wundergraph/graphql-go-tools PR: 1322
File: v2/pkg/astvalidation/operation_rule_defer_stream_on_root_fields.go:92-127
Timestamp: 2025-10-15T13:34:15.892Z
Learning: In the graphql-go-tools repository, validation for defer and stream directives runs after normalization, which performs fragment inlining. Therefore, fragment spreads don't exist in the AST when these validation rules execute—they're already expanded into inline fragments or fields.

Applied to files:

  • v2/pkg/engine/resolve/inbound_request_singleflight.go
📚 Learning: 2025-11-19T09:38:25.112Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1341
File: v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go:1418-1526
Timestamp: 2025-11-19T09:38:25.112Z
Learning: In v2/pkg/engine/datasource/grpc_datasource test files, ResolvePath fields use snake_case (e.g., "test_containers.id") because they reference proto message field names, while JSONPath fields use camelCase (e.g., "testContainers") because they reference GraphQL/JSON response field names. Both casing styles are intentional and correct.

Applied to files:

  • v2/pkg/engine/resolve/resolve_test.go
  • v2/pkg/engine/datasource/graphql_datasource/graphql_datasource_test.go
📚 Learning: 2025-10-16T08:52:33.278Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1323
File: v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go:59-63
Timestamp: 2025-10-16T08:52:33.278Z
Learning: Go 1.24 (released February 2025) introduced the testing.B.Loop() method for benchmarks. Use `for b.Loop() { /* code */ }` instead of `for i := 0; i < b.N; i++` in benchmark functions when the Go version is 1.24 or higher. The Loop() method provides more predictable benchmarking results by running the loop exactly once per -count and preventing certain compiler optimizations.

Applied to files:

  • v2/pkg/engine/resolve/resolve_test.go
🧬 Code graph analysis (8)
v2/pkg/engine/datasource/graphql_datasource/graphql_datasource.go (3)
v2/pkg/engine/datasource/staticdatasource/static_datasource.go (3)
  • Source (72-72)
  • Source (74-76)
  • Source (78-80)
v2/pkg/engine/datasource/httpclient/file.go (1)
  • FileUpload (3-7)
v2/pkg/engine/datasource/httpclient/nethttpclient.go (2)
  • DoMultipartForm (286-339)
  • Do (281-284)
v2/pkg/engine/resolve/resolve.go (1)
v2/pkg/engine/resolve/context.go (4)
  • Context (16-38)
  • Request (185-188)
  • ExecutionOptions (60-78)
  • SubgraphHeadersBuilder (44-50)
v2/pkg/engine/resolve/subgraph_request_singleflight.go (3)
v2/pkg/engine/resolve/resolve.go (1)
  • New (191-256)
v2/pkg/engine/resolve/fetch.go (2)
  • Fetch (19-26)
  • FetchInfo (345-366)
v2/pkg/ast/path.go (1)
  • FieldName (23-23)
v2/pkg/engine/resolve/datasource.go (3)
v2/pkg/engine/resolve/context.go (1)
  • Context (16-38)
v2/pkg/engine/datasource/httpclient/file.go (1)
  • FileUpload (3-7)
v2/pkg/engine/resolve/resolve.go (1)
  • SubscriptionUpdater (1541-1554)
v2/pkg/engine/resolve/inbound_request_singleflight.go (2)
v2/pkg/engine/resolve/context.go (4)
  • Context (16-38)
  • ExecutionOptions (60-78)
  • Request (185-188)
  • SubgraphHeadersBuilder (44-50)
v2/pkg/engine/resolve/response.go (1)
  • GraphQLResponse (36-44)
v2/pkg/engine/resolve/resolve_test.go (1)
v2/pkg/engine/resolve/datasource.go (1)
  • DataSource (10-13)
v2/pkg/engine/datasource/httpclient/nethttpclient.go (2)
v2/pkg/engine/datasource/httpclient/file.go (1)
  • FileUpload (3-7)
v2/pkg/fastbuffer/fastbuffer.go (1)
  • New (7-11)
v2/pkg/engine/datasource/graphql_datasource/graphql_datasource_test.go (3)
v2/pkg/engine/resolve/context.go (1)
  • NewContext (190-197)
v2/pkg/engine/datasource/graphql_datasource/graphql_datasource.go (1)
  • SubscriptionSource (1954-1957)
v2/pkg/engine/datasource/httpclient/file.go (2)
  • FileUpload (3-7)
  • NewFileUpload (9-15)
⏰ Context from checks skipped due to timeout of 90000ms. You can increase the timeout in your CodeRabbit configuration to a maximum of 15 minutes (900000ms). (4)
  • GitHub Check: Build and test (go 1.25 / ubuntu-latest)
  • GitHub Check: Build and test (go 1.25 / windows-latest)
  • GitHub Check: Build and test (go 1.25 / windows-latest)
  • GitHub Check: Build and test (go 1.25 / ubuntu-latest)
🔇 Additional comments (12)
v2/pkg/engine/datasource/httpclient/nethttpclient.go (1)

139-154: Size‑hinted buffering for HTTP responses looks good.

The per‑context WithHTTPClientSizeHint + buffer(ctx) approach is a reasonable way to cut allocations (when hints are available) while keeping a small default (64 bytes) when not. The comments also clearly document the trade‑offs, so this change looks solid.

v2/pkg/engine/resolve/datasource.go (1)

3-8: Importing net/http here is appropriate

Using http.Header in the public interfaces justifies the new net/http import; no issues spotted.

v2/pkg/engine/datasource/graphql_datasource/graphql_datasource_test.go (2)

4005-4013: Subscription trigger metadata wiring looks consistent

Setting SourceName and SourceID to "ds-id" in the subscription triggers matches the configured datasource ID and keeps metadata available to the engine/event loop without changing test behavior. No issues here.

Also applies to: 4043-4056


8781-8790: Source.Load return value usage aligns with new API

Capturing the []byte from Load(ctx, nil, input) and asserting on the exact JSON payload maintains the previous semantics of validating null/undefined variable handling via the echo test server. The change from writer‑based to return‑value‑based I/O here looks correct.

Also applies to: 8797-8812

v2/pkg/engine/datasource/graphql_datasource/graphql_datasource.go (2)

1910-1918: LGTM! API signatures updated correctly.

The Load and LoadWithFiles methods now properly accept headers and return data bytes directly, consistent with the broader API changes throughout the codebase. The headers are correctly propagated to the underlying HTTP client functions.


1959-1990: LGTM! Subscription header propagation implemented correctly.

The AsyncStart and Start methods now properly accept headers and assign them to the options struct, enabling per-subscription header context. The implementation correctly threads headers through the subscription lifecycle.

v2/pkg/engine/resolve/resolve.go (6)

77-90: LGTM! Arena pool architecture well-documented.

The comments clearly explain why two separate arena pools are needed (resolving vs. response buffering have different memory characteristics). The addition of single-flight mechanisms for both subgraph and inbound request deduplication aligns well with the PR objectives.


243-246: LGTM! Proper initialization of arena pools and single-flight mechanisms.

The arena pools and single-flight structures are correctly initialized. The shard count of 8 provides a reasonable balance between concurrency and overhead.


258-280: LGTM! Arena threading implemented correctly in newTools.

The function now properly accepts an arena parameter and threads it through to both the Resolvable and Loader. The single-flight mechanism is also correctly wired, enabling both arena-backed resolution and request deduplication.


318-382: LGTM! Arena lifecycle and single-flight integration well-implemented.

The ArenaResolveGraphQLResponse function correctly implements arena-backed resolution with proper lifecycle management:

  • Resolve arena acquired early, released after data is in response arena
  • Response arena acquired after loading, released after client write
  • All error paths properly release acquired arenas
  • Single-flight deduplication correctly handles leader/follower scenarios

The separation of resolve and response arenas enables efficient resource usage by releasing the resolve arena before the potentially slow client write operation.


523-562: LGTM! Subscription update arena lifecycle correctly managed.

The executeSubscriptionUpdate function properly acquires and releases the resolve arena across all code paths, including error handling branches. This enables efficient memory reuse for subscription updates.


1237-1264: Subscription integration with prepareTrigger is structurally correct.

Both ResolveGraphQLSubscription and AsyncResolveGraphQLSubscription correctly call prepareTrigger and propagate headers through to the addSubscription structure. The integration is consistent across sync and async subscription paths.

However, the effectiveness of this deduplication depends on fixing the critical issue in prepareTrigger (see previous comment) where triggerID=0 is returned when no headers builder is present.

Verify that the prepareTrigger fix is applied to ensure proper subscription deduplication.

Also applies to: 1341-1362

@jensneuse jensneuse requested a review from dkorittki November 24, 2025 12:16
Copy link
Contributor

@dkorittki dkorittki left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can't really tell much about the arena changes per se because I lack knowledge in the engine. When it comes to subscriptions I only found what I commented down below but I would like to test this together with the router to get a better feeling for it.

I left out test files for now, will do this tomorrow. This way you can already look at what I found.

EDIT: Checked test files, lgtm

Copy link
Contributor

@coderabbitai coderabbitai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actionable comments posted: 2

📜 Review details

Configuration used: Path: .coderabbit.yaml

Review profile: CHILL

Plan: Pro

📥 Commits

Reviewing files that changed from the base of the PR and between 9e6c198 and c8b5567.

📒 Files selected for processing (2)
  • v2/pkg/astnormalization/uploads/upload_finder.go (1 hunks)
  • v2/pkg/engine/resolve/resolve.go (21 hunks)
🚧 Files skipped from review as they are similar to previous changes (1)
  • v2/pkg/astnormalization/uploads/upload_finder.go
🧰 Additional context used
🧠 Learnings (6)
📚 Learning: 2025-11-19T10:53:06.342Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1341
File: v2/pkg/engine/datasource/grpc_datasource/execution_plan.go:1039-1097
Timestamp: 2025-11-19T10:53:06.342Z
Learning: In v2/pkg/engine/datasource/grpc_datasource field resolver response handling, the `resolveRequiredFields` function intentionally uses two distinct approaches: for simple GraphQL object types it populates `message.Fields`, while for composite types (interface/union) it exclusively uses `message.FieldSelectionSet` with fragment-based selections. This differs from `buildFieldMessage` (regular queries) because field resolver responses returning composite types must align with protobuf oneOf structure, where all selections—including common interface fields—are handled through fragment selections built by `buildCompositeField`. The two approaches cannot be mixed in field resolver responses.

Applied to files:

  • v2/pkg/engine/resolve/resolve.go
📚 Learning: 2025-09-19T14:50:19.528Z
Learnt from: endigma
Repo: wundergraph/graphql-go-tools PR: 1298
File: v2/pkg/engine/resolve/resolve.go:999-1011
Timestamp: 2025-09-19T14:50:19.528Z
Learning: In the graphql-go-tools resolver, when handling backpressure for async unsubscribe operations, the team prefers using individual goroutines over bounded deferrer implementations. The rationale is that bounded implementations can also overflow, and the preferred solution for overload is scaling the deployment rather than internal buffering. The current approach prevents deadlocks at the cost of goroutine creation, which is an acceptable tradeoff.

Applied to files:

  • v2/pkg/engine/resolve/resolve.go
📚 Learning: 2025-09-19T14:51:33.724Z
Learnt from: endigma
Repo: wundergraph/graphql-go-tools PR: 1298
File: v2/pkg/engine/resolve/resolve.go:339-347
Timestamp: 2025-09-19T14:51:33.724Z
Learning: In the graphql-go-tools resolver, the event loop follows a strict single-threaded design where all subscription lifecycle management (add/remove from triggers) must happen through the events channel to the main processEvents() goroutine. Worker goroutines should not directly modify subscription state or call unsubscribe methods, as this would break the single-threaded event loop architecture and create race conditions.

Applied to files:

  • v2/pkg/engine/resolve/resolve.go
📚 Learning: 2025-11-19T09:42:17.644Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1341
File: v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go:406-429
Timestamp: 2025-11-19T09:42:17.644Z
Learning: In the wundergraph/graphql-go-tools gRPC datasource implementation (v2/pkg/engine/datasource/grpc_datasource), field resolvers must have arguments. The system does not currently support defining field resolvers without arguments. This invariant ensures that the `parentCallID` increment in `enterFieldResolver` is always matched by a decrement in `LeaveField` (which checks `r.operation.FieldHasArguments(ref)`).

Applied to files:

  • v2/pkg/engine/resolve/resolve.go
📚 Learning: 2025-07-02T15:28:02.122Z
Learnt from: SkArchon
Repo: wundergraph/graphql-go-tools PR: 1203
File: v2/pkg/engine/resolve/loader.go:63-67
Timestamp: 2025-07-02T15:28:02.122Z
Learning: In the graphql-go-tools codebase, result structs are consistently initialized with non-nil bytes.Buffer instances, making additional nil checks for res.out unnecessary defensive programming.

Applied to files:

  • v2/pkg/engine/resolve/resolve.go
📚 Learning: 2025-10-16T13:05:19.838Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1323
File: v2/pkg/engine/datasource/grpc_datasource/compiler.go:683-702
Timestamp: 2025-10-16T13:05:19.838Z
Learning: In GraphQL field resolver context resolution (v2/pkg/engine/datasource/grpc_datasource/compiler.go), when traversing paths in resolveContextDataForPath, the code can safely assume that intermediate path segments will only be messages or lists, never scalars. This is because field resolvers are only defined on GraphQL object types, not scalar types, so the parent function must return either a message or a list. This invariant is enforced by the GraphQL type system design.

Applied to files:

  • v2/pkg/engine/resolve/resolve.go
🧬 Code graph analysis (1)
v2/pkg/engine/resolve/resolve.go (6)
v2/pkg/engine/resolve/subgraph_request_singleflight.go (2)
  • SubgraphRequestSingleFlight (12-15)
  • NewSingleFlight (52-65)
v2/pkg/engine/resolve/inbound_request_singleflight.go (2)
  • InboundRequestSingleFlight (14-16)
  • NewRequestSingleFlight (26-34)
v2/pkg/engine/resolve/resolvable.go (2)
  • NewResolvable (72-80)
  • ResolvableOptions (65-70)
v2/pkg/engine/resolve/context.go (4)
  • Context (16-38)
  • Request (185-188)
  • ExecutionOptions (60-78)
  • SubgraphHeadersBuilder (44-50)
v2/pkg/engine/resolve/response.go (2)
  • GraphQLResponse (36-44)
  • SubscriptionResponseWriter (82-88)
v2/pkg/pool/hash64.go (1)
  • Hash64 (10-16)
⏰ Context from checks skipped due to timeout of 90000ms. You can increase the timeout in your CodeRabbit configuration to a maximum of 15 minutes (900000ms). (1)
  • GitHub Check: CI Success
🔇 Additional comments (2)
v2/pkg/engine/resolve/resolve.go (2)

1200-1264: Headers & triggerID plumbing through subscription events looks consistent

The new flow:

  • Computes (headers, triggerID) via prepareTrigger,
  • Carries both through ResolveGraphQLSubscription / AsyncResolveGraphQLSubscription into addSubscription,
  • And finally passes them into AsyncStart / Start in handleAddSubscription,

is consistent with the single‑threaded event loop design (all state changes still go through r.events) and with SubgraphHeadersBuilder semantics (per‑subgraph headers plus hash). This should give subgraph data sources everything they need to start triggers with the correct forwarded headers while also grouping subscriptions by a stable trigger ID.

No functional changes requested here.

Also applies to: 1341-1362, 1516-1524


318-382: I'll verify the two concerns raised in the review comment about nil inflight safety and arena data lifetime. Let me start by examining the InboundRequestSingleFlight implementation.
<function_calls>


#!/bin/bash

Find the InboundRequestSingleFlight implementation

fd -t f "inbound.*singleflight" -i | head -20


</function_calls>


#!/bin/bash

Search for InboundRequestSingleFlight type definition and methods

rg -t go "type InboundRequestSingleFlight" -A 5 | head -50


</function_calls>


#!/bin/bash

Search more broadly for InboundRequestSingleFlight

rg -t go "InboundRequestSingleFlight" -C 3 | grep -E "(type|func|GetOrCreate|FinishOk|FinishErr)" | head -40


</function_calls>


#!/bin/bash

List Go files in the resolve directory to locate the singleflight implementation

ls -la v2/pkg/engine/resolve/ | grep -E ".go$"


</function_calls>


#!/bin/bash

Search for files containing singleflight

find v2/pkg/engine -name "*.go" -type f -exec grep -l "InboundRequestSingleFlight" {} ;


</function_calls>


#!/bin/bash

Find all Go files in the resolve package

find v2/pkg/engine -type f -name "*.go" | sort


</function_calls>


#!/bin/bash

Search more broadly for the singleflight pattern

rg -t go "func.*GetOrCreate" v2/pkg/engine/ -B 5 -A 10


</function_calls>


#!/bin/bash

Look at the resolve.go file to understand the flow

wc -l v2/pkg/engine/resolve/resolve.go


</function_calls>
<function_calls>


#!/bin/bash

Check the specific function in resolve.go

sed -n '318,382p' v2/pkg/engine/resolve/resolve.go


</function_calls>
<function_calls>


#!/bin/bash

Search for inboundRequestSingleFlight field definition in Resolver

rg -t go "inboundRequestSingleFlight" v2/pkg/engine/resolve/ -B 2 -A 2


</function_calls>
<function_calls>


#!/bin/bash

Search for the singleflight package/type used

rg -t go "type.*Resolver struct" v2/pkg/engine/resolve/resolve.go -A 30


</function_calls>
<function_calls>


#!/bin/bash

Look at the imports to understand what singleflight is being used

head -50 v2/pkg/engine/resolve/resolve.go


</function_calls>
<function_calls>


#!/bin/bash

Find the singleflight implementation in the codebase

find . -path ./vendor -prune -o -type f -name "*.go" -exec grep -l "type.*Singleflight" {} ;


</function_calls>
<function_calls>


#!/bin/bash

Search for GetOrCreate and FinishOk methods in the entire codebase

rg -t go "func.*GetOrCreate|func.*FinishOk|func.*FinishErr" --max-count 50


</function_calls>
<function_calls>


#!/bin/bash

Look for where inboundRequestSingleFlight is created/assigned

rg -t go "inboundRequestSingleFlight\s*=" v2/pkg/engine/ -B 3 -A 3


</function_calls>
<function_calls>


#!/bin/bash

Search in common locations for the singleflight type

find v2/pkg -type d -name "singleflight"


</function_calls>
<function_calls>


#!/bin/bash

Search for the entire Resolver struct definition

rg -t go "type Resolver struct" v2/pkg/engine/resolve/resolve.go -A 50 | head -80


</function_calls>
<function_calls>


#!/bin/bash

Let me look more broadly at the engine package

find v2/pkg/engine -type f -name "*.go" | xargs grep -l "GetOrCreate|FinishOk" 2>/dev/null


</function_calls>
<function_calls>


#!/bin/bash

Look at all files in engine directory to find singleflight

ls -la v2/pkg/engine/


</function_calls>
<function_calls>


#!/bin/bash

Check the engine package structure

find v2/pkg/engine -type f -name "*.go" | wc -l
find v2/pkg/engine -type d | head -20


</function_calls>
<function_calls>


#!/bin/bash

Search for any file containing both InboundRequest and Singleflight

rg -t go "InboundRequest.*Singleflight|Singleflight.*InboundRequest" -A 5 | head -50


</function_calls>
<function_calls>


#!/bin/bash

Simpler approach: search for the phrase "FinishOk" across entire codebase

rg -t go "FinishOk\s*(" --max-count 20


</function_calls>
<function_calls>


#!/bin/bash

Let me look at the git structure to understand the repo better

git ls-files v2/pkg/engine/ | head -50


</function_calls>
<function_calls>


#!/bin/bash

Look for Go files with "singleflight" in name or content

git ls-files | xargs grep -l "GetOrCreate" | head -20


</function_calls>
<function_calls>


#!/bin/bash

Search for GetOrCreate definition

rg -t go "func.*\bGetOrCreate\b" -A 10


</function_calls>
<function_calls>


#!/bin/bash

Check the exact lines referenced in resolve.go around the function

sed -n '1,50p' v2/pkg/engine/resolve/resolve.go


</function_calls>
<function_calls>


#!/bin/bash

Search more carefully for inbound singleflight patterns

rg -t go "inboundRequestSingleFlight" --type-list | head -5
rg -t go "inboundRequestSingleFlight" | head -20


</function_calls>
<function_calls>


#!/bin/bash

Let's look at the handler or plan package which might contain singleflight

find v2/pkg -name "plan" -o -name "handler" | head -10


</function_calls>
<function_calls>


#!/bin/bash

Get the full structure of the repo

git ls-files v2/pkg | grep -E "(singleflight|plan|handler)" | head -20


</function_calls>
<function_calls>


#!/bin/bash

Look at all directories in v2

ls -la v2/
ls -la v2/pkg/
ls -la v2/pkg/engine/


</function_calls>
<function_calls>


#!/bin/bash

Let me search for where FinishOk is implemented (the method definition)

rg -t go "func\s+(\s*\w+\s+*\w*\s*)\s+FinishOk"


</function_calls>
<function_calls>


#!/bin/bash

Search for FinishOk with different pattern

rg "FinishOk" --type go -B 5 | head -100


</function_calls>
<function_calls>


#!/bin/bash

Maybe it's in a different location. Let me look at the actual resolver again more carefully

sed -n '300,400p' v2/pkg/engine/resolve/resolve.go | cat -n


</function_calls>
<function_calls>


#!/bin/bash

Search for the custom type or interface used

python3 << 'EOF'
import subprocess
import re

Get all lines from resolve.go

result = subprocess.run(['sed', '-n', '318,382p', 'v2/pkg/engine/resolve/resolve.go'],
capture_output=True, text=True)
lines = result.stdout

Look for the method calls

method_calls = re.findall(r'r.inboundRequestSingleFlight.(\w+)', lines)
print("Method calls found:")
for call in set(method_calls):
print(f" - {call}")

Now search for the type

result2 = subprocess.run(['rg', '-t', 'go', 'inboundRequestSingleFlight.*:'],
capture_output=True, text=True)
print("\nType definition search results:")
print(result2.stdout)
EOF


</function_calls>
<function_calls>


#!/bin/bash

Look at imports and struct definition more carefully

sed -n '1,150p' v2/pkg/engine/resolve/resolve.go | head -100


</function_calls>
<function_calls>


#!/bin/bash

Look for where Resolver is initialized

rg -t go "NewResolver|func New" v2/pkg/engine/resolve/ -A 10 | head -60


</function_calls>
<function_calls>


#!/bin/bash

Let me check the plan package

ls -la v2/pkg/engine/plan/


</function_calls>
<function_calls>


#!/bin/bash

Search across all go files for Singleflight type

rg -t go "type.*Singleflight" | head -30


</function_calls>
<function_calls>


#!/bin/bash

Look in handler package

ls -la v2/pkg/engine/handler/


</function_calls>
<function_calls>


#!/bin/bash

Search for the actual definition of the type being used

rg -t go "FinishOk\s*(" --context 15 | head -80


</function_calls>
<function_calls>


#!/bin/bash

Try a different approach - find all files in common package locations

find v2 -type f -name "*.go" -exec grep -l "Singleflight" {} ; | head -10


</function_calls>
<function_calls>


#!/bin/bash

Look for imports in resolve.go

rg -t go "^import|from" v2/pkg/engine/resolve/resolve.go | head -30


</function_calls>
<function_calls>


#!/bin/bash

Get imports more carefully

sed -n '1,100p' v2/pkg/engine/resolve/resolve.go | grep -A 50 "import"


</function_calls>
<function_calls>


#!/bin/bash

Look at structure and imports

head -100 v2/pkg/engine/resolve/resolve.go


</function_calls>

Now let me search for the actual implementation of InboundRequestSingleFlight:
<function_calls>


#!/bin/bash

Search all go files for method definitions on anything containing Singleflight

find v2 -type f -name "*.go" -exec grep -l "func.*FinishOk|func.*GetOrCreate" {} ;


</function_calls>
<function_calls>


#!/bin/bash

Look at the handler module which likely contains singleflight

cat v2/pkg/engine/handler/handler.go | head -100


</function_calls>
<function_calls>


#!/bin/bash

Find files in handler directory

ls -la v2/pkg/engine/handler/


</function_calls>
<function_calls>


#!/bin/bash

Search for GetOrCreate in handler files

grep -r "GetOrCreate" v2/pkg/engine/handler/ --include="*.go"


</function_calls>
<function_calls>


#!/bin/bash

Try looking in graphql package or handler extensions

find v2 -type f -name "singleflight"


</function_calls>
<function_calls>


#!/bin/bash

Let me search the entire v2 directory for Singleflight

rg "Singleflight" v2/ --type go | head -30


</function_calls>

Let me try a more efficient search:
<function_calls>


#!/bin/bash

Find all unique types that have FinishOk method

rg "FinishOk\s*(" --type go -B 20 | rg "^[^-].func.(.*)" | head -20


</function_calls>
<function_calls>


#!/bin/bash

Let me look at what package has the singleflight

rg "FinishOk" --type go --context 20 | grep "func|type" | head -40


</function_calls>
<function_calls>


#!/bin/bash

Direct search: where is FinishOk defined as a method?

ast-grep --pattern 'func ($) FinishOk($) { $$$ }'


</function_calls>
<function_calls>


#!/bin/bash

Let me just find the file containing FinishOk definition

rg "func.*FinishOk" --type go


</function_calls>
<function_calls>


#!/bin/bash

Search for any definition of FinishOk

rg "FinishOk" --type go --max-count 50


</function_calls>

Let me check if there's a specific file in the root that contains singleflight:
<function_calls>


#!/bin/bash

Search at repo root level

rg "Singleflight|FinishOk|GetOrCreate" --type go --max-depth 2 | head -50


</function_calls>
<function_calls>


#!/bin/bash

Look for any go file in root or immediate subdirs

find . -maxdepth 3 -name "*.go" -type f | head -20


</function_calls>
<function_calls>


#!/bin/bash

Check the structure

ls -la


</function_calls>
<function_calls>


#!/bin/bash

Look at root level Go files

ls -la *.go 2>/dev/null || echo "No root level go files"


</function_calls>
<function_calls>


#!/bin/bash

Let me check if inboundRequestSingleFlight is maybe an external package or interface

rg "inboundRequestSingleFlight" --type go -B 5 -A 5 | head -100


</function_calls>

Let me take a different approach and check what the exact type declaration is:
<function_calls>


#!/bin/bash

Look at the struct containing inboundRequestSingleFlight more carefully

rg "type Resolver struct" --type go -A 100 | head -150


</function_calls>
<function_calls>


#!/bin/bash

Show the exact line from the file

sed -n '50,200p' v2/pkg/engine/resolve/resolve.go | cat -n


</function_calls>

Now let me search for the type name of inboundRequestSingleFlight:
<function_calls>


#!/bin/bash

Check what type inboundRequestSingleFlight is

python3 << 'EOF'
import re

with open('v2/pkg/engine/resolve/resolve.go', 'r') as f:
content = f.read()

Find the Resolver struct definition

match = re.search(r'type\s+Resolver\s+struct\s*{([^}]+)}', content, re.DOTALL)
if match:
struct_body = match.group(1)
# Find inboundRequestSingleFlight field
field_match = re.search(r'inboundRequestSingleFlight\s+(\S+)', struct_body)
if field_match:
print(f"inboundRequestSingleFlight type: {field_match.group(1)}")
else:
print("Field not found in struct")
# Print first part of struct
print("Struct fields:")
print(struct_body[:500])
else:
print("Resolver struct not found")
EOF


</function_calls>
<function_calls>


#!/bin/bash

Now search for that type name with GetOrCreate and FinishOk

First let me look at the field more explicitly

grep -n "inboundRequestSingleFlight" v2/pkg/engine/resolve/resolve.go | head -5


</function_calls>
<function_calls>


#!/bin/bash

Get the context around those lines

sed -n '60,120p' v2/pkg/engine/resolve/resolve.go


</function_calls>

Perfect! Now let me search for the InboundRequestSingleFlight type:
<function_calls>


#!/bin/bash

Search for InboundRequestSingleFlight type definition

rg "type InboundRequestSingleFlight" --type go -A 10


</function_calls>
<function_calls>


#!/bin/bash

Search for the type in plan package

ls -la v2/pkg/engine/plan/
grep -r "InboundRequestSingleFlight" v2/pkg/engine/plan/ --include="*.go"


</function_calls>
<function_calls>


#!/bin/bash

Try handler package

grep -r "InboundRequestSingleFlight" v2/pkg/engine/handler/ --include="*.go" | head -20


</function_calls>
<function_calls>


#!/bin/bash

Let me check all available packages

find v2/pkg/engine -type d | sort


</function_calls>
<function_calls>


#!/bin/bash

Search in datasource package

ls -la v2/pkg/engine/datasource/


</function_calls>
<function_calls>


#!/bin/bash

Look in execution package

ls -la v2/pkg/execution/


</function_calls>
<function_calls>


#!/bin/bash

Now search broadly for InboundRequestSingleFlight

rg "InboundRequestSingleFlight" --type go | head -30


</function_calls>

Perfect! Now I found it. Let me examine the implementation:
<function_calls>


#!/bin/bash

Look at the complete file with InboundRequestSingleFlight

head -100 v2/pkg/execution/execution_engine.go


</function_calls>
<function_calls>


#!/bin/bash

Search for the type definition in execution_engine

rg "type InboundRequestSingleFlight" v2/pkg/execution/execution_engine.go -A 20


</function_calls>
<function_calls>


#!/bin/bash

Look for FinishOk and FinishErr methods

rg "func.*FinishOk|func.*FinishErr|func.*GetOrCreate" v2/pkg/execution/execution_engine.go -A 15 | head -150


</function_calls>
<function_calls>


#!/bin/bash

Get the full file or at least the relevant parts

wc -l v2/pkg/execution/execution_engine.go


</function_calls>
<function_calls>


#!/bin/bash

Look at the complete InboundRequestSingleFlight structure and its methods

rg "type InboundRequestSingleFlight" v2/pkg/execution/execution_engine.go -A 5 | head -30


</function_calls>
<function_calls>


#!/bin/bash

Show GetOrCreate method

rg "GetOrCreate" v2/pkg/execution/execution_engine.go -B 2 -A 20 | head -80


</function_calls>
<function_calls>


#!/bin/bash

Show FinishOk method

rg "FinishOk" v2/pkg/execution/execution_engine.go -B 2 -A 30


</function_calls>
<function_calls>


#!/bin/bash

Show FinishErr method

rg "FinishErr" v2/pkg/execution/execution_engine.go -B 2 -A 15


</function_calls>

Perfect! Now let me examine these methods in detail:
<function_calls>


#!/bin/bash

Let's see lines around these methods to understand the full implementation

python3 << 'EOF'
import subprocess

result = subprocess.run(['rg', 'GetOrCreate|FinishOk|FinishErr', 'v2/pkg/execution/execution_engine.go', '-n'],
capture_output=True, text=True)
lines = result.stdout.strip().split('\n')
for line in lines[:20]:
print(line)
EOF


</function_calls>
<function_calls>


#!/bin/bash

Get the file and look at specific line ranges

sed -n '1,100p' v2/pkg/execution/execution_engine.go | cat -n


</function_calls>
<function_calls>


#!/bin/bash

Find line numbers of the methods

grep -n "func.*GetOrCreate|func.*FinishOk|func.*FinishErr" v2/pkg/execution/execution_engine.go


</function_calls>
<function_calls>


#!/bin/bash

Show the InboundRequestSingleFlight type and methods

sed -n '20,150p' v2/pkg/execution/execution_engine.go | cat -n


</function_calls>
<function_calls>


#!/bin/bash

Look further in the file for the methods

sed -n '150,300p' v2/pkg/execution/execution_engine.go | cat -n


</function_calls>
<function_calls>


#!/bin/bash

Look at more of the file

sed -n '1,400p' v2/pkg/execution/execution_engine.go | tail -100 | cat -n


</function_calls>

Let me use Python to extract the exact methods:
<function_calls>


#!/bin/bash
python3 << 'EOF'
import re

with open('v2/pkg/execution/execution_engine.go', 'r') as f:
lines = f.readlines()

Find methods containing GetOrCreate, FinishOk, FinishErr

for i, line in enumerate(lines):
if any(x in line for x in ['GetOrCreate', 'FinishOk', 'FinishErr', 'type InboundRequestSingleFlight']):
# Print surrounding context
start = max(0, i - 2)
end = min(len(lines), i + 20)
print(f"\n=== Lines {start+1}-{end} ===")
for j in range(start, end):
print(f"{j+1:4d}: {lines[j]}", end='')
print()
EOF


</function_calls>

Great! Now I have the implementation details. Let me verify my analysis of the concerns:

Now let me verify the exact implementation details one more time to be absolutely sure:
<function_calls>


#!/bin/bash

Get the exact text of FinishOk to verify data copying

sed -n '1,400p' v2/pkg/execution/execution_engine.go | grep -A 20 "func.*FinishOk"


</function_calls>
<function_calls>


#!/bin/bash

Get the exact text of FinishErr to verify nil handling

sed -n '1,400p' v2/pkg/execution/execution_engine.go | grep -A 10 "func.*FinishErr"


</function_calls>
<function_calls>


#!/bin/bash

Get GetOrCreate to verify what it returns

sed -n '1,400p' v2/pkg/execution/execution_engine.go | grep -A 15 "func.*GetOrCreate"


</function_calls>

Now let me double-check by reading the actual implementation more precisely:
<function_calls>


#!/bin/bash

Look at InboundRequestSingleFlight structure itself

sed -n '1,60p' v2/pkg/execution/execution_engine.go


</function_calls>
<function_calls>


#!/bin/bash

Count lines and search systematically

wc -l v2/pkg/execution/execution_engine.go

Now look at middle section

sed -n '200,350p' v2/pkg/execution/execution_engine.go | cat -n


</function_calls>
<function_calls>


#!/bin/bash

Let me check if these methods are defined elsewhere or inherited

rg "FinishOk\s*(" v2/pkg/execution/ --type go -B 5 -A 15 | head -100


</function_calls>

Let me search more broadly to understand the complete picture:
<function_calls>


#!/bin/bash

Search for the complete implementation across the execution package

grep -r "func.*InboundRequestSingleFlight.*FinishOk|func.*InboundRequestSingleFlight.*FinishErr|func.*InboundRequestSingleFlight.GetOrCreate" v2/pkg/execution/ --include=".go"


</function_calls>
<function_calls>


#!/bin/bash

Let me look at the complete singleflight implementation - maybe it's in a separate file

find v2/pkg/execution -name "*.go" | xargs grep -l "FinishOk"


</function_calls>
<function_calls>


#!/bin/bash

Look at that inbound_request file

ls -la v2/pkg/execution/


</function_calls>
<function_calls>


#!/bin/bash

Check if there's an inbound_request file

find v2/pkg/execution -name "inbound" -type f


</function_calls>
<function_calls>


#!/bin/bash

List all files in execution directory

ls v2/pkg/execution/


</function_calls>
<function_calls>


#!/bin/bash

Read the complete inbound_request.go file

cat v2/pkg/execution/inbound_request.go


</function_calls>

Perfect! Now I have the complete implementation. Let me analyze it:

Based on my verification, I found that both concerns raised in the review comment are already addressed in the implementation:

  1. Nil inflight safety: The FinishOk and FinishErr methods already have explicit nil checks (if inflight == nil { return }), making the unconditional calls in ArenaResolveGraphQLResponse safe.

  2. Arena data lifetime: The FinishOk method copies the data (data := append(make([]byte, 0, len(data)), data...)) to heap-allocated memory, so releasing the arena after calling FinishOk is safe.

The code as written is correct and no changes are needed.

Comment on lines +77 to +89
// resolveArenaPool is the arena pool dedicated for Loader & Resolvable.
// ArenaPool automatically adjusts arena buffer sizes per workload.
// Resolving & response buffering are very different tasks;
// as such, it was best to have two arena pools in terms of memory usage.
// A single pool for both was much less efficient.
resolveArenaPool *arena.Pool
// responseBufferPool is the arena pool dedicated for response buffering before sending to the client
responseBufferPool *arena.Pool

// subgraphRequestSingleFlight is used to de-duplicate subgraph requests
subgraphRequestSingleFlight *SubgraphRequestSingleFlight
// inboundRequestSingleFlight is used to de-duplicate subgraph requests
inboundRequestSingleFlight *InboundRequestSingleFlight
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue | 🟡 Minor

Clarify inbound singleflight comment and consider shard configurability

The split between resolveArenaPool and responseBufferPool plus the two singleflight structs looks good and matches the intended separation of concerns. Two small nits:

  • The comment for inboundRequestSingleFlight still says “de-duplicate subgraph requests”, but this type is meant for inbound client request deduplication (per ExecutionOptions.DisableInboundRequestDeduplication and the type name). I’d reword to avoid confusion for future readers.
  • The hard‑coded shard count 8 is fine as a default, but if you anticipate tuning under load, consider exposing it via ResolverOptions (or at least a named constant) rather than a magic number.

Also applies to: 243-247

🤖 Prompt for AI Agents
In v2/pkg/engine/resolve/resolve.go around lines 77-89 (and similarly lines
243-247), update the comment for inboundRequestSingleFlight to accurately state
it de-duplicates inbound client requests (not subgraph requests), and replace
the hard-coded shard count (currently 8) with a configurable option: either add
a field in ResolverOptions to set inbound deduplication shard count or define a
named constant (e.g., inboundRequestShardCount) and use that constant; adjust
constructor/initialization to read the option or constant accordingly.

Comment on lines +1182 to +1198
// prepareTrigger safely gets the headers for the trigger Subgraph and computes the hash across headers and input
// the generated has is the unique triggerID
// the headers must be forwarded to the DataSource to create the trigger
func (r *Resolver) prepareTrigger(ctx *Context, sourceName string, input []byte) (headers http.Header, triggerID uint64) {
keyGen := pool.Hash64.Get()
_, _ = keyGen.Write(input)
if ctx.SubgraphHeadersBuilder != nil {
var headersHash uint64
headers, headersHash = ctx.SubgraphHeadersBuilder.HeadersForSubgraph(sourceName)
var b [8]byte
binary.LittleEndian.PutUint64(b[:], headersHash)
_, _ = keyGen.Write(b[:])
}
triggerID = keyGen.Sum64()
pool.Hash64.Put(keyGen)
return headers, triggerID
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue | 🟡 Minor

prepareTrigger now avoids triggerID collisions; fix minor comment typo

The new prepareTrigger implementation correctly:

  • Always hashes the input bytes, so subscriptions without a SubgraphHeadersBuilder no longer all share triggerID == 0.
  • Optionally mixes in the per‑subgraph headersHash when a builder is present, which aligns with the SubgraphHeadersBuilder contract.

That resolves the earlier risk of unrelated subscriptions being grouped under the same trigger ID when no headers builder was configured. One tiny polish:

  • Comment line 1183 has a typo: “the generated has is the unique triggerID” → “the generated hash is the unique triggerID”.
🤖 Prompt for AI Agents
v2/pkg/engine/resolve/resolve.go around lines 1182 to 1198: fix the typo in the
function comment on line 1183 where "the generated has is the unique triggerID"
should read "the generated hash is the unique triggerID"; update that word only
so the comment correctly describes that the generated hash is the unique
triggerID.

Copy link
Contributor

@coderabbitai coderabbitai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actionable comments posted: 2

♻️ Duplicate comments (4)
v2/pkg/engine/resolve/resolve.go (2)

76-89: Approve arena pool and singleflight additions, but note existing comment inaccuracy.

The dual-pool design (resolve vs response buffer) is a sound optimization. The singleflight mechanisms for both inbound and subgraph deduplication are properly initialized.

Note: The comment on line 88 still says "de-duplicate subgraph requests" but this is inboundRequestSingleFlight (for client request deduplication). This was flagged in a prior review cycle.


1187-1203: Typo in comment: "has" should be "hash".

Line 1188: "the generated has is the unique triggerID" → "the generated hash is the unique triggerID".

The implementation correctly computes a unique triggerID by hashing input bytes and optionally mixing in the per-subgraph headers hash.

 // prepareTrigger safely gets the headers for the trigger Subgraph and computes the hash across headers and input
-// the generated has is the unique triggerID
+// the generated hash is the unique triggerID
 // the headers must be forwarded to the DataSource to create the trigger
v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go (1)

291-312: Remove remaining debug fmt.Println and simplify unmarshal

The Load call and "body" envelope look correct here, but the extra alias and print add noise:

bytes := output
fmt.Println(string(bytes))

err = json.Unmarshal(bytes, &resp)

You can simplify and avoid spurious test output:

-	bytes := output
-	fmt.Println(string(bytes))
-
-	err = json.Unmarshal(bytes, &resp)
+	err = json.Unmarshal(output, &resp)

This matches the rest of the file, where tests assert on decoded data instead of printing raw responses.

v2/pkg/engine/resolve/resolve_test.go (1)

429-435: Prefer DoAndReturn over Do(...).Return(...) for gomock expectations

This mock still chains Do(...).Return(...) even though the Do callback has return values that gomock ignores; you also specify the same return values again in Return. Using DoAndReturn keeps side effects and return values in one place and avoids confusion.

-	mockDataSource.EXPECT().
-		Load(gomock.Any(), gomock.Any(), []byte(`{"id":1}`)).
-		Do(func(ctx context.Context, headers http.Header, input []byte) ([]byte, error) {
-			return []byte(`{"name":"Jens"}`), nil
-		}).
-		Return([]byte(`{"name":"Jens"}`), nil)
+	mockDataSource.EXPECT().
+		Load(gomock.Any(), gomock.Any(), []byte(`{"id":1}`)).
+		DoAndReturn(func(ctx context.Context, headers http.Header, input []byte) ([]byte, error) {
+			return []byte(`{"name":"Jens"}`), nil
+		})
In gomock, when using `Do` plus `Return` vs `DoAndReturn`, are the values returned from the `Do` callback actually ignored, with `Return` being the source of the method's return values?
🧹 Nitpick comments (5)
v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go (1)

448-455: Consider a small helper to build the {"query":...,"body":...} envelope

Virtually every test now repeats:

input := fmt.Sprintf(`{"query":%q,"body":%s}`, query, vars)
output, err := ds.Load(context.Background(), nil, []byte(input))

The one remaining divergence at line 222 suggests this is easy to get subtly wrong.

Consider a tiny helper like:

func makeInput(query, vars string) []byte {
	return []byte(fmt.Sprintf(`{"query":%q,"body":%s}`, query, vars))
}

(or a variant that handles {} / null) to centralize the envelope shape and prevent future inconsistencies.

Also applies to: 3723-3725, 4550-4551

v2/pkg/engine/resolve/resolve_test.go (4)

4068-4523: Arena helper + mirror tests give good coverage of the arena path

testFnArena plus TestResolver_ArenaResolveGraphQLResponse mirror a representative subset of the existing ResolveGraphQLResponse tests, which is a solid way to ensure the arena-based path stays behaviorally identical (including BigInt, arrays, nested objects, error handling, skip-loader, variables). If duplication ever becomes painful, you could consider factoring out shared GraphQLResponse builders and running them through both helpers, but the current form is clear and fine.


4525-4641: Request deduplication test is strong; consider asserting single datasource load

The new TestResolver_ArenaResolveGraphQLResponse_RequestDeduplication nicely validates leader/follower behavior (ResolveDeduplicated flag and identical outputs) under concurrent requests sharing Request.ID and VariablesHash, using blockingDataSource/blockingWriter to avoid flakiness. To tighten the guarantee that dedup actually coalesces loads, you could track a call counter in blockingDataSource and assert it equals 1 here.

For example:

 type blockingDataSource struct {
-	data        []byte
+	data        []byte
+	calls       atomic.Int32
@@
 func (f *blockingDataSource) Load(ctx context.Context, headers http.Header, input []byte) (data []byte, err error) {
+	f.calls.Add(1)
 	f.waitForRelease()
 	return f.data, nil
}
@@
 // in the test, after wg.Wait():
-	for _, res := range results {
+	for _, res := range results {
 		require.NoError(t, res.err)
 		require.NotNil(t, res.info)
 	}
+
+	require.Equal(t, int32(1), ds.calls.Load(), "expected exactly one underlying datasource load")

5173-5181: Remove the unused fakeStreamRequestId variable

The fakeStreamRequestId atomic.Int32 variable is no longer referenced anywhere and can be removed:

-var fakeStreamRequestId atomic.Int32
-
 type _fakeStream struct {

4911-4980: Header-variable test is good; you can also assert propagated headers if desired

TestResolver_WithHeader correctly exercises case-insensitive header variable resolution by asserting the datasource sees the header value as the request body. If you also want to cover the new header-plumbing path, you could assert that the headers argument contains the original header:

fakeService.EXPECT().
 	Load(gomock.Any(), gomock.Any(), gomock.Any()).
-	DoAndReturn(func(ctx context.Context, headers http.Header, input []byte) ([]byte, error) {
-		actual := string(input)
-		assert.Equal(t, "foo", actual)
+	DoAndReturn(func(ctx context.Context, headers http.Header, input []byte) ([]byte, error) {
+		actual := string(input)
+		assert.Equal(t, "foo", actual)
+		assert.Equal(t, "foo", headers.Get("Authorization"))
 		return []byte(`{"bar":"baz"}`), nil
 	})

This keeps the variable behavior while also validating that request headers are forwarded. Since http.Header canonicalizes all keys, headers.Get("Authorization") will reliably retrieve the value regardless of how tc.header was cased when set.

📜 Review details

Configuration used: Path: .coderabbit.yaml

Review profile: CHILL

Plan: Pro

📥 Commits

Reviewing files that changed from the base of the PR and between c8b5567 and a6c9da8.

📒 Files selected for processing (5)
  • v2/pkg/astvisitor/visitor.go (5 hunks)
  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go (6 hunks)
  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go (29 hunks)
  • v2/pkg/engine/resolve/resolve.go (21 hunks)
  • v2/pkg/engine/resolve/resolve_test.go (31 hunks)
🧰 Additional context used
🧠 Learnings (13)
📚 Learning: 2025-08-29T09:35:47.969Z
Learnt from: ysmolski
Repo: wundergraph/graphql-go-tools PR: 1282
File: v2/pkg/engine/plan/visitor.go:5-5
Timestamp: 2025-08-29T09:35:47.969Z
Learning: The wundergraph/graphql-go-tools project does not support Go versions < 1.23, so compatibility concerns for features available in Go 1.21+ (like cmp.Or) should not be raised.

Applied to files:

  • v2/pkg/astvisitor/visitor.go
  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go
📚 Learning: 2025-11-19T09:38:25.112Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1341
File: v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go:1418-1526
Timestamp: 2025-11-19T09:38:25.112Z
Learning: In v2/pkg/engine/datasource/grpc_datasource test files, ResolvePath fields use snake_case (e.g., "test_containers.id") because they reference proto message field names, while JSONPath fields use camelCase (e.g., "testContainers") because they reference GraphQL/JSON response field names. Both casing styles are intentional and correct.

Applied to files:

  • v2/pkg/engine/resolve/resolve_test.go
  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go
  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go
📚 Learning: 2025-11-19T10:53:06.342Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1341
File: v2/pkg/engine/datasource/grpc_datasource/execution_plan.go:1039-1097
Timestamp: 2025-11-19T10:53:06.342Z
Learning: In v2/pkg/engine/datasource/grpc_datasource field resolver response handling, the `resolveRequiredFields` function intentionally uses two distinct approaches: for simple GraphQL object types it populates `message.Fields`, while for composite types (interface/union) it exclusively uses `message.FieldSelectionSet` with fragment-based selections. This differs from `buildFieldMessage` (regular queries) because field resolver responses returning composite types must align with protobuf oneOf structure, where all selections—including common interface fields—are handled through fragment selections built by `buildCompositeField`. The two approaches cannot be mixed in field resolver responses.

Applied to files:

  • v2/pkg/engine/resolve/resolve_test.go
  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go
  • v2/pkg/engine/resolve/resolve.go
📚 Learning: 2025-11-19T09:42:17.644Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1341
File: v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go:406-429
Timestamp: 2025-11-19T09:42:17.644Z
Learning: In the wundergraph/graphql-go-tools gRPC datasource implementation (v2/pkg/engine/datasource/grpc_datasource), field resolvers must have arguments. The system does not currently support defining field resolvers without arguments. This invariant ensures that the `parentCallID` increment in `enterFieldResolver` is always matched by a decrement in `LeaveField` (which checks `r.operation.FieldHasArguments(ref)`).

Applied to files:

  • v2/pkg/engine/resolve/resolve_test.go
  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go
  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go
  • v2/pkg/engine/resolve/resolve.go
📚 Learning: 2025-12-02T08:25:26.682Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1341
File: v2/pkg/engine/datasource/grpc_datasource/execution_plan.go:1053-1118
Timestamp: 2025-12-02T08:25:26.682Z
Learning: In v2/pkg/engine/datasource/grpc_datasource field resolver response handling for composite types (interface/union), buildFieldResolverTypeMessage correctly combines both message.Fields and message.FieldSelectionSet: message.Fields contains interface-level fields that can be selected directly on the interface (such as __typename or fields defined in the interface itself), while message.FieldSelectionSet contains type-specific fields from inline fragments. This mixing is intentional and correct for GraphQL interfaces, as interface-level fields exist outside inline fragment selections and must be handled separately from type-specific fragment selections.

Applied to files:

  • v2/pkg/engine/resolve/resolve_test.go
  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go
  • v2/pkg/engine/resolve/resolve.go
📚 Learning: 2025-09-19T14:50:19.528Z
Learnt from: endigma
Repo: wundergraph/graphql-go-tools PR: 1298
File: v2/pkg/engine/resolve/resolve.go:999-1011
Timestamp: 2025-09-19T14:50:19.528Z
Learning: In the graphql-go-tools resolver, when handling backpressure for async unsubscribe operations, the team prefers using individual goroutines over bounded deferrer implementations. The rationale is that bounded implementations can also overflow, and the preferred solution for overload is scaling the deployment rather than internal buffering. The current approach prevents deadlocks at the cost of goroutine creation, which is an acceptable tradeoff.

Applied to files:

  • v2/pkg/engine/resolve/resolve_test.go
  • v2/pkg/engine/resolve/resolve.go
📚 Learning: 2025-07-02T15:28:02.122Z
Learnt from: SkArchon
Repo: wundergraph/graphql-go-tools PR: 1203
File: v2/pkg/engine/resolve/loader.go:63-67
Timestamp: 2025-07-02T15:28:02.122Z
Learning: In the graphql-go-tools codebase, result structs are consistently initialized with non-nil bytes.Buffer instances, making additional nil checks for res.out unnecessary defensive programming.

Applied to files:

  • v2/pkg/engine/resolve/resolve_test.go
  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go
  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go
  • v2/pkg/engine/resolve/resolve.go
📚 Learning: 2025-10-16T13:05:19.838Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1323
File: v2/pkg/engine/datasource/grpc_datasource/compiler.go:683-702
Timestamp: 2025-10-16T13:05:19.838Z
Learning: In GraphQL field resolver context resolution (v2/pkg/engine/datasource/grpc_datasource/compiler.go), when traversing paths in resolveContextDataForPath, the code can safely assume that intermediate path segments will only be messages or lists, never scalars. This is because field resolvers are only defined on GraphQL object types, not scalar types, so the parent function must return either a message or a list. This invariant is enforced by the GraphQL type system design.

Applied to files:

  • v2/pkg/engine/resolve/resolve_test.go
  • v2/pkg/engine/resolve/resolve.go
📚 Learning: 2025-10-16T08:52:33.278Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1323
File: v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go:59-63
Timestamp: 2025-10-16T08:52:33.278Z
Learning: Go 1.24 (released February 2025) introduced the testing.B.Loop() method for benchmarks. Use `for b.Loop() { /* code */ }` instead of `for i := 0; i < b.N; i++` in benchmark functions when the Go version is 1.24 or higher. The Loop() method provides more predictable benchmarking results by running the loop exactly once per -count and preventing certain compiler optimizations.

Applied to files:

  • v2/pkg/engine/resolve/resolve_test.go
📚 Learning: 2025-08-08T09:43:07.433Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1262
File: v2/pkg/engine/datasource/grpc_datasource/json_builder.go:0-0
Timestamp: 2025-08-08T09:43:07.433Z
Learning: In v2/pkg/engine/datasource/grpc_datasource/json_builder.go, mergeEntities intentionally uses the loop index when calling indexMap.getResultIndex because the index map is type-aware, making per-type counters unnecessary under the current assumptions. Avoid suggesting per-type ordinal counters for this path in future reviews.

Applied to files:

  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go
  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go
📚 Learning: 2025-07-28T12:44:56.405Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1246
File: v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor.go:457-457
Timestamp: 2025-07-28T12:44:56.405Z
Learning: In the graphql-go-tools gRPC datasource, only non-null lists use protobuf's repeated field syntax directly. For nullable or nested lists, wrapper types are used because protobuf repeated fields cannot be nullable. The TypeIsNonNullList check ensures only appropriate list types are marked as repeated in the protobuf message structure.

Applied to files:

  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go
📚 Learning: 2025-10-15T13:34:15.892Z
Learnt from: ysmolski
Repo: wundergraph/graphql-go-tools PR: 1322
File: v2/pkg/astvalidation/operation_rule_defer_stream_on_root_fields.go:92-127
Timestamp: 2025-10-15T13:34:15.892Z
Learning: In the graphql-go-tools repository, validation for defer and stream directives runs after normalization, which performs fragment inlining. Therefore, fragment spreads don't exist in the AST when these validation rules execute—they're already expanded into inline fragments or fields.

Applied to files:

  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go
📚 Learning: 2025-09-19T14:51:33.724Z
Learnt from: endigma
Repo: wundergraph/graphql-go-tools PR: 1298
File: v2/pkg/engine/resolve/resolve.go:339-347
Timestamp: 2025-09-19T14:51:33.724Z
Learning: In the graphql-go-tools resolver, the event loop follows a strict single-threaded design where all subscription lifecycle management (add/remove from triggers) must happen through the events channel to the main processEvents() goroutine. Worker goroutines should not directly modify subscription state or call unsubscribe methods, as this would break the single-threaded event loop architecture and create race conditions.

Applied to files:

  • v2/pkg/engine/resolve/resolve.go
🧬 Code graph analysis (3)
v2/pkg/engine/resolve/resolve_test.go (10)
v2/pkg/engine/resolve/context.go (3)
  • Context (16-38)
  • ExecutionOptions (60-78)
  • Request (185-188)
v2/pkg/engine/datasource/httpclient/nethttpclient.go (1)
  • Do (281-284)
v2/pkg/graphqljsonschema/jsonschema.go (1)
  • Any (296-296)
v2/pkg/engine/resolve/response.go (2)
  • GraphQLResponse (36-44)
  • GraphQLResponseInfo (59-61)
v2/pkg/ast/ast_operation_definition.go (2)
  • OperationType (11-11)
  • OperationTypeQuery (15-15)
v2/pkg/engine/resolve/fetchtree.go (2)
  • Single (59-67)
  • SingleWithPath (69-82)
v2/pkg/engine/resolve/datasource.go (1)
  • DataSource (10-13)
v2/pkg/engine/resolve/inputtemplate.go (2)
  • InputTemplate (31-37)
  • TemplateSegment (22-29)
v2/pkg/engine/resolve/variables_renderer.go (1)
  • NewPlainVariableRenderer (57-61)
v2/pkg/engine/resolve/resolve.go (1)
  • New (191-256)
v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go (1)
v2/pkg/engine/resolve/datasource.go (1)
  • DataSource (10-13)
v2/pkg/engine/resolve/resolve.go (3)
v2/pkg/engine/resolve/subgraph_request_singleflight.go (2)
  • SubgraphRequestSingleFlight (12-15)
  • NewSingleFlight (52-65)
v2/pkg/engine/resolve/inbound_request_singleflight.go (2)
  • InboundRequestSingleFlight (14-16)
  • NewRequestSingleFlight (26-34)
v2/pkg/engine/resolve/context.go (2)
  • Request (185-188)
  • SubgraphHeadersBuilder (44-50)
🪛 GitHub Check: Linters (1.25)
v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go

[failure] 182-182:
File is not properly formatted (gci)

⏰ Context from checks skipped due to timeout of 90000ms. You can increase the timeout in your CodeRabbit configuration to a maximum of 15 minutes (900000ms). (4)
  • GitHub Check: Build and test (go 1.25 / ubuntu-latest)
  • GitHub Check: Build and test (go 1.25 / windows-latest)
  • GitHub Check: Build and test (go 1.25 / windows-latest)
  • GitHub Check: Build and test (go 1.25 / ubuntu-latest)
🔇 Additional comments (36)
v2/pkg/astvisitor/visitor.go (5)

8-9: LGTM!

Import added for the arena package aligns with the PR's memory management improvements.


100-101: LGTM!

The arena field added to Walker enables arena-backed allocations during AST traversal. The interface type allows for nil when arena-based allocation is not needed.


147-149: LGTM!

Proper arena reset before returning Walker to pool, preventing stale references and memory leaks.


1401-1405: LGTM!

Lazy initialization of arena on first Walk with reset on subsequent walks is appropriate for pooled Walker reuse.


1856-1858: LGTM!

Using arena.SliceAppend for selection refs reduces per-iteration heap allocations in the hot path of selection set walking.

v2/pkg/engine/resolve/resolve.go (6)

243-246: LGTM!

Arena pools and singleflight primitives properly initialized. The hardcoded shard count of 8 is a reasonable default for concurrent access distribution.


258-278: LGTM!

Extended newTools signature correctly wires singleflight and arena into both Resolvable and Loader, enabling deduplication and arena-backed allocations throughout the resolution pipeline.


322-387: LGTM! Well-structured arena lifecycle management.

The new ArenaResolveGraphQLResponse properly:

  • Handles inbound request deduplication with early return for followers
  • Acquires/releases arenas at appropriate points
  • Separates resolve arena from response buffer arena
  • Releases resolve arena before writing to client (minimizing arena hold time)
  • Handles all error paths with proper arena release

528-567: LGTM!

Arena lifecycle in executeSubscriptionUpdate correctly acquires before use and releases on all exit paths (error and success).


820-822: LGTM!

Headers are now correctly propagated to both AsyncStart and Start methods, enabling per-subgraph header forwarding to data sources.


1520-1528: LGTM!

The addSubscription struct now carries sourceName and headers, enabling proper trigger identification and header forwarding throughout the subscription lifecycle.

v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go (4)

50-51: LGTM!

Arena pool added to DataSource for per-request memory management, aligning with the PR's arena-centric patterns.


97-110: LGTM!

Clean pool lifecycle management:

  • Acquires pool items with unique keys based on input hash + index
  • Deferred ReleaseMany ensures cleanup on all exit paths
  • Per-call pool items for concurrent goroutines prevent contention

189-198: LGTM!

The acquirePoolItem helper generates deterministic keys by hashing input bytes plus index, ensuring unique pool items for concurrent calls within the same request.


207-208: Verify if LoadWithFiles should be implemented or panic is acceptable.

The interface now expects (data []byte, err error) return, but this method panics. If file uploads are never expected for gRPC, consider returning an explicit error instead of panicking to avoid runtime crashes.

-func (d *DataSource) LoadWithFiles(ctx context.Context, headers http.Header, input []byte, files []*httpclient.FileUpload) (data []byte, err error) {
-	panic("unimplemented")
+func (d *DataSource) LoadWithFiles(ctx context.Context, headers http.Header, input []byte, files []*httpclient.FileUpload) (data []byte, err error) {
+	return nil, fmt.Errorf("gRPC datasource does not support file uploads")
}
v2/pkg/engine/datasource/grpc_datasource/grpc_datasource_test.go (16)

57-59: Benchmark Load usage and envelope shape look correct

The benchmark now calls ds.Load(ctx, nil, []byte(...)) and wraps the GraphQL payload as {"query": "...","body":<vars-json>}, consistent with the rest of the file and the new API.


95-97: Consistent benchmark adaptation to new Load API

The second benchmark follows the same Load(ctx, nil, input) pattern and correct "body":<vars-json> envelope; this keeps both benchmarks aligned with the production input shape.


383-401: Adaptation to Load return value in response‑mapping test is sound

Using output, err := ds.Load(ctx, nil, []byte(inputJSON)) and unmarshalling directly from output matches the new API and keeps the test logic intact.


475-493: gRPC error test still exercises GraphQL error path correctly

The test now treats the returned output []byte as the sole source of GraphQL errors while expecting err == nil, which is exactly what this scenario wants. The "body":<vars-json> envelope is also consistent with other tests.


564-567: JSON builder setup matches new API

Constructing the JSON builder via

jsonBuilder := newJSONBuilder(nil, nil, gjson.Result{})
responseJSON, err := jsonBuilder.marshalResponseJSON(&response, responseMessage)

is consistent with the updated signature and keeps this unit test focused purely on protobuf → JSON mapping.


801-813: Interface tests correctly switched to Load’s []byte result

The animal‑interface tests now consistently pass the {"query":...,"body":vars} envelope and unmarshal from the returned output []byte; this preserves their original intent with the new API shape.


1071-1083: Union tests use the new Load contract consistently

The union‑type suite’s Load invocation and json.Unmarshal(output, &resp) pattern are correct and aligned with the rest of the file.


1207-1219: Category query tests: Load/unmarshal wiring looks good

The category tests’ switch to output, err := ds.Load(ctx, nil, input) and unmarshalling from output matches the new datasource API without altering assertions.


1287-1311: Order‑totals test correctly uses returned JSON bytes

Using the returned output []byte from Load as the sole JSON source in Test_DataSource_Load_WithTotalCalculation is correct and keeps the test focused on payload semantics.


1377-1395: Typename test remains valid after Load refactor

Test_DataSource_Load_WithTypename now feeds the standard {"query":...,"body":{}} envelope into Load and unmarshals from output; the checks on __typename and user fields remain unchanged.


1846-1858: Alias tests correctly migrated to new Load signature

In the alias suite, using output, err := ds.Load(ctx, nil, input) followed by json.Unmarshal(output, &resp) is consistent and keeps the alias behaviour checks intact.


2224-2236: Nullable‑fields tests: envelope and JSON handling are consistent

The nullable‑field tests now follow the same {"query":...,"body":vars} convention and unmarshal from the output []byte, which is exactly what the new API expects.


2525-2537: Nested‑list tests use Load and json.Unmarshal correctly

Test_DataSource_Load_WithNestedLists now consistently uses ds.Load(ctx, nil, input) and unmarshals from output, matching the envelope pattern used elsewhere in the file.


3724-3732: Federated _entities tests: new Load usage is correct

Test_DataSource_Load_WithEntity_Calls now drives the datasource via Load(ctx, nil, input) and unmarshals output into graphqlResponse, which keeps the federation‑error and success scenarios intact under the new API.


4024-4032: Composite _entities tests correctly consume Load’s []byte

The composite‑types suite uses the same envelope and unmarshalling pattern, so it continues to exercise interface/union resolution without relying on the old writer‑based API.


4550-4558: Field‑resolver tests: Load refactor is wired correctly

All field‑resolver cases now use output, err := ds.Load(ctx, nil, input) and json.Unmarshal(output, &resp), so they remain valid while exercising the updated datasource behaviour.

v2/pkg/engine/resolve/resolve_test.go (5)

72-151: Blocking test helpers for dedup look sound and race-free

blockingDataSource and blockingWriter use sync.Once and closed channels to coordinate leader/follower behavior without races or double-closes, and they only block in test-controlled places. This is a good fit for exercising the dedup and arena path without polluting production code.


5204-5231: Updated subscription stream Start signature aligns with new interface

The _fakeStream.Start method now matches the subscription datasource interface by accepting headers http.Header and still uses the context’s ctx.ctx for cancellation. Since tests don’t need headers here, ignoring the parameter is fine and keeps stream behavior unchanged.


6690-6799: Invalid subscription filter template test matches expected behavior

Test_ResolveGraphQLSubscriptionWithFilter/should err when subscription filter has multiple templates correctly exercises the new validation by using a single SubscriptionFieldFilter.Values template that expands to multiple values (via two context vars). Treating this as an invalid template and emitting a GraphQL error per upstream event is a sensible contract, and the test asserts both message count and payloads precisely.


7117-7430: Arena benchmark mirrors nested batching plan and is well-structured

Benchmark_NestedBatchingArena reuses the full nested-batching query plan with fakeDataSourceWithInputCheck and runs it through ArenaResolveGraphQLResponse under RunParallel, checking the exact JSON output. This gives a realistic perf signal for the arena path and shares structure with the non-arena benchmark above, which should make regressions easy to spot.


7432-7740: “No-check” nested batching benchmark is a useful lower-bound baseline

Benchmark_NoCheckNestedBatching swaps in plain FakeDataSource instances (no input validation) but otherwise keeps the batching plan identical, providing a lower-bound throughput baseline versus the stricter benchmarks. This separation between correctness-checking and raw throughput measurement is a good pattern for perf work.

Copy link
Contributor

@coderabbitai coderabbitai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actionable comments posted: 1

🧹 Nitpick comments (1)
v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go (1)

186-186: Consider explicitly returning nil for clarity.

The named return parameter err is only assigned within the TopologicalSortResolve callback. While err will be nil when reaching this line (since any error causes an early return at line 182), explicitly returning nil would make the success path clearer.

-	return value.MarshalTo(nil), err
+	return value.MarshalTo(nil), nil
📜 Review details

Configuration used: Path: .coderabbit.yaml

Review profile: CHILL

Plan: Pro

📥 Commits

Reviewing files that changed from the base of the PR and between a6c9da8 and 85774fa.

📒 Files selected for processing (1)
  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go (6 hunks)
🧰 Additional context used
🧠 Learnings (7)
📓 Common learnings
Learnt from: endigma
Repo: wundergraph/graphql-go-tools PR: 1298
File: v2/pkg/engine/resolve/resolve.go:999-1011
Timestamp: 2025-09-19T14:50:19.528Z
Learning: In the graphql-go-tools resolver, when handling backpressure for async unsubscribe operations, the team prefers using individual goroutines over bounded deferrer implementations. The rationale is that bounded implementations can also overflow, and the preferred solution for overload is scaling the deployment rather than internal buffering. The current approach prevents deadlocks at the cost of goroutine creation, which is an acceptable tradeoff.
📚 Learning: 2025-08-08T09:43:07.433Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1262
File: v2/pkg/engine/datasource/grpc_datasource/json_builder.go:0-0
Timestamp: 2025-08-08T09:43:07.433Z
Learning: In v2/pkg/engine/datasource/grpc_datasource/json_builder.go, mergeEntities intentionally uses the loop index when calling indexMap.getResultIndex because the index map is type-aware, making per-type counters unnecessary under the current assumptions. Avoid suggesting per-type ordinal counters for this path in future reviews.

Applied to files:

  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go
📚 Learning: 2025-11-19T10:53:06.342Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1341
File: v2/pkg/engine/datasource/grpc_datasource/execution_plan.go:1039-1097
Timestamp: 2025-11-19T10:53:06.342Z
Learning: In v2/pkg/engine/datasource/grpc_datasource field resolver response handling, the `resolveRequiredFields` function intentionally uses two distinct approaches: for simple GraphQL object types it populates `message.Fields`, while for composite types (interface/union) it exclusively uses `message.FieldSelectionSet` with fragment-based selections. This differs from `buildFieldMessage` (regular queries) because field resolver responses returning composite types must align with protobuf oneOf structure, where all selections—including common interface fields—are handled through fragment selections built by `buildCompositeField`. The two approaches cannot be mixed in field resolver responses.

Applied to files:

  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go
📚 Learning: 2025-12-02T08:25:26.682Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1341
File: v2/pkg/engine/datasource/grpc_datasource/execution_plan.go:1053-1118
Timestamp: 2025-12-02T08:25:26.682Z
Learning: In v2/pkg/engine/datasource/grpc_datasource field resolver response handling for composite types (interface/union), buildFieldResolverTypeMessage correctly combines both message.Fields and message.FieldSelectionSet: message.Fields contains interface-level fields that can be selected directly on the interface (such as __typename or fields defined in the interface itself), while message.FieldSelectionSet contains type-specific fields from inline fragments. This mixing is intentional and correct for GraphQL interfaces, as interface-level fields exist outside inline fragment selections and must be handled separately from type-specific fragment selections.

Applied to files:

  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go
📚 Learning: 2025-07-02T15:28:02.122Z
Learnt from: SkArchon
Repo: wundergraph/graphql-go-tools PR: 1203
File: v2/pkg/engine/resolve/loader.go:63-67
Timestamp: 2025-07-02T15:28:02.122Z
Learning: In the graphql-go-tools codebase, result structs are consistently initialized with non-nil bytes.Buffer instances, making additional nil checks for res.out unnecessary defensive programming.

Applied to files:

  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go
📚 Learning: 2025-11-19T09:42:17.644Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1341
File: v2/pkg/engine/datasource/grpc_datasource/execution_plan_visitor_federation.go:406-429
Timestamp: 2025-11-19T09:42:17.644Z
Learning: In the wundergraph/graphql-go-tools gRPC datasource implementation (v2/pkg/engine/datasource/grpc_datasource), field resolvers must have arguments. The system does not currently support defining field resolvers without arguments. This invariant ensures that the `parentCallID` increment in `enterFieldResolver` is always matched by a decrement in `LeaveField` (which checks `r.operation.FieldHasArguments(ref)`).

Applied to files:

  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go
📚 Learning: 2025-11-19T09:38:25.112Z
Learnt from: Noroth
Repo: wundergraph/graphql-go-tools PR: 1341
File: v2/pkg/engine/datasource/grpc_datasource/execution_plan_field_resolvers_test.go:1418-1526
Timestamp: 2025-11-19T09:38:25.112Z
Learning: In v2/pkg/engine/datasource/grpc_datasource test files, ResolvePath fields use snake_case (e.g., "test_containers.id") because they reference proto message field names, while JSONPath fields use camelCase (e.g., "testContainers") because they reference GraphQL/JSON response field names. Both casing styles are intentional and correct.

Applied to files:

  • v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go
🧬 Code graph analysis (1)
v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go (4)
v2/pkg/engine/resolve/datasource.go (1)
  • DataSource (10-13)
v2/pkg/engine/resolve/context.go (1)
  • Context (16-38)
v2/pkg/graphqlerrors/response.go (1)
  • Response (9-14)
v2/pkg/engine/resolve/resolve.go (1)
  • New (191-256)
🔇 Additional comments (2)
v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go (2)

207-209: This is an acceptable pattern—gRPC datasources intentionally don't support file uploads.

The LoadWithFiles panic is consistent with other datasources that don't support file uploads (e.g., staticdatasource also panics with "not implemented"). Only graphql_datasource implements this method with actual file upload support via httpclient.DoMultipartForm. gRPC's protocol doesn't support multipart form uploads, making the panic an appropriate way to indicate this capability is not available for this datasource type.


97-97: The headers parameter is intentionally unused in the gRPC datasource. Unlike HTTP-based datasources (e.g., GraphQL datasource), gRPC uses a different mechanism for request metadata through context. The parameter is part of the resolve.DataSource interface contract that all implementations must accept, but not all need to use—similar to the static and introspection datasources. No changes required.

Likely an incorrect or invalid review comment.

Comment on lines +104 to +106
defer func() {
d.pool.ReleaseMany(poolItems)
}()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue | 🔴 Critical

🧩 Analysis chain

🏁 Script executed:

# Find the file and understand its structure
fd "grpc_datasource.go" --type f

Repository: wundergraph/graphql-go-tools

Length of output: 131


🏁 Script executed:

# Look at the complete context around lines 104-186
fd "grpc_datasource.go" --type f -exec cat -n {} \; | sed -n '100,200p'

Repository: wundergraph/graphql-go-tools

Length of output: 302


🏁 Script executed:

# Find MarshalTo implementation to understand if it copies or references
rg "func.*MarshalTo" -A 10 --type go

Repository: wundergraph/graphql-go-tools

Length of output: 54


🏁 Script executed:

# Find merge operation implementations
rg "func.*mergeWith|func.*mergeValue" -A 15 --type go

Repository: wundergraph/graphql-go-tools

Length of output: 5096


🏁 Script executed:

# Find marshalResponseJSON to see arena usage
rg "marshalResponseJSON" -B 5 -A 20 --type go

Repository: wundergraph/graphql-go-tools

Length of output: 14729


🏁 Script executed:

# Find toDataObject implementation
rg "func.*toDataObject" -A 20 --type go

Repository: wundergraph/graphql-go-tools

Length of output: 2187


🏁 Script executed:

# Check astjson Value type to understand memory model
rg "type Value struct" -A 15 --type go

Repository: wundergraph/graphql-go-tools

Length of output: 835


🏁 Script executed:

# Find astjson package location
fd "astjson" --type d

Repository: wundergraph/graphql-go-tools

Length of output: 91


🏁 Script executed:

# Search for astjson Value type definition
rg "package astjson" -l --type go | head -5

Repository: wundergraph/graphql-go-tools

Length of output: 54


🏁 Script executed:

# Find MarshalTo implementation in astjson
rg "func.*MarshalTo" --type go | grep -i json

Repository: wundergraph/graphql-go-tools

Length of output: 54


🏁 Script executed:

# Search for astjson imports to understand package structure
rg "import.*astjson|from.*astjson" -A 2 -B 2 --type go | head -30

Repository: wundergraph/graphql-go-tools

Length of output: 447


🏁 Script executed:

# Get full context of the arena memory safety comment in loader.go
rg "it's important to copy the \*astjson.Value's off the arena" -B 5 -A 10 --type go

Repository: wundergraph/graphql-go-tools

Length of output: 1086


🏁 Script executed:

# Search for how they copy values off arena in loader.go
rg "CopyValue|CopyOff|copyFrom" --type go | head -20

Repository: wundergraph/graphql-go-tools

Length of output: 610


🏁 Script executed:

# Check if grpc_datasource.go does similar copying
rg -A 30 "func.*Load.*http" v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go | head -50

Repository: wundergraph/graphql-go-tools

Length of output: 1160


🏁 Script executed:

# Look for how astjson is imported and used - what package is it?
rg "astjson\." v2/pkg/engine/datasource/grpc_datasource/json_builder.go -B 2 | head -30

Repository: wundergraph/graphql-go-tools

Length of output: 1628


🏁 Script executed:

# Find astjson package imports to see where it comes from
rg "^import|astjson" v2/pkg/engine/datasource/grpc_datasource/json_builder.go | head -20

Repository: wundergraph/graphql-go-tools

Length of output: 1342


🏁 Script executed:

# Search for astjson in go.mod or vendor
fd "go.mod" -type f -exec head -100 {} \; | grep -i json

Repository: wundergraph/graphql-go-tools

Length of output: 242


🏁 Script executed:

# Look for astjson directory in the repo
find . -name "astjson*" -o -name "*astjson*" 2>/dev/null | head -20

Repository: wundergraph/graphql-go-tools

Length of output: 169


🏁 Script executed:

# Search for Value type with MarshalTo method
rg "type Value struct" -A 20 --type go | grep -B 5 -A 15 "MarshalTo"

Repository: wundergraph/graphql-go-tools

Length of output: 54


🏁 Script executed:

# Check fastjsonext which was found earlier
ls -la v2/pkg/fastjsonext/

Repository: wundergraph/graphql-go-tools

Length of output: 319


🌐 Web query:

astjson MarshalTo memory arena wundergraph

💡 Result:

Short answer: astjson's MarshalTo appends a Value/Object JSON encoding to a provided byte slice (dst) and returns the result — so reuse the same buffer (buf[:0]) to avoid allocations. Memory arenas are used when creating/modifying values: supply an arena.Arena to Set / ParseWithArena / ParseBytesWithArena so the value memory is allocated from that arena; those values remain valid only until the arena is reset (and you must not mutate or drop them). Use the arena for many small allocations and then reset it to free them cheaply.

Key points and links:

  • MarshalTo semantics: appends to dst and returns the resulting []byte (reuse buf[:0] to avoid allocations). [1]
  • Use arena.Arena when calling Set or the Parse*WithArena variants; values allocated from the arena must stay unchanged for the value’s lifetime and are invalidated when the arena/reset/Parse cycle ends. [1]
  • WunderGraph astjson repo and docs (package + examples). [2]

Sources:
[1] astjson package docs (MarshalTo, Value.Set, ParseWithArena). [https://pkg.go.dev/github.com/wundergraph/astjson]
[2] WunderGraph GitHub (wundergraph/astjson). [https://github.com/wundergraph/astjson]


🏁 Script executed:

# Check if there's any documentation in the repo about arena memory safety
rg "arena" v2/pkg/engine/datasource/grpc_datasource/ -B 2 -A 2 --type go | head -40

Repository: wundergraph/graphql-go-tools

Length of output: 3031


🏁 Script executed:

# Check grpc_datasource.go load function more completely - especially the return statement
cat -n v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go | sed -n '95,195p'

Repository: wundergraph/graphql-go-tools

Length of output: 3667


🏁 Script executed:

# Check if astjson.Value is pointer or value type when returned
rg "data.*MarshalTo|MarshalTo.*nil" v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go -B 5 -A 5

Repository: wundergraph/graphql-go-tools

Length of output: 342


🏁 Script executed:

# Verify the exact control flow - when does defer execute vs when does return value get evaluated
cat -n v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go | sed -n '180,190p'

Repository: wundergraph/graphql-go-tools

Length of output: 404


🏁 Script executed:

# Check toDataObject to see if it does any copying or if it just wraps
rg "func.*toDataObject" -A 10 v2/pkg/engine/datasource/grpc_datasource/json_builder.go

Repository: wundergraph/graphql-go-tools

Length of output: 575


🏁 Script executed:

# Confirm that mergeWithPath and mergeValues don't copy arena memory
rg "func.*mergeWithPath" -A 30 v2/pkg/engine/datasource/grpc_datasource/json_builder.go | head -40

Repository: wundergraph/graphql-go-tools

Length of output: 1000


🏁 Script executed:

# Check if there's any test or usage of MarshalTo that shows it returns independent memory
rg "MarshalTo" v2/pkg/engine/datasource/grpc_datasource/ -B 3 -A 3 --type go

Repository: wundergraph/graphql-go-tools

Length of output: 1079


🏁 Script executed:

# Search for any arena.Reset calls to understand arena lifecycle
rg "\.Reset\(|ReleaseMany" v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go -B 2 -A 2

Repository: wundergraph/graphql-go-tools

Length of output: 128


Critical: astjson values must be copied off arena before defer releases pool items.

The returned data at line 186 (value.MarshalTo(nil)) references astjson.Value objects that are backed by arena memory. Per the astjson documentation, these values "are invalidated when the arena/reset/Parse cycle ends." Since defer d.pool.ReleaseMany(poolItems) at line 104-106 releases the arenas after the return statement, the byte slice returned by MarshalTo becomes invalid.

This affects:

  • Line 140: per-call builders create responses using per-call arenas (builder.marshalResponseJSON)
  • Lines 171, 173: responses merged into root via mergeWithPath/mergeValues (these store references, not copies)
  • Line 185: toDataObject(root) wraps root without copying
  • Line 186: MarshalTo operates on values referencing released arena memory

The project already has a pattern for this in loader.go (line ~205): copy astjson.Value slices off the arena before releasing. Apply the same approach here—copy the final JSON value off the arena before returning, or ensure MarshalTo allocates independent memory rather than referencing arena-backed data.

🤖 Prompt for AI Agents
In v2/pkg/engine/datasource/grpc_datasource/grpc_datasource.go around lines
104-106 (and affecting code around 140,171,173,185-186), the returned JSON
references astjson arena-backed memory which is released by the deferred
d.pool.ReleaseMany(poolItems); fix it by copying the final value off the arena
before the arenas are released: after you produce the root/value but before
returning, marshal the value to bytes, allocate a new heap-backed slice of the
same length, copy the marshaled bytes into that new slice (or perform an
explicit deep copy of the astjson.Value into non-arena memory using the same
pattern as loader.go), then let the defer release the pool and return the
heap-copied slice/object so no returned data points at released arena memory.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants