Skip to content

kafka: release sarama clients on cleanup and init failures#4437

Merged
ti-chi-bot[bot] merged 2 commits intopingcap:masterfrom
wlwilliamx:fix/release-sarama-admin-resources-4064
Mar 17, 2026
Merged

kafka: release sarama clients on cleanup and init failures#4437
ti-chi-bot[bot] merged 2 commits intopingcap:masterfrom
wlwilliamx:fix/release-sarama-admin-resources-4064

Conversation

@wlwilliamx
Copy link
Copy Markdown
Collaborator

@wlwilliamx wlwilliamx commented Mar 11, 2026

What problem does this PR solve?

Issue Number: close #4064

When a Kafka sink repeatedly enters warning/retry (for example, when the target topic is missing), TiCDC can leave Sarama clients alive across retries.

For producers created from an existing client, Sarama wraps the provided client with a nopCloserClient. That means closing the producer alone does not release the original client owned by TiCDC. If the original client is not closed, background goroutines, broker connections, and related caches/metrics can accumulate.

What is changed and how it works?

This PR tightens Sarama client cleanup in Kafka sink paths by:

  • keeping admin normal-close aligned with Sarama ownership semantics
  • closing the Sarama client when sarama.NewClusterAdminFromClient fails
  • explicitly closing the underlying client in saramaSyncProducer.Close()
  • closing the Sarama client when sarama.NewSyncProducerFromClient fails
  • closing the Sarama client when sarama.NewAsyncProducerFromClient fails
  • adding unit tests for admin and sync producer cleanup behavior

Check List

Tests

  • Unit test
  • Manual test (add detailed scripts or steps below)

Unit test:

  • go test ./pkg/sink/kafka -run 'TestSarama(AdminClientCloseDelegatesClientCleanupToAdmin|AdminClientCloseFallsBackToClientWhenAdminIsNil|SyncProducerCloseClosesClientAndProducer|SyncProducerCloseStillClosesProducerWhenClientCloseFails)' -count=1

Questions

Will it cause performance regression or break compatibility?

No. The change only makes Kafka Sarama client cleanup more complete on close and on partially initialized failure paths.

Do you need to update user documentation, design documentation or monitoring documentation?

No.

Release note

Release Kafka Sarama clients on cleanup and initialization failures to avoid leaking background goroutines, broker connections, and related caches during retries.

@ti-chi-bot ti-chi-bot Bot added the release-note Denotes a PR that will be considered when it comes time to generate release notes. label Mar 11, 2026
@coderabbitai
Copy link
Copy Markdown
Contributor

coderabbitai Bot commented Mar 11, 2026

📝 Walkthrough

Walkthrough

Refactors Kafka sink cleanup to use small sarama interfaces, ensures clients are closed on failure paths, makes Close() nil-safe and ordering-explicit for admin/client/producer, adds unit tests for Close behavior, and expands goleak ignore list for long‑lived Sarama and systemd/dbus routines. (46 words)

Changes

Cohort / File(s) Summary
Leak detection
pkg/leakutil/leak_helper.go
Adds goleak ignores for systemd/dbus Conn.inWorker and Sarama background/updater goroutines to suppress benign long‑lived routines.
Kafka admin refactor
pkg/sink/kafka/admin.go
Replaces concrete sarama types with saramaClient and saramaClusterAdmin interfaces; makes saramaAdminClient.Close() nil-safe and attempts to close admin first, falling back to closing client.
Kafka admin tests
pkg/sink/kafka/admin_test.go
New tests and test doubles asserting Close() invokes admin and client cleanup and is safe when admin is nil.
Sarama factory error‑path cleanup
pkg/sink/kafka/sarama_factory.go
Closes underlying Sarama client on failure paths (ClusterAdmin/producer creation failures) to prevent goroutine/resource leaks.
Sync producer refactor
pkg/sink/kafka/sarama_sync_producer.go
Introduces saramaSyncClient and saramaSyncProducerClient interfaces; separates client and producer fields and makes Close() explicitly close client then producer with logging and error handling.
Sync producer tests
pkg/sink/kafka/sarama_sync_producer_test.go
Adds tests and doubles verifying Close() calls client and producer Close exactly once and remains robust when client.Close() errors.

Sequence Diagram(s)

sequenceDiagram
  participant CF as Changefeed (owner)
  participant Wrapper as saramaAdminClient / saramaSyncProducer
  participant Admin as saramaClusterAdmin
  participant Client as saramaClient / saramaSyncClient
  participant Producer as saramaSyncProducerClient
  rect rgba(200,230,255,0.5)
    CF->>Wrapper: Close()
  end
  alt admin exists
    Wrapper->>Admin: Close()
    Admin-->>Wrapper: success
    Wrapper-->>CF: return (done)
  else admin close fails or admin nil
    Wrapper->>Admin: Close() (if non-nil) 
    Admin-->>Wrapper: error / nil
    Wrapper->>Client: Close()
    Client-->>Wrapper: success/error
    alt producer present (sync producer case)
      Wrapper->>Producer: Close()
      Producer-->>Wrapper: success/error
    end
    Wrapper-->>CF: return
  end
Loading

Estimated code review effort

🎯 3 (Moderate) | ⏱️ ~25 minutes

Poem

🐰 I nibbled leaks in midnight code,

Closed the paths where goroutines strode,
Admin first, then client too,
Tests hop in to prove it's true,
Now streams sleep sound — a cleaner road.

🚥 Pre-merge checks | ✅ 4 | ❌ 1

❌ Failed checks (1 warning)

Check name Status Explanation Resolution
Docstring Coverage ⚠️ Warning Docstring coverage is 0.00% which is insufficient. The required threshold is 80.00%. Write docstrings for the functions missing them to satisfy the coverage threshold.
✅ Passed checks (4 passed)
Check name Status Explanation
Linked Issues check ✅ Passed The PR directly addresses issue #4064 by fixing resource leaks through proper cleanup of underlying sarama clients during admin closure and error paths, preventing unbounded CPU/memory growth during retries.
Out of Scope Changes check ✅ Passed All changes are directly scoped to fixing Kafka admin resource cleanup: admin client interface updates, producer client cleanup, leak helper additions, and corresponding unit tests without introducing unrelated modifications.
Title check ✅ Passed The title accurately summarizes the main change: ensuring Sarama clients are released during cleanup and initialization failure paths, which directly addresses the resource leak described in the PR objectives.
Description check ✅ Passed The pull request description comprehensively covers the problem, solution, testing approach, compatibility assessment, and includes a well-formatted release note following all required sections.

✏️ Tip: You can configure your own custom pre-merge checks in the settings.

✨ Finishing Touches
🧪 Generate unit tests (beta)
  • Create PR with unit tests
📝 Coding Plan
  • Generate coding plan for human review comments

Thanks for using CodeRabbit! It's free for OSS, and your support helps us grow. If you like it, consider giving us a shout-out.

❤️ Share

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

Tip

CodeRabbit can enforce grammar and style rules using `languagetool`.

Configure the reviews.tools.languagetool setting to enable/disable rules and categories. Refer to the LanguageTool Community to learn more.

@ti-chi-bot ti-chi-bot Bot added the size/L Denotes a PR that changes 100-499 lines, ignoring generated files. label Mar 11, 2026
Copy link
Copy Markdown
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

🧹 Nitpick comments (1)
pkg/sink/kafka/admin_test.go (1)

24-64: Assert the shutdown order, not just that both flags flipped.

The production fix depends on closing admin before client. This test still passes if a future refactor swaps those calls, so it won't catch the most important regression. Consider recording call order in the fakes and asserting admin closes before client. As per coding guidelines, **/*_test.go: "favor deterministic tests and use testify/require"

Also applies to: 66-85

🤖 Prompt for AI Agents
Verify each finding against the current code and only fix it if needed.

In `@pkg/sink/kafka/admin_test.go` around lines 24 - 64, The test currently only
checks that testSaramaClient.closed and testSaramaClusterAdmin.closed became
true; update the fakes (testSaramaClient.Close and testSaramaClusterAdmin.Close)
to append a timestamped or sequential marker to a shared slice (e.g., callOrder)
when Close is called, then in the test assert using testify/require that the
callOrder shows admin's Close occurred before client's Close (i.e., "admin"
appears earlier than "client"); ensure you protect the shared slice with a mutex
if needed and use require.Equal/require.True to make the order assertion
deterministic.
🤖 Prompt for all review comments with AI agents
Verify each finding against the current code and only fix it if needed.

Inline comments:
In `@pkg/leakutil/leak_helper.go`:
- Around line 36-37: Remove the two Sarama goroutines
("github.com/IBM/sarama.(*client).backgroundMetadataUpdater" and
"github.com/IBM/sarama.(*Broker).responseReceiver") from the shared defaultOpts
allowlist in leak_helper.go and instead add those goleak.IgnoreTopFunction
entries only in the specific tests that intentionally tolerate async
producer/client shutdown; update the tests that currently expect those leaks to
append these IgnoreTopFunction options when calling goleak.VerifyNone (or the
test-specific leak verification helper) so the global defaultOpts no longer
hides the regression caused by not closing the Sarama client.

In `@pkg/sink/kafka/sarama_factory.go`:
- Around line 90-94: The cleanup path after sarama.NewClusterAdminFromClient
failure currently calls client.Close() and discards any error; capture the
result of client.Close() (e.g., closeErr) and log that closeErr (using the
package logger) while still returning the original admin-construction error
(err); update the block in sarama_factory.go that handles the
NewClusterAdminFromClient failure so it logs the client close failure for
visibility but preserves returning errors.Trace(err).

---

Nitpick comments:
In `@pkg/sink/kafka/admin_test.go`:
- Around line 24-64: The test currently only checks that testSaramaClient.closed
and testSaramaClusterAdmin.closed became true; update the fakes
(testSaramaClient.Close and testSaramaClusterAdmin.Close) to append a
timestamped or sequential marker to a shared slice (e.g., callOrder) when Close
is called, then in the test assert using testify/require that the callOrder
shows admin's Close occurred before client's Close (i.e., "admin" appears
earlier than "client"); ensure you protect the shared slice with a mutex if
needed and use require.Equal/require.True to make the order assertion
deterministic.

ℹ️ Review info
⚙️ Run configuration

Configuration used: Organization UI

Review profile: CHILL

Plan: Pro

Run ID: cd98f55d-1708-4dd6-bb25-978b89f59e75

📥 Commits

Reviewing files that changed from the base of the PR and between 27c7bbe and fb7f65a.

📒 Files selected for processing (4)
  • pkg/leakutil/leak_helper.go
  • pkg/sink/kafka/admin.go
  • pkg/sink/kafka/admin_test.go
  • pkg/sink/kafka/sarama_factory.go

Comment on lines +36 to +37
goleak.IgnoreTopFunction("github.com/IBM/sarama.(*client).backgroundMetadataUpdater"),
goleak.IgnoreTopFunction("github.com/IBM/sarama.(*Broker).responseReceiver"),
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

⚠️ Potential issue | 🟠 Major

Don't add these Sarama goroutines to the global goleak allowlist.

These are the exact goroutines leaked when the admin wrapper forgets to close its underlying Sarama client. Putting them in defaultOpts will make the suite ignore the regression this PR is trying to prevent. Please scope these ignores to the few tests that intentionally tolerate async producer shutdown instead of the shared default list.

🤖 Prompt for AI Agents
Verify each finding against the current code and only fix it if needed.

In `@pkg/leakutil/leak_helper.go` around lines 36 - 37, Remove the two Sarama
goroutines ("github.com/IBM/sarama.(*client).backgroundMetadataUpdater" and
"github.com/IBM/sarama.(*Broker).responseReceiver") from the shared defaultOpts
allowlist in leak_helper.go and instead add those goleak.IgnoreTopFunction
entries only in the specific tests that intentionally tolerate async
producer/client shutdown; update the tests that currently expect those leaks to
append these IgnoreTopFunction options when calling goleak.VerifyNone (or the
test-specific leak verification helper) so the global defaultOpts no longer
hides the regression caused by not closing the Sarama client.

Comment on lines 90 to 94
if err != nil {
// `sarama.NewClusterAdminFromClient` does not take ownership of the client,
// so we need to close it on failures to avoid leaking background goroutines.
_ = client.Close()
return nil, errors.Trace(err)
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

⚠️ Potential issue | 🟡 Minor

Don't swallow the client-close failure on this cleanup path.

If admin creation fails and client.Close() also fails, we lose the only signal that the leak-prevention cleanup did not complete. Please log closeErr here while still returning the original admin-construction error.

Suggested change
 	if err != nil {
 		// `sarama.NewClusterAdminFromClient` does not take ownership of the client,
 		// so we need to close it on failures to avoid leaking background goroutines.
-		_ = client.Close()
+		if closeErr := client.Close(); closeErr != nil {
+			log.Warn("close kafka client after admin creation failed",
+				zap.Stringer("changefeedID", changefeedID),
+				zap.Error(closeErr))
+		}
 		return nil, errors.Trace(err)
 	}
📝 Committable suggestion

‼️ IMPORTANT
Carefully review the code before committing. Ensure that it accurately replaces the highlighted code, contains no missing lines, and has no issues with indentation. Thoroughly test & benchmark the code to ensure it meets the requirements.

Suggested change
if err != nil {
// `sarama.NewClusterAdminFromClient` does not take ownership of the client,
// so we need to close it on failures to avoid leaking background goroutines.
_ = client.Close()
return nil, errors.Trace(err)
if err != nil {
// `sarama.NewClusterAdminFromClient` does not take ownership of the client,
// so we need to close it on failures to avoid leaking background goroutines.
if closeErr := client.Close(); closeErr != nil {
log.Warn("close kafka client after admin creation failed",
zap.Stringer("changefeedID", changefeedID),
zap.Error(closeErr))
}
return nil, errors.Trace(err)
}
🤖 Prompt for AI Agents
Verify each finding against the current code and only fix it if needed.

In `@pkg/sink/kafka/sarama_factory.go` around lines 90 - 94, The cleanup path
after sarama.NewClusterAdminFromClient failure currently calls client.Close()
and discards any error; capture the result of client.Close() (e.g., closeErr)
and log that closeErr (using the package logger) while still returning the
original admin-construction error (err); update the block in sarama_factory.go
that handles the NewClusterAdminFromClient failure so it logs the client close
failure for visibility but preserves returning errors.Trace(err).

@gemini-code-assist
Copy link
Copy Markdown

Summary of Changes

Hello, I'm Gemini Code Assist1! I'm currently reviewing this pull request and will post my feedback shortly. In the meantime, here's a summary to help you and other reviewers quickly get up to speed!

This pull request addresses a critical resource leakage issue within the Kafka admin client, particularly when the sink retries against missing topics. By ensuring that both the ClusterAdmin and its underlying sarama.Client are explicitly closed during normal operation and upon creation failures, the changes prevent the accumulation of background goroutines, cached metadata, and metrics. This significantly improves the stability and resource efficiency of the Kafka sink.

Highlights

  • Resource Leak Prevention: Ensured that the underlying sarama.Client is properly closed when saramaAdminClient.Close() is called, preventing background goroutine and memory leaks, especially during Kafka sink retries.
  • Robust Error Handling: Implemented cleanup for the sarama.Client when sarama.NewClusterAdminFromClient fails, as this API does not take ownership of the client, thus preventing leaks during client creation errors.
  • Improved Testability and Safety: Introduced interfaces for sarama.Client and sarama.ClusterAdmin to enhance testability and added comprehensive unit tests to verify correct close behavior and nil-tolerant cleanup.
  • Leak Detection Configuration Update: Updated goleak ignore list to account for new github.com/IBM/sarama library functions and github.com/godbus/dbus systemd helpers, reducing false positives in leak detection.

🧠 New Feature in Public Preview: You can now enable Memory to help Gemini Code Assist learn from your team's feedback. This makes future code reviews more consistent and personalized to your project's style. Click here to enable Memory in your admin console.

Changelog
  • pkg/leakutil/leak_helper.go
    • Added goleak.IgnoreAnyFunction for github.com/godbus/dbus to prevent false positives from systemd-related long-lived dbus connections.
    • Added goleak.IgnoreTopFunction for github.com/IBM/sarama background metadata updater and response receiver goroutines to handle asynchronous producer closing.
  • pkg/sink/kafka/admin.go
    • Introduced saramaClient and saramaClusterAdmin interfaces to abstract the sarama.Client and sarama.ClusterAdmin types, improving testability.
    • Modified the saramaAdminClient struct to use the new interfaces.
    • Updated the Close() method to explicitly close both the admin and the client fields, with nil checks, ensuring all resources are released.
  • pkg/sink/kafka/admin_test.go
    • Added a new test file for Kafka admin client.
    • Implemented mock testSaramaClient and testSaramaClusterAdmin for testing purposes.
    • Added TestSaramaAdminClientCloseClosesAdminAndClient to verify that Close() correctly shuts down both the admin and the underlying client.
    • Added TestSaramaAdminClientCloseToleratesNilFields to ensure Close() handles nil admin or client fields without panicking.
  • pkg/sink/kafka/sarama_factory.go
    • Added a client.Close() call in the newAdminClient function's error handling path to ensure the underlying sarama.Client is closed if sarama.NewClusterAdminFromClient fails.
Activity
  • CodeRabbit AI provided a summary of the changes, highlighting bug fixes in resource cleanup, enhanced shutdown logic, extended leak detection configuration, and added comprehensive tests.
Using Gemini Code Assist

The full guide for Gemini Code Assist can be found on our documentation page, here are some quick tips.

Invoking Gemini

You can request assistance from Gemini at any point by creating a comment using either /gemini <command> or @gemini-code-assist <command>. Below is a summary of the supported commands on the current page.

Feature Command Description
Code Review /gemini review Performs a code review for the current pull request in its current state.
Pull Request Summary /gemini summary Provides a summary of the current pull request in its current state.
Comment @gemini-code-assist Responds in comments when explicitly tagged, both in pull request comments and review comments.
Help /gemini help Displays a list of available commands.

Customization

To customize Gemini Code Assist for GitHub experience, repository maintainers can create a configuration file and/or provide a custom code review style guide (such as PEP-8 for Python) by creating and adding files to a .gemini/ folder in the base of the repository. Detailed instructions can be found here.

Limitations & Feedback

Gemini Code Assist may make mistakes. Please leave feedback on any instances where its feedback is incorrect or counter productive. You can react with 👍 and 👎 on @gemini-code-assist comments. If you're interested in giving your feedback about your experience with Gemini Code Assist for GitHub and other Google products, sign up here.

You can also get AI-powered code generation, chat, as well as code reviews directly in the IDE at no cost with the Gemini Code Assist IDE Extension.

Footnotes

  1. Review the Privacy Notices, Generative AI Prohibited Use Policy, Terms of Service, and learn how to configure Gemini Code Assist in GitHub here. Gemini can make mistakes, so double check it and use code with caution.

Copy link
Copy Markdown

@gemini-code-assist gemini-code-assist Bot left a comment

Choose a reason for hiding this comment

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

Code Review

This pull request effectively addresses a critical resource leak in the Kafka admin client by ensuring that underlying Sarama client resources are properly released, even when the sarama.NewClusterAdminFromClient call fails. The introduction of interfaces for saramaClient and saramaClusterAdmin significantly improves the testability and maintainability of the saramaAdminClient. Additionally, the Close() method has been made more robust by handling nil fields and ensuring a correct shutdown sequence. The comprehensive unit tests added in admin_test.go validate these improvements, covering both successful closure and nil-tolerance scenarios. The updates to the goleak ignore list also help in reducing false positives during leak detection, contributing to more stable tests.

if err != nil {
// `sarama.NewClusterAdminFromClient` does not take ownership of the client,
// so we need to close it on failures to avoid leaking background goroutines.
_ = client.Close()
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

critical

This is a critical fix. Explicitly closing the client here prevents resource leaks (background goroutines and cached metadata/metrics) if sarama.NewClusterAdminFromClient fails, as this function does not take ownership of the client. This directly addresses the problem described in the PR.

@ti-chi-bot ti-chi-bot Bot added needs-1-more-lgtm Indicates a PR needs 1 more LGTM. approved labels Mar 12, 2026
Comment thread pkg/sink/kafka/admin.go
admin saramaClusterAdmin
}

type saramaClient interface {
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

these 2 interface looks similar to the admin interface.

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

It's not suggested to test on the sarama.

"github.com/stretchr/testify/require"
)

type testSaramaClient struct {
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

You should use gomock to generate a mocked admin client for the test purpose.

Comment thread pkg/sink/kafka/admin.go
// Close admin first (may send RPCs), then close the underlying client to stop
// sarama background goroutines and release related caches/metrics.
if a.admin != nil {
if err := a.admin.Close(); err != nil {
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

this call should already close the admin underline client.

@ti-chi-bot ti-chi-bot Bot added size/XL Denotes a PR that changes 500-999 lines, ignoring generated files. and removed size/L Denotes a PR that changes 100-499 lines, ignoring generated files. labels Mar 17, 2026
Copy link
Copy Markdown
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 (1)
pkg/sink/kafka/sarama_factory.go (1)

90-94: ⚠️ Potential issue | 🟡 Minor

Log the client close error for observability.

The cleanup path correctly closes the client to prevent goroutine leaks, but discarding the close error (_ = client.Close()) hides potential cleanup failures. As flagged in a previous review, logging this error improves observability without changing the returned error.

Suggested change
 	if err != nil {
 		// `sarama.NewClusterAdminFromClient` does not take ownership of the client,
 		// so we need to close it on failures to avoid leaking background goroutines.
-		_ = client.Close()
+		if closeErr := client.Close(); closeErr != nil {
+			log.Warn("close kafka client after admin creation failed",
+				zap.Stringer("changefeedID", changefeedID),
+				zap.Error(closeErr))
+		}
 		return nil, errors.Trace(err)
 	}
🤖 Prompt for AI Agents
Verify each finding against the current code and only fix it if needed.

In `@pkg/sink/kafka/sarama_factory.go` around lines 90 - 94, The cleanup path
swallows client.Close() errors, reducing observability; modify the error
handling after sarama.NewClusterAdminFromClient fails to capture the close error
(e.g., errClose := client.Close()) and log it (using the package's logger or
standard log) without altering the returned error from the function, so we still
return errors.Trace(err) while recording any failure from client.Close().
🧹 Nitpick comments (1)
pkg/sink/kafka/sarama_sync_producer_test.go (1)

57-87: Consider adding tests for nil fields and double-close idempotency.

The current tests cover the happy path and client-error scenario well. For completeness, consider adding:

  1. Test with nil client to verify nil-safety
  2. Test calling Close() twice to verify idempotent behavior via the closed atomic flag
🤖 Prompt for AI Agents
Verify each finding against the current code and only fix it if needed.

In `@pkg/sink/kafka/sarama_sync_producer_test.go` around lines 57 - 87, Add two
unit tests for saramaSyncProducer.Close: one named like
TestSaramaSyncProducerCloseWithNilClient that constructs a saramaSyncProducer
with producer set (and client == nil) and asserts Close() does not panic and
only closes the producer (producer.closeCalls == 1, client nil safe); and
another named like TestSaramaSyncProducerDoubleCloseIdempotent that calls
Close() twice on a saramaSyncProducer with both client and producer set and
asserts closeCalls for both client and producer increment only once (idempotent
via closed atomic flag) and no panic on second call; reference
saramaSyncProducer.Close, testSaramaSyncProducer, and testSyncProducerClient to
locate code.
🤖 Prompt for all review comments with AI agents
Verify each finding against the current code and only fix it if needed.

Inline comments:
In `@pkg/sink/kafka/sarama_factory.go`:
- Around line 154-158: When NewAsyncProducerFromClient fails in the
sarama.NewAsyncProducerFromClient block, the code currently discards the error
returned by client.Close(); instead capture and log that close error for
consistency: call client.Close(), check its returned error and emit a log entry
(or attach it to the returned error) before returning
errors.WrapError(errors.ErrKafkaNewProducer, err); update the failing block
around sarama.NewAsyncProducerFromClient and client.Close to include this logged
close error so the close failure isn’t silently ignored.
- Around line 126-130: When NewSyncProducerFromClient fails we currently discard
client.Close() errors; change the error path in the
sarama.NewSyncProducerFromClient handling to capture the result of
client.Close(), and log that close error (with context) before returning the
wrapped ErrKafkaNewProducer using errors.WrapError. Locate the block around
sarama.NewSyncProducerFromClient and replace the silent call to client.Close()
with capturing its error (e.g., errClose := client.Close()) and emit a log entry
including errClose and context (consistent with how the admin creation path logs
close failures) prior to returning the wrapped error.

---

Duplicate comments:
In `@pkg/sink/kafka/sarama_factory.go`:
- Around line 90-94: The cleanup path swallows client.Close() errors, reducing
observability; modify the error handling after sarama.NewClusterAdminFromClient
fails to capture the close error (e.g., errClose := client.Close()) and log it
(using the package's logger or standard log) without altering the returned error
from the function, so we still return errors.Trace(err) while recording any
failure from client.Close().

---

Nitpick comments:
In `@pkg/sink/kafka/sarama_sync_producer_test.go`:
- Around line 57-87: Add two unit tests for saramaSyncProducer.Close: one named
like TestSaramaSyncProducerCloseWithNilClient that constructs a
saramaSyncProducer with producer set (and client == nil) and asserts Close()
does not panic and only closes the producer (producer.closeCalls == 1, client
nil safe); and another named like TestSaramaSyncProducerDoubleCloseIdempotent
that calls Close() twice on a saramaSyncProducer with both client and producer
set and asserts closeCalls for both client and producer increment only once
(idempotent via closed atomic flag) and no panic on second call; reference
saramaSyncProducer.Close, testSaramaSyncProducer, and testSyncProducerClient to
locate code.

ℹ️ Review info
⚙️ Run configuration

Configuration used: Organization UI

Review profile: CHILL

Plan: Pro

Run ID: 250cb26c-6326-4989-9fa1-b2156d4d4c83

📥 Commits

Reviewing files that changed from the base of the PR and between fb7f65a and dd8e9f1.

📒 Files selected for processing (5)
  • pkg/sink/kafka/admin.go
  • pkg/sink/kafka/admin_test.go
  • pkg/sink/kafka/sarama_factory.go
  • pkg/sink/kafka/sarama_sync_producer.go
  • pkg/sink/kafka/sarama_sync_producer_test.go
🚧 Files skipped from review as they are similar to previous changes (1)
  • pkg/sink/kafka/admin_test.go

Comment on lines 126 to 130
p, err := sarama.NewSyncProducerFromClient(client)
if err != nil {
_ = client.Close()
return nil, errors.WrapError(errors.ErrKafkaNewProducer, err)
}
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

⚠️ Potential issue | 🟡 Minor

Log the client close error for consistency.

Same concern as the admin creation path: silently discarding the close error loses visibility into potential cleanup failures.

Suggested change
 	p, err := sarama.NewSyncProducerFromClient(client)
 	if err != nil {
-		_ = client.Close()
+		if closeErr := client.Close(); closeErr != nil {
+			log.Warn("close kafka client after sync producer creation failed",
+				zap.Stringer("changefeedID", f.changefeedID),
+				zap.Error(closeErr))
+		}
 		return nil, errors.WrapError(errors.ErrKafkaNewProducer, err)
 	}
📝 Committable suggestion

‼️ IMPORTANT
Carefully review the code before committing. Ensure that it accurately replaces the highlighted code, contains no missing lines, and has no issues with indentation. Thoroughly test & benchmark the code to ensure it meets the requirements.

Suggested change
p, err := sarama.NewSyncProducerFromClient(client)
if err != nil {
_ = client.Close()
return nil, errors.WrapError(errors.ErrKafkaNewProducer, err)
}
p, err := sarama.NewSyncProducerFromClient(client)
if err != nil {
if closeErr := client.Close(); closeErr != nil {
log.Warn("close kafka client after sync producer creation failed",
zap.Stringer("changefeedID", f.changefeedID),
zap.Error(closeErr))
}
return nil, errors.WrapError(errors.ErrKafkaNewProducer, err)
}
🤖 Prompt for AI Agents
Verify each finding against the current code and only fix it if needed.

In `@pkg/sink/kafka/sarama_factory.go` around lines 126 - 130, When
NewSyncProducerFromClient fails we currently discard client.Close() errors;
change the error path in the sarama.NewSyncProducerFromClient handling to
capture the result of client.Close(), and log that close error (with context)
before returning the wrapped ErrKafkaNewProducer using errors.WrapError. Locate
the block around sarama.NewSyncProducerFromClient and replace the silent call to
client.Close() with capturing its error (e.g., errClose := client.Close()) and
emit a log entry including errClose and context (consistent with how the admin
creation path logs close failures) prior to returning the wrapped error.

Comment on lines 154 to 158
p, err := sarama.NewAsyncProducerFromClient(client)
if err != nil {
_ = client.Close()
return nil, errors.WrapError(errors.ErrKafkaNewProducer, err)
}
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

⚠️ Potential issue | 🟡 Minor

Log the client close error for consistency.

Same pattern as other creation paths: log the close error rather than discarding it.

Suggested change
 	p, err := sarama.NewAsyncProducerFromClient(client)
 	if err != nil {
-		_ = client.Close()
+		if closeErr := client.Close(); closeErr != nil {
+			log.Warn("close kafka client after async producer creation failed",
+				zap.Stringer("changefeedID", f.changefeedID),
+				zap.Error(closeErr))
+		}
 		return nil, errors.WrapError(errors.ErrKafkaNewProducer, err)
 	}
📝 Committable suggestion

‼️ IMPORTANT
Carefully review the code before committing. Ensure that it accurately replaces the highlighted code, contains no missing lines, and has no issues with indentation. Thoroughly test & benchmark the code to ensure it meets the requirements.

Suggested change
p, err := sarama.NewAsyncProducerFromClient(client)
if err != nil {
_ = client.Close()
return nil, errors.WrapError(errors.ErrKafkaNewProducer, err)
}
p, err := sarama.NewAsyncProducerFromClient(client)
if err != nil {
if closeErr := client.Close(); closeErr != nil {
log.Warn("close kafka client after async producer creation failed",
zap.Stringer("changefeedID", f.changefeedID),
zap.Error(closeErr))
}
return nil, errors.WrapError(errors.ErrKafkaNewProducer, err)
}
🤖 Prompt for AI Agents
Verify each finding against the current code and only fix it if needed.

In `@pkg/sink/kafka/sarama_factory.go` around lines 154 - 158, When
NewAsyncProducerFromClient fails in the sarama.NewAsyncProducerFromClient block,
the code currently discards the error returned by client.Close(); instead
capture and log that close error for consistency: call client.Close(), check its
returned error and emit a log entry (or attach it to the returned error) before
returning errors.WrapError(errors.ErrKafkaNewProducer, err); update the failing
block around sarama.NewAsyncProducerFromClient and client.Close to include this
logged close error so the close failure isn’t silently ignored.

@wlwilliamx wlwilliamx changed the title kafka: release sarama admin resources on close kafka: release sarama clients on cleanup and init failures Mar 17, 2026
"go.uber.org/zap"
)

type saramaSyncClient interface {
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

redundant interface。

@ti-chi-bot
Copy link
Copy Markdown

ti-chi-bot Bot commented Mar 17, 2026

[APPROVALNOTIFIER] This PR is APPROVED

This pull-request has been approved by: 3AceShowHand, lidezhu

The full list of commands accepted by this bot can be found here.

The pull request process is described here

Details Needs approval from an approver in each of these files:
  • OWNERS [3AceShowHand,lidezhu]

Approvers can indicate their approval by writing /approve in a comment
Approvers can cancel approval by writing /approve cancel in a comment

@ti-chi-bot ti-chi-bot Bot added lgtm and removed needs-1-more-lgtm Indicates a PR needs 1 more LGTM. labels Mar 17, 2026
@ti-chi-bot
Copy link
Copy Markdown

ti-chi-bot Bot commented Mar 17, 2026

[LGTM Timeline notifier]

Timeline:

  • 2026-03-12 09:55:28.248755279 +0000 UTC m=+516759.760812930: ☑️ agreed by lidezhu.
  • 2026-03-17 11:50:12.788929371 +0000 UTC m=+271339.876586908: ☑️ agreed by 3AceShowHand.

@wlwilliamx
Copy link
Copy Markdown
Collaborator Author

/test all

@wlwilliamx
Copy link
Copy Markdown
Collaborator Author

/retest

@ti-chi-bot ti-chi-bot Bot merged commit a39c233 into pingcap:master Mar 17, 2026
23 of 25 checks passed
tenfyzhong pushed a commit that referenced this pull request Mar 18, 2026
close #4064

(cherry picked from commit a39c233)
Signed-off-by: tenfyzhong <tenfy@tenfy.cn>
@wlwilliamx wlwilliamx added the needs-cherry-pick-release-8.5 Should cherry pick this PR to release-8.5 branch. label Mar 18, 2026
@ti-chi-bot
Copy link
Copy Markdown
Member

In response to a cherrypick label: new pull request created to branch release-8.5: #4519.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

approved lgtm needs-cherry-pick-release-8.5 Should cherry pick this PR to release-8.5 branch. release-note Denotes a PR that will be considered when it comes time to generate release notes. size/XL Denotes a PR that changes 500-999 lines, ignoring generated files.

Projects

None yet

Development

Successfully merging this pull request may close these issues.

TiCDC Kafka sink: missing topic keeps changefeed in warning (expected) but pegs CPU/memory

4 participants