This is an automated email from the ASF dual-hosted git repository.
zeroshade pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/arrow-adbc.git
The following commit(s) were added to refs/heads/main by this push:
new fb6306f4f fix: resolve Goroutine leak in database connection close
(#3491)
fb6306f4f is described below
commit fb6306f4f2cda2c75234882422b7bde0742db23e
Author: bruceNu1l <[email protected]>
AuthorDate: Wed Oct 22 00:38:00 2025 +0800
fix: resolve Goroutine leak in database connection close (#3491)
When a connection is opened via databaseImpl, the cache clientCache
initializes
*flightsql.Client objects which require proper cleanup on connection
close to
prevent Goroutine leaks.
Previously, closing a connection released only 3 of the 6 Goroutines
created per
connection, leaving 3 Goroutines associated with the cached client
unmanaged.
This resulted in accumulated Goroutine leaks over time.
The fix ensures all Goroutines are properly cleaned up when connections
are closed.
---
go/adbc/driver/flightsql/flightsql_connection.go | 1 +
go/adbc/driver/flightsql/flightsql_database.go | 8 +++++++-
2 files changed, 8 insertions(+), 1 deletion(-)
diff --git a/go/adbc/driver/flightsql/flightsql_connection.go
b/go/adbc/driver/flightsql/flightsql_connection.go
index 024950d3e..473d38770 100644
--- a/go/adbc/driver/flightsql/flightsql_connection.go
+++ b/go/adbc/driver/flightsql/flightsql_connection.go
@@ -1131,6 +1131,7 @@ func (c *connectionImpl) Close() error {
}
}
+ c.clientCache.Purge()
err = c.cl.Close()
c.cl = nil
return adbcFromFlightStatus(err, "Close")
diff --git a/go/adbc/driver/flightsql/flightsql_database.go
b/go/adbc/driver/flightsql/flightsql_database.go
index f79db14a9..316ad778b 100644
--- a/go/adbc/driver/flightsql/flightsql_database.go
+++ b/go/adbc/driver/flightsql/flightsql_database.go
@@ -494,7 +494,13 @@ func (d *databaseImpl) Open(ctx context.Context)
(adbc.Connection, error) {
if err != nil {
d.Logger.Debug("failed to close client",
"error", err.Error())
}
- }).Build()
+ }).PurgeVisitorFunc(func(_ interface{}, client interface{}) {
+ conn := client.(*flightsql.Client)
+ err := conn.Close()
+ if err != nil {
+ d.Logger.Debug("failed to close client", "error",
err.Error())
+ }
+ }).Build()
var cnxnSupport support