Skip to content

Commit

Permalink
Experimental feature of rows blocks be appended directly to the batch (
Browse files Browse the repository at this point in the history
  • Loading branch information
jkaflik authored Mar 14, 2024
1 parent 22a3351 commit 9491310
Show file tree
Hide file tree
Showing 2 changed files with 117 additions and 0 deletions.
37 changes: 37 additions & 0 deletions conn_batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -133,6 +133,13 @@ func (b *batch) Append(v ...any) error {
if b.err != nil {
return b.err
}

if len(v) > 0 {
if r, ok := v[0].(*rows); ok {
return b.appendRowsBlocks(r)
}
}

if err := b.block.Append(v...); err != nil {
b.err = errors.Wrap(ErrBatchInvalid, err.Error())
b.release(err)
Expand All @@ -141,6 +148,36 @@ func (b *batch) Append(v ...any) error {
return nil
}

// appendRowsBlocks is an experimental feature that allows rows blocks be appended directly to the batch.
// This API is not stable and may be changed in the future.
// See: tests/batch_block_test.go
func (b *batch) appendRowsBlocks(r *rows) error {
var lastReadLock *proto.Block
var blockNum int

for r.Next() {
if lastReadLock == nil { // make sure the first block is logged
b.conn.debugf("[batch.appendRowsBlocks] blockNum = %d", blockNum)
}

// rows.Next() will read the next block from the server only if the current block is empty
// only if new block is available we should flush the current block
// the last block will be handled by the batch.Send() method
if lastReadLock != nil && lastReadLock != r.block {
if err := b.Flush(); err != nil {
return err
}
blockNum++
b.conn.debugf("[batch.appendRowsBlocks] blockNum = %d", blockNum)
}

b.block = r.block
lastReadLock = r.block
}

return nil
}

func (b *batch) AppendStruct(v any) error {
if b.err != nil {
return b.err
Expand Down
80 changes: 80 additions & 0 deletions tests/batch_block_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,80 @@
// Licensed to ClickHouse, Inc. under one or more contributor
// license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright
// ownership. ClickHouse, Inc. licenses this file to you under
// the Apache License, Version 2.0 (the "License"); you may
// not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

package tests

import (
"sync/atomic"
"testing"

"github.com/ClickHouse/clickhouse-go/v2"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"golang.org/x/net/context"
)

// TestBatchAppendRows tests experimental batch rows blocks append feature.
// This API is not stable and may be changed in the future.
func TestBatchAppendRows(t *testing.T) {
te, err := GetTestEnvironment(testSet)
require.NoError(t, err)
blocksRead := atomic.Uint64{}
opts := ClientOptionsFromEnv(te, clickhouse.Settings{})
opts.Debug = true
opts.Debugf = func(format string, args ...interface{}) {
if format == "[batch.appendRowsBlocks] blockNum = %d" {
blocksRead.Store(uint64(args[0].(int))) // store the last block number read from rows
}
}

conn, err := GetConnectionWithOptions(&opts)
require.NoError(t, err)

ctx := context.Background()

// given we have two tables and a million rows in the source table
var tables = []string{"source", "target"}
for _, table := range tables {
require.NoError(t, conn.Exec(context.Background(), "create table if not exists "+table+" (number1 Int, number2 String, number3 Tuple(String, Int), number4 DateTime) engine = MergeTree() order by tuple()"))
defer conn.Exec(context.Background(), "drop table if exists "+table)
}

require.NoError(t, conn.Exec(ctx, "INSERT INTO source SELECT number, 'string', tuple('foo', number), now() FROM system.numbers LIMIT 1000000"))

// when we create a batch with direct data block access 10 times

selectCtx := clickhouse.Context(ctx, clickhouse.WithSettings(clickhouse.Settings{
"max_block_size": 1000,
}))

sourceRows, err := conn.Query(selectCtx, "SELECT * FROM source")
require.NoError(t, err)
defer sourceRows.Close()

b, err := conn.PrepareBatch(ctx, "INSERT INTO target")
require.NoError(t, err)
require.NoError(t, b.Append(sourceRows))
require.NoError(t, b.Send())

// then we should be able to see the data in the target table
row := conn.QueryRow(ctx, "SELECT count() FROM source")
require.NoError(t, row.Err())
var count uint64
require.NoError(t, row.Scan(&count))
assert.Equal(t, uint64(1000000), count)
assert.Equal(t, uint64(999), blocksRead.Load())
}

0 comments on commit 9491310

Please sign in to comment.