Skip to content

Add Close function to batch interface #1566

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 2 commits into from
Jun 3, 2025
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 4 additions & 0 deletions clickhouse.go
Original file line number Diff line number Diff line change
Expand Up @@ -147,6 +147,8 @@ func (ch *clickhouse) Exec(ctx context.Context, query string, args ...any) error
if err != nil {
return err
}
conn.debugf("[acquired] connection [%d]", conn.id)

if err := conn.exec(ctx, query, args...); err != nil {
ch.release(conn, err)
return err
Expand Down Expand Up @@ -345,6 +347,8 @@ func (ch *clickhouse) release(conn *connect, err error) {
return
}
conn.released = true
conn.debugf("[released] connection [%d]", conn.id)

select {
case <-ch.open:
default:
Expand Down
15 changes: 15 additions & 0 deletions conn_batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -320,6 +320,21 @@ func (b *batch) closeQuery() error {
return nil
}

func (b *batch) Close() error {
if b.sent || b.released {
return nil
}

if err := b.closeQuery(); err != nil {
return err
}
b.sent = true

b.release(nil)

return nil
}

type batchColumn struct {
err error
batch driver.Batch
Expand Down
5 changes: 5 additions & 0 deletions conn_http_batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,11 @@ func (b *httpBatch) Flush() error {
return nil
}

func (b *httpBatch) Close() error {
b.sent = true
return nil
}

func (b *httpBatch) Abort() error {
defer func() {
b.sent = true
Expand Down
2 changes: 2 additions & 0 deletions examples/clickhouse_api/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,8 @@ func BatchInsert() error {
if err != nil {
return err
}
defer batch.Close()

for i := 0; i < 1000; i++ {
err := batch.Append(
uint8(42),
Expand Down
1 change: 1 addition & 0 deletions lib/driver/driver.go
Original file line number Diff line number Diff line change
Expand Up @@ -87,6 +87,7 @@ type (
IsSent() bool
Rows() int
Columns() []column.Interface
Close() error
}
BatchColumn interface {
Append(any) error
Expand Down
22 changes: 22 additions & 0 deletions tests/abort_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,3 +56,25 @@ func TestAbort(t *testing.T) {
}
}
}

func TestBatchClose(t *testing.T) {
conn, err := GetNativeConnection(nil, nil, &clickhouse.Compression{
Method: clickhouse.CompressionLZ4,
})
require.NoError(t, err)
ctx := context.Background()

batch, err := conn.PrepareBatch(ctx, "INSERT INTO function null('x UInt64') VALUES (1)")
require.NoError(t, err)
require.NoError(t, batch.Close())
require.NoError(t, batch.Close()) // No error on multiple calls

batch, err = conn.PrepareBatch(ctx, "INSERT INTO function null('x UInt64') VALUES (1)")
require.NoError(t, err)
if assert.NoError(t, batch.Append(uint8(1))) && assert.NoError(t, batch.Send()) {
var col1 uint8
if err := conn.QueryRow(ctx, "SELECT 1").Scan(&col1); assert.NoError(t, err) {
assert.Equal(t, uint8(1), col1)
}
}
}
Loading