From 6a19102dbe30c1a329baf8ad8c6bab205c9a93cb Mon Sep 17 00:00:00 2001 From: Cocoa Date: Mon, 3 Jun 2024 21:55:03 +0100 Subject: [PATCH] feat(go/adbc/driver): clickhouse first poc --- .../driver/clickhouse/clickhouse_database.go | 120 ++ go/adbc/driver/clickhouse/connection.go | 340 +++ go/adbc/driver/clickhouse/driver.go | 88 + go/adbc/driver/clickhouse/record_reader.go | 506 +++++ go/adbc/driver/clickhouse/statement.go | 266 +++ go/adbc/go.mod | 10 + go/adbc/go.sum | 76 +- go/adbc/pkg/Makefile | 2 + go/adbc/pkg/clickhouse/driver.go | 1899 +++++++++++++++++ go/adbc/pkg/clickhouse/utils.c | 445 ++++ go/adbc/pkg/clickhouse/utils.h | 183 ++ 11 files changed, 3933 insertions(+), 2 deletions(-) create mode 100644 go/adbc/driver/clickhouse/clickhouse_database.go create mode 100644 go/adbc/driver/clickhouse/connection.go create mode 100644 go/adbc/driver/clickhouse/driver.go create mode 100644 go/adbc/driver/clickhouse/record_reader.go create mode 100644 go/adbc/driver/clickhouse/statement.go create mode 100644 go/adbc/pkg/clickhouse/driver.go create mode 100644 go/adbc/pkg/clickhouse/utils.c create mode 100644 go/adbc/pkg/clickhouse/utils.h diff --git a/go/adbc/driver/clickhouse/clickhouse_database.go b/go/adbc/driver/clickhouse/clickhouse_database.go new file mode 100644 index 0000000000..3b4693869a --- /dev/null +++ b/go/adbc/driver/clickhouse/clickhouse_database.go @@ -0,0 +1,120 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF 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 clickhouse + +import ( + "context" + "github.com/ClickHouse/clickhouse-go/v2" + "github.com/apache/arrow-adbc/go/adbc" + "github.com/apache/arrow-adbc/go/adbc/driver/internal/driverbase" + "strings" +) + +type databaseImpl struct { + driverbase.DatabaseImplBase + + address []string + protocol clickhouse.Protocol + database string + username string + password string +} + +func (d *databaseImpl) Open(ctx context.Context) (adbc.Connection, error) { + conn := &connectionImpl{ + ConnectionImplBase: driverbase.NewConnectionImplBase(&d.DatabaseImplBase), + address: d.address, + protocol: d.protocol, + database: d.database, + username: d.username, + password: d.password, + resultRecordBufferSize: defaultQueryResultBufferSize, + prefetchConcurrency: defaultQueryPrefetchConcurrency, + } + + err := conn.newConnection(ctx) + if err != nil { + return nil, err + } + + return driverbase.NewConnectionBuilder(conn). + WithAutocommitSetter(conn). + WithCurrentNamespacer(conn). + WithTableTypeLister(conn). + Connection(), nil +} + +func (d *databaseImpl) Close() error { return nil } + +func (d *databaseImpl) GetOption(key string) (string, error) { + switch key { + case OptionStringAddress: + return strings.Join(d.address, ","), nil + case OptionStringProtocol: + switch d.protocol { + case clickhouse.Native: + return OptionValueProtocolNative, nil + case clickhouse.HTTP: + return OptionValueProtocolHTTP, nil + } + case OptionStringDataset: + return d.database, nil + case OptionStringUsername: + return d.username, nil + case OptionStringPassword: + return d.password, nil + } + return d.DatabaseImplBase.GetOption(key) +} + +func (d *databaseImpl) SetOptions(options map[string]string) error { + for k, v := range options { + err := d.SetOption(k, v) + if err != nil { + return err + } + } + return nil +} + +func (d *databaseImpl) SetOption(key string, value string) error { + switch key { + case OptionStringAddress: + addresses := strings.Split(value, ",") + d.address = make([]string, len(addresses)) + for index, address := range addresses { + d.address[index] = strings.TrimSpace(address) + } + case OptionStringProtocol: + switch strings.ToLower(value) { + case OptionValueProtocolNative: + d.protocol = clickhouse.Native + case OptionValueProtocolHTTP: + d.protocol = clickhouse.HTTP + } + case OptionStringDataset: + d.database = value + case OptionStringUsername: + d.username = value + case OptionStringPassword: + d.password = value + default: + return d.DatabaseImplBase.SetOption(key, value) + } + return nil +} diff --git a/go/adbc/driver/clickhouse/connection.go b/go/adbc/driver/clickhouse/connection.go new file mode 100644 index 0000000000..a3b8c1bd35 --- /dev/null +++ b/go/adbc/driver/clickhouse/connection.go @@ -0,0 +1,340 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF 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 clickhouse + +import ( + "context" + "crypto/tls" + "fmt" + "github.com/ClickHouse/clickhouse-go/v2" + clickhouseDriver "github.com/ClickHouse/clickhouse-go/v2/lib/driver" + "github.com/apache/arrow-adbc/go/adbc" + "github.com/apache/arrow-adbc/go/adbc/driver/internal/driverbase" + "github.com/apache/arrow/go/v17/arrow" + "strings" +) + +type connectionImpl struct { + driverbase.ConnectionImplBase + + address []string + protocol clickhouse.Protocol + database string + username string + password string + table string + + resultRecordBufferSize int + prefetchConcurrency int + + conn clickhouseDriver.Conn +} + +// GetCurrentCatalog implements driverbase.CurrentNamespacer. +func (c *connectionImpl) GetCurrentCatalog() (string, error) { + return c.database, nil +} + +// GetCurrentDbSchema implements driverbase.CurrentNamespacer. +func (c *connectionImpl) GetCurrentDbSchema() (string, error) { + return c.table, nil +} + +// SetCurrentCatalog implements driverbase.CurrentNamespacer. +func (c *connectionImpl) SetCurrentCatalog(value string) error { + c.database = value + return nil +} + +// SetCurrentDbSchema implements driverbase.CurrentNamespacer. +func (c *connectionImpl) SetCurrentDbSchema(value string) error { + c.table = value + return nil +} + +// ListTableTypes implements driverbase.TableTypeLister. +func (c *connectionImpl) ListTableTypes(ctx context.Context) ([]string, error) { + // todo: find clickhouse enums/get from connection for this + return []string{"BASE TABLE", "TEMPORARY TABLE", "VIEW"}, nil +} + +// SetAutocommit implements driverbase.AutocommitSetter. +func (c *connectionImpl) SetAutocommit(enabled bool) error { + if enabled { + return nil + } + return adbc.Error{ + Code: adbc.StatusNotImplemented, + Msg: "SetAutocommit to `false` is not yet implemented", + } +} + +// Commit commits any pending transactions on this connection, it should +// only be used if autocommit is disabled. +// +// Behavior is undefined if this is mixed with SQL transaction statements. +func (c *connectionImpl) Commit(_ context.Context) error { + return adbc.Error{ + Code: adbc.StatusNotImplemented, + Msg: "Commit not yet implemented for BigQuery driver", + } +} + +// Rollback rolls back any pending transactions. Only used if autocommit +// is disabled. +// +// Behavior is undefined if this is mixed with SQL transaction statements. +func (c *connectionImpl) Rollback(_ context.Context) error { + return adbc.Error{ + Code: adbc.StatusNotImplemented, + Msg: "Rollback not yet implemented for BigQuery driver", + } +} + +// Close closes this connection and releases any associated resources. +func (c *connectionImpl) Close() error { + return c.conn.Close() +} + +func (c *connectionImpl) newConnection(ctx context.Context) error { + var err error + if c.address == nil || len(c.address) == 0 { + return adbc.Error{ + Code: adbc.StatusInvalidArgument, + Msg: "clickhouse address not given", + } + } + connectionOptions := clickhouse.Options{ + Addr: c.address, + Protocol: c.protocol, + Auth: clickhouse.Auth{ + Database: c.database, + Username: c.username, + Password: c.password, + }, + Compression: &clickhouse.Compression{ + Method: clickhouse.CompressionLZ4, + }, + Debugf: func(format string, v ...interface{}) { + fmt.Printf(format, v) + }, + TLS: &tls.Config{InsecureSkipVerify: true}, + } + c.conn, err = clickhouse.Open(&connectionOptions) + if err != nil { + return adbc.Error{ + Code: adbc.StatusInternal, + Msg: err.Error(), + } + } + if err := c.conn.Ping(ctx); err != nil { + if exception, ok := err.(*clickhouse.Exception); ok { + return adbc.Error{ + Code: adbc.StatusInternal, + Msg: fmt.Sprintf("Exception [%d] %s \n%s\n", exception.Code, exception.Message, exception.StackTrace), + } + } + return adbc.Error{ + Code: adbc.StatusInternal, + Msg: err.Error(), + } + } + return err +} + +// Metadata methods +// Generally these methods return an array.RecordReader that +// can be consumed to retrieve metadata about the database as Arrow +// data. The returned metadata has an expected schema given in the +// doc strings of the specific methods. Schema fields are nullable +// unless otherwise marked. While no Statement is used in these +// methods, the result set may count as an active statement to the +// driver for the purposes of concurrency management (e.g. if the +// driver has a limit on concurrent active statements and it must +// execute a SQL query internally in order to implement the metadata +// method). +// +// Some methods accept "search pattern" arguments, which are strings +// that can contain the special character "%" to match zero or more +// characters, or "_" to match exactly one character. (See the +// documentation of DatabaseMetaData in JDBC or "Pattern Value Arguments" +// in the ODBC documentation.) Escaping is not currently supported. +// GetObjects gets a hierarchical view of all catalogs, database schemas, +// tables, and columns. +// +// The result is an Arrow Dataset with the following schema: +// +// Field Name | Field Type +// ----------------------------|---------------------------- +// catalog_name | utf8 +// catalog_db_schemas | list +// +// DB_SCHEMA_SCHEMA is a Struct with the fields: +// +// Field Name | Field Type +// ----------------------------|---------------------------- +// db_schema_name | utf8 +// db_schema_tables | list +// +// TABLE_SCHEMA is a Struct with the fields: +// +// Field Name | Field Type +// ----------------------------|---------------------------- +// table_name | utf8 not null +// table_type | utf8 not null +// table_columns | list +// table_constraints | list +// +// COLUMN_SCHEMA is a Struct with the fields: +// +// Field Name | Field Type | Comments +// ----------------------------|---------------------|--------- +// column_name | utf8 not null | +// ordinal_position | int32 | (1) +// remarks | utf8 | (2) +// xdbc_data_type | int16 | (3) +// xdbc_type_name | utf8 | (3) +// xdbc_column_size | int32 | (3) +// xdbc_decimal_digits | int16 | (3) +// xdbc_num_prec_radix | int16 | (3) +// xdbc_nullable | int16 | (3) +// xdbc_column_def | utf8 | (3) +// xdbc_sql_data_type | int16 | (3) +// xdbc_datetime_sub | int16 | (3) +// xdbc_char_octet_length | int32 | (3) +// xdbc_is_nullable | utf8 | (3) +// xdbc_scope_catalog | utf8 | (3) +// xdbc_scope_schema | utf8 | (3) +// xdbc_scope_table | utf8 | (3) +// xdbc_is_autoincrement | bool | (3) +// xdbc_is_generatedcolumn | utf8 | (3) +// +// 1. The column's ordinal position in the table (starting from 1). +// 2. Database-specific description of the column. +// 3. Optional Value. Should be null if not supported by the driver. +// xdbc_values are meant to provide JDBC/ODBC-compatible metadata +// in an agnostic manner. +// +// CONSTRAINT_SCHEMA is a Struct with the fields: +// +// Field Name | Field Type | Comments +// ----------------------------|---------------------|--------- +// constraint_name | utf8 | +// constraint_type | utf8 not null | (1) +// constraint_column_names | list not null | (2) +// constraint_column_usage | list | (3) +// +// 1. One of 'CHECK', 'FOREIGN KEY', 'PRIMARY KEY', or 'UNIQUE'. +// 2. The columns on the current table that are constrained, in order. +// 3. For FOREIGN KEY only, the referenced table and columns. +// +// USAGE_SCHEMA is a Struct with fields: +// +// Field Name | Field Type +// ----------------------------|---------------------------- +// fk_catalog | utf8 +// fk_db_schema | utf8 +// fk_table | utf8 not null +// fk_column_name | utf8 not null +// +// For the parameters: If nil is passed, then that parameter will not +// be filtered by at all. If an empty string, then only objects without +// that property (ie: catalog or db schema) will be returned. +// +// tableName and columnName must be either nil (do not filter by +// table name or column name) or non-empty. +// +// All non-empty, non-nil strings should be a search pattern (as described +// earlier). + +func (c *connectionImpl) GetTableSchema(ctx context.Context, catalog *string, dbSchema *string, tableName string) (*arrow.Schema, error) { + return c.getTableSchemaWithFilter(ctx, catalog, dbSchema, tableName, nil) +} + +// NewStatement initializes a new statement object tied to this connection +func (c *connectionImpl) NewStatement() (adbc.Statement, error) { + return &statement{ + connectionImpl: c, + resultRecordBufferSize: c.resultRecordBufferSize, + prefetchConcurrency: c.prefetchConcurrency, + }, nil +} + +func (c *connectionImpl) GetOption(key string) (string, error) { + switch key { + case OptionStringProtocol: + switch c.protocol { + case clickhouse.HTTP: + return OptionValueProtocolHTTP, nil + case clickhouse.Native: + return OptionValueProtocolNative, nil + } + default: + return c.ConnectionImplBase.GetOption(key) + } + return "", nil +} + +func (c *connectionImpl) SetOption(key string, value string) error { + switch key { + case OptionStringProtocol: + switch strings.ToLower(value) { + case OptionValueProtocolNative: + c.protocol = clickhouse.Native + case OptionValueProtocolHTTP: + c.protocol = clickhouse.HTTP + } + default: + return adbc.Error{ + Code: adbc.StatusInvalidArgument, + Msg: fmt.Sprintf("unknown statement string type option `%s`", key), + } + } + return nil +} + +func (c *connectionImpl) GetOptionInt(key string) (int64, error) { + switch key { + case OptionIntQueryResultBufferSize: + return int64(c.resultRecordBufferSize), nil + case OptionIntQueryPrefetchConcurrency: + return int64(c.prefetchConcurrency), nil + default: + return c.ConnectionImplBase.GetOptionInt(key) + } +} + +func (c *connectionImpl) SetOptionInt(key string, value int64) error { + switch key { + case OptionIntQueryResultBufferSize: + c.resultRecordBufferSize = int(value) + return nil + case OptionIntQueryPrefetchConcurrency: + c.prefetchConcurrency = int(value) + return nil + default: + return c.ConnectionImplBase.SetOptionInt(key, value) + } +} + +func (c *connectionImpl) getTableSchemaWithFilter(ctx context.Context, catalog *string, dbSchema *string, tableName string, columnName *string) (*arrow.Schema, error) { + return nil, adbc.Error{ + Code: adbc.StatusNotImplemented, + Msg: "getTableSchemaWithFilter is not yet implemented", + } +} diff --git a/go/adbc/driver/clickhouse/driver.go b/go/adbc/driver/clickhouse/driver.go new file mode 100644 index 0000000000..21d9f42437 --- /dev/null +++ b/go/adbc/driver/clickhouse/driver.go @@ -0,0 +1,88 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF 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 clickhouse + +import ( + "github.com/ClickHouse/clickhouse-go/v2" + "github.com/apache/arrow-adbc/go/adbc" + "github.com/apache/arrow-adbc/go/adbc/driver/internal/driverbase" + "github.com/apache/arrow/go/v17/arrow/memory" + "runtime/debug" +) + +const ( + // OptionStringAddress comma-separated addresses + OptionStringAddress = "adbc.clickhouse.address" + OptionStringProtocol = "adbc.clickhouse.protocol" + OptionStringDataset = "adbc.clickhouse.sql.database" + OptionStringUsername = "adbc.clickhouse.sql.username" + OptionStringPassword = "adbc.clickhouse.sql.password" + + OptionValueProtocolHTTP = "http" + OptionValueProtocolNative = "native" + + OptionIntQueryResultBufferSize = "adbc.clickhouse.sql.query.result_buffer_size" + OptionIntQueryPrefetchConcurrency = "adbc.clickhouse.sql.query.prefetch_concurrency" + + defaultQueryResultBufferSize = 200 + defaultQueryPrefetchConcurrency = 10 +) + +var ( + infoVendorVersion string +) + +func init() { + if info, ok := debug.ReadBuildInfo(); ok { + for _, dep := range info.Deps { + switch { + case dep.Path == "github.com/ClickHouse/clickhouse-go/v2": + infoVendorVersion = dep.Version + } + } + } +} + +type driverImpl struct { + driverbase.DriverImplBase +} + +// NewDriver creates a new BigQuery driver using the given Arrow allocator. +func NewDriver(alloc memory.Allocator) adbc.Driver { + info := driverbase.DefaultDriverInfo("ClickHouse") + if infoVendorVersion != "" { + if err := info.RegisterInfoCode(adbc.InfoVendorVersion, infoVendorVersion); err != nil { + panic(err) + } + } + return driverbase.NewDriver(&driverImpl{ + DriverImplBase: driverbase.NewDriverImplBase(info, alloc), + }) +} + +func (d *driverImpl) NewDatabase(opts map[string]string) (adbc.Database, error) { + db := &databaseImpl{ + DatabaseImplBase: driverbase.NewDatabaseImplBase(&d.DriverImplBase), + protocol: clickhouse.Native, + } + if err := db.SetOptions(opts); err != nil { + return nil, err + } + + return driverbase.NewDatabase(db), nil +} diff --git a/go/adbc/driver/clickhouse/record_reader.go b/go/adbc/driver/clickhouse/record_reader.go new file mode 100644 index 0000000000..30bc4a95d7 --- /dev/null +++ b/go/adbc/driver/clickhouse/record_reader.go @@ -0,0 +1,506 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF 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 clickhouse + +import ( + "context" + "fmt" + clickhouseDriver "github.com/ClickHouse/clickhouse-go/v2/lib/driver" + "github.com/apache/arrow-adbc/go/adbc" + "github.com/apache/arrow/go/v17/arrow" + "github.com/apache/arrow/go/v17/arrow/array" + "github.com/apache/arrow/go/v17/arrow/memory" + "golang.org/x/sync/errgroup" + "io" + "reflect" + "strings" + "sync/atomic" +) + +type reader struct { + refCount int64 + schema *arrow.Schema + chs []chan arrow.Record + curChIndex int + rec arrow.Record + err error + + cancelFn context.CancelFunc +} + +func checkContext(ctx context.Context, maybeErr error) error { + if maybeErr != nil { + return maybeErr + } else if ctx.Err() == context.Canceled { + return adbc.Error{Msg: ctx.Err().Error(), Code: adbc.StatusCancelled} + } else if ctx.Err() == context.DeadlineExceeded { + return adbc.Error{Msg: ctx.Err().Error(), Code: adbc.StatusTimeout} + } + return ctx.Err() +} + +func appendNullableBool(fieldBuilder array.Builder, v **bool) { + currentValues := *v + b := fieldBuilder.(*array.BooleanBuilder) + if currentValues == nil { + b.AppendNull() + } else { + b.Append(*currentValues) + } +} + +func appendNullableInt8(fieldBuilder array.Builder, v **int8) { + currentValues := *v + b := fieldBuilder.(*array.Int8Builder) + if currentValues == nil { + b.AppendNull() + } else { + b.Append(*currentValues) + } +} + +func appendNullableUint8(fieldBuilder array.Builder, v **uint8) { + currentValues := *v + b := fieldBuilder.(*array.Uint8Builder) + if currentValues == nil { + b.AppendNull() + } else { + b.Append(*currentValues) + } +} + +func appendNullableInt16(fieldBuilder array.Builder, v **int16) { + currentValues := *v + b := fieldBuilder.(*array.Int16Builder) + if currentValues == nil { + b.AppendNull() + } else { + b.Append(*currentValues) + } +} + +func appendNullableUint16(fieldBuilder array.Builder, v **uint16) { + currentValues := *v + b := fieldBuilder.(*array.Uint16Builder) + if currentValues == nil { + b.AppendNull() + } else { + b.Append(*currentValues) + } +} + +func appendNullableInt32(fieldBuilder array.Builder, v **int32) { + currentValues := *v + b := fieldBuilder.(*array.Int32Builder) + if currentValues == nil { + b.AppendNull() + } else { + b.Append(*currentValues) + } +} + +func appendNullableUint32(fieldBuilder array.Builder, v **uint32) { + currentValues := *v + b := fieldBuilder.(*array.Uint32Builder) + if currentValues == nil { + b.AppendNull() + } else { + b.Append(*currentValues) + } +} + +func appendNullableInt64(fieldBuilder array.Builder, v **int64) { + currentValues := *v + b := fieldBuilder.(*array.Int64Builder) + if currentValues == nil { + b.AppendNull() + } else { + b.Append(*currentValues) + } +} + +func appendNullableUint64(fieldBuilder array.Builder, v **uint64) { + currentValues := *v + b := fieldBuilder.(*array.Uint64Builder) + if currentValues == nil { + b.AppendNull() + } else { + b.Append(*currentValues) + } +} + +func appendNullableFloat32(fieldBuilder array.Builder, v **float32) { + currentValues := *v + b := fieldBuilder.(*array.Float32Builder) + if currentValues == nil { + b.AppendNull() + } else { + b.Append(*currentValues) + } +} + +func appendFloat32(fieldBuilder array.Builder, v *float32) { + b := fieldBuilder.(*array.Float32Builder) + b.Append(*v) +} + +func appendNullableFloat64(fieldBuilder array.Builder, v **float64) { + currentValues := *v + b := fieldBuilder.(*array.Float64Builder) + if currentValues == nil { + b.AppendNull() + } else { + b.Append(*currentValues) + } +} + +func appendFloat64(fieldBuilder array.Builder, v *float64) { + b := fieldBuilder.(*array.Float64Builder) + b.Append(*v) +} + +func appendNullableString(fieldBuilder array.Builder, v **string) { + currentValues := *v + b := fieldBuilder.(*array.StringBuilder) + if currentValues == nil { + b.AppendNull() + } else { + b.Append(*currentValues) + } +} + +func appendString(fieldBuilder array.Builder, v *string) { + b := fieldBuilder.(*array.StringBuilder) + b.Append(*v) +} + +func sendBatch(schema *arrow.Schema, fieldBuilders []array.Builder, numRows int, ch chan arrow.Record) { + fieldValues := make([]arrow.Array, len(fieldBuilders)) + for i := range fieldBuilders { + fieldValues[i] = fieldBuilders[i].NewArray() + } + rec := array.NewRecord(schema, fieldValues, int64(numRows)) + ch <- rec +} + +// kicks off a goroutine for each endpoint and returns a reader which +// gathers all of the records as they come in. +func newRecordReader(ctx context.Context, parameters array.RecordReader, conn clickhouseDriver.Conn, query string, alloc memory.Allocator, resultRecordBufferSize, prefetchConcurrency int) (rdr *reader, totalRows int64, err error) { + rows, err := conn.Query(ctx, query) + if err != nil { + return nil, 0, err + } + + // todo: handle bulk queries + numChannels := 1 + chs := make([]chan arrow.Record, numChannels) + ch := make(chan arrow.Record, resultRecordBufferSize) + chs[0] = ch + group, ctx := errgroup.WithContext(ctx) + group.SetLimit(prefetchConcurrency) + ctx, cancelFn := context.WithCancel(ctx) + defer func() { + if err != nil { + close(ch) + cancelFn() + } + }() + + rdr = &reader{ + refCount: 1, + chs: chs, + err: nil, + cancelFn: cancelFn, + schema: nil, + } + + fields := make([]arrow.Field, 0) + columnTypes := rows.ColumnTypes() + for _, c := range columnTypes { + typeLen := len(c.DatabaseTypeName()) + field, err := buildSchemaField(c.Name(), c.DatabaseTypeName(), 0, typeLen) + if err != nil { + return nil, 0, err + } + fields = append(fields, field) + } + schema := arrow.NewSchema(fields, nil) + rdr.schema = schema + + group.Go(func() error { + var vars = make([]any, len(columnTypes)) + schemaFields := schema.Fields() + fieldBuilders := make([]array.Builder, len(columnTypes)) + for i := range columnTypes { + vars[i] = reflect.New(columnTypes[i].ScanType()).Interface() + schemaField := schemaFields[i] + fieldBuilders[i] = array.NewBuilder(alloc, schemaField.Type) + } + + rowsProcessed := 0 + for rows.Next() && ctx.Err() == nil { + if err := rows.Scan(vars...); err != nil { + return err + } + for i, v := range vars { + fieldBuilder := fieldBuilders[i] + if v == nil { + return adbc.Error{ + Code: adbc.StatusInternal, + } + } + + switch v := v.(type) { + case **bool: + appendNullableBool(fieldBuilder, v) + case **int8: + appendNullableInt8(fieldBuilder, v) + case **uint8: + appendNullableUint8(fieldBuilder, v) + case **int16: + appendNullableInt16(fieldBuilder, v) + case **uint16: + appendNullableUint16(fieldBuilder, v) + case **int32: + appendNullableInt32(fieldBuilder, v) + case **uint32: + appendNullableUint32(fieldBuilder, v) + case **int64: + appendNullableInt64(fieldBuilder, v) + case **uint64: + appendNullableUint64(fieldBuilder, v) + case **float32: + appendNullableFloat32(fieldBuilder, v) + case *float32: + appendFloat32(fieldBuilder, v) + case **float64: + appendNullableFloat64(fieldBuilder, v) + case *float64: + appendFloat64(fieldBuilder, v) + case **string: + appendNullableString(fieldBuilder, v) + case *string: + appendString(fieldBuilder, v) + default: + return adbc.Error{ + Code: adbc.StatusNotImplemented, + Msg: fmt.Sprintf("type `%T` is not supported yet", v), + } + } + } + rowsProcessed++ + if rowsProcessed == resultRecordBufferSize { + sendBatch(schema, fieldBuilders, rowsProcessed, ch) + rowsProcessed = 0 + } + } + if rowsProcessed > 0 { + sendBatch(schema, fieldBuilders, rowsProcessed, ch) + } + if err = rows.Close(); err != nil { + return err + } + if err = rows.Err(); err != nil { + return err + } + return checkContext(ctx, rdr.Err()) + }) + + lastChannelIndex := len(chs) - 1 + go func() { + // place this here so that we always clean up, but they can't be in a + // separate goroutine. Otherwise we'll have a race condition between + // the call to wait and the calls to group.Go to kick off the jobs + // to perform the pre-fetching (GH-1283). + rdr.err = group.Wait() + // don't close the last channel until after the group is finished, + // so that Next() can only return after reader.err may have been set + close(chs[lastChannelIndex]) + }() + + return rdr, totalRows, nil +} + +func (r *reader) Retain() { + atomic.AddInt64(&r.refCount, 1) +} + +func (r *reader) Release() { + if atomic.AddInt64(&r.refCount, -1) == 0 { + if r.rec != nil { + r.rec.Release() + } + r.cancelFn() + for _, ch := range r.chs { + for rec := range ch { + rec.Release() + } + } + } +} + +func (r *reader) Err() error { + return r.err +} + +func (r *reader) Next() bool { + if r.rec != nil { + r.rec.Release() + r.rec = nil + } + + if r.curChIndex >= len(r.chs) { + return false + } + var ok bool + for r.curChIndex < len(r.chs) { + if r.rec, ok = <-r.chs[r.curChIndex]; ok { + break + } + r.curChIndex++ + } + return r.rec != nil +} + +func (r *reader) Schema() *arrow.Schema { + return r.schema +} + +func (r *reader) Record() arrow.Record { + return r.rec +} + +func parseType(typeString string, typeIndex, typeLen int) (string, int, error) { + typeEnd := -2 + for i := typeIndex; i < typeLen; i++ { + c := typeString[i] + if (c >= '0' && c <= '9') || (c >= 'a' && c <= 'z') || (c >= 'A' && c <= 'Z') || c == '_' { + typeEnd = i + } else { + break + } + } + typeEnd += 1 + if typeEnd == -1 { + return "", 0, adbc.Error{ + Code: adbc.StatusInternal, + Msg: fmt.Sprintf("unknown type `%s`", typeString), + } + } + + return typeString[typeIndex:typeEnd], typeEnd, nil +} + +func expectRightBracket(typeString string, typeIndex, typeLen int) (int, error) { + if typeIndex == typeLen { + return 0, io.EOF + } + + for i := typeIndex; i < typeLen; i++ { + if typeString[i] == ')' { + return i + 1, nil + } else if typeString[i] == ' ' || typeString[i] == '\t' || typeString[i] == '\n' { + continue + } else { + return 0, adbc.Error{ + Code: adbc.StatusInvalidArgument, + Msg: fmt.Sprintf("expecting a `)` but got: `%c` at %d", typeString[i], i), + } + } + } + return typeLen, io.EOF +} + +var ( + simpleDataType = map[string]arrow.DataType{ + "Int8": arrow.PrimitiveTypes.Int8, + "UInt8": arrow.PrimitiveTypes.Uint8, + "Int16": arrow.PrimitiveTypes.Int16, + "UInt16": arrow.PrimitiveTypes.Uint16, + "Int32": arrow.PrimitiveTypes.Int32, + "UInt32": arrow.PrimitiveTypes.Uint32, + "Int64": arrow.PrimitiveTypes.Int64, + "UInt64": arrow.PrimitiveTypes.Uint64, + "Float32": arrow.PrimitiveTypes.Float32, + "Float64": arrow.PrimitiveTypes.Float64, + "String": arrow.BinaryTypes.String, + } +) + +func buildField(name, typeName string) (arrow.Field, error) { + dataType, ok := simpleDataType[typeName] + if !ok { + return arrow.Field{}, adbc.Error{ + Code: adbc.StatusNotImplemented, + Msg: fmt.Sprintf("unsupported type: `%s`", typeName), + } + } + + field := arrow.Field{ + Name: name, + Type: dataType, + Nullable: false, + } + return field, nil +} + +func ensureConsumedTypeString(typeString string, typeIndex, typeLen int) error { + if typeIndex == typeLen { + return nil + } + for i := typeIndex; i < typeLen; i++ { + if typeString[i] == ' ' || typeString[i] == '\t' || typeString[i] == '\n' { + continue + } else { + return adbc.Error{ + Code: adbc.StatusInternal, + Msg: fmt.Sprintf("cannot fully consume the type string `%s`, parsed up to index %d", typeString, i), + } + } + } + return nil +} + +func buildSchemaField(name string, typeString string, typeIndex, typeLen int) (arrow.Field, error) { + if strings.HasPrefix(typeString, "Nullable(") { + typeName, typeIndex, err := parseType(typeString, typeIndex+9, typeLen) + if err != nil { + return arrow.Field{}, err + } + typeIndex, err = expectRightBracket(typeString, typeIndex, typeLen) + if err != nil { + return arrow.Field{}, err + } + if err = ensureConsumedTypeString(typeString, typeIndex, typeLen); err != nil { + return arrow.Field{}, err + } + + field, err := buildField(name, typeName) + field.Nullable = true + return field, err + } else { + typeName, typeIndex, err := parseType(typeString, typeIndex, typeLen) + if err != nil { + return arrow.Field{}, err + } + if err = ensureConsumedTypeString(typeString, typeIndex, typeLen); err != nil { + return arrow.Field{}, err + } + return buildField(name, typeName) + } +} diff --git a/go/adbc/driver/clickhouse/statement.go b/go/adbc/driver/clickhouse/statement.go new file mode 100644 index 0000000000..6336c19331 --- /dev/null +++ b/go/adbc/driver/clickhouse/statement.go @@ -0,0 +1,266 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF 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 clickhouse + +import ( + "context" + "fmt" + "github.com/apache/arrow-adbc/go/adbc" + "github.com/apache/arrow/go/v17/arrow" + "github.com/apache/arrow/go/v17/arrow/array" +) + +type statement struct { + connectionImpl *connectionImpl + query string + paramBinding arrow.Record + streamBinding array.RecordReader + resultRecordBufferSize int + prefetchConcurrency int +} + +// Close releases any relevant resources associated with this statement +// and closes it (particularly if it is a prepared statement). +// +// A statement instance should not be used after Close is called. +func (st *statement) Close() error { + st.clearParameters() + return nil +} + +func (st *statement) GetOption(key string) (string, error) { + val, err := st.connectionImpl.GetOption(key) + if err == nil { + return val, nil + } + return "", err +} + +func (st *statement) GetOptionInt(key string) (int64, error) { + switch key { + case OptionIntQueryResultBufferSize: + return int64(st.resultRecordBufferSize), nil + case OptionIntQueryPrefetchConcurrency: + return int64(st.prefetchConcurrency), nil + default: + val, err := st.connectionImpl.GetOptionInt(key) + if err == nil { + return val, nil + } + return 0, err + } +} + +func (st *statement) SetOption(key string, v string) error { + return adbc.Error{ + Code: adbc.StatusInvalidArgument, + Msg: fmt.Sprintf("unknown statement string type option `%s`", key), + } +} + +func (st *statement) SetOptionInt(key string, value int64) error { + switch key { + case OptionIntQueryResultBufferSize: + st.resultRecordBufferSize = int(value) + return nil + case OptionIntQueryPrefetchConcurrency: + st.prefetchConcurrency = int(value) + return nil + default: + return adbc.Error{ + Code: adbc.StatusInvalidArgument, + Msg: fmt.Sprintf("unknown statement string type option `%s`", key), + } + } +} + +// SetSqlQuery sets the query string to be executed. +// +// The query can then be executed with any of the Execute methods. +// For queries expected to be executed repeatedly, Prepare should be +// called before execution. +func (st *statement) SetSqlQuery(query string) error { + st.query = query + return nil +} + +// ExecuteQuery executes the current query or prepared statement +// and returns a RecordReader for the results along with the number +// of rows affected if known, otherwise it will be -1. +// +// This invalidates any prior result sets on this statement. +func (st *statement) ExecuteQuery(ctx context.Context) (array.RecordReader, int64, error) { + parameters, err := st.getBoundParameterReader() + if err != nil { + return nil, -1, err + } + + return newRecordReader(ctx, parameters, st.connectionImpl.conn, st.query, st.connectionImpl.Alloc, st.resultRecordBufferSize, st.prefetchConcurrency) +} + +// ExecuteUpdate executes a statement that does not generate a result +// set. It returns the number of rows affected if known, otherwise -1. +func (st *statement) ExecuteUpdate(ctx context.Context) (int64, error) { + return 0, adbc.Error{ + Code: adbc.StatusNotImplemented, + Msg: "ExecuteUpdate not yet implemented", + } +} + +// ExecuteSchema gets the schema of the result set of a query without executing it. +func (st *statement) ExecuteSchema(ctx context.Context) (*arrow.Schema, error) { + return nil, adbc.Error{ + Code: adbc.StatusNotImplemented, + Msg: "ExecuteSchema not yet implemented", + } +} + +// Prepare turns this statement into a prepared statement to be executed +// multiple times. This invalidates any prior result sets. +func (st *statement) Prepare(_ context.Context) error { + return adbc.Error{ + Code: adbc.StatusNotImplemented, + Msg: "Prepare not yet implemented", + } +} + +// SetSubstraitPlan allows setting a serialized Substrait execution +// plan into the query or for querying Substrait-related metadata. +// +// Drivers are not required to support both SQL and Substrait semantics. +// If they do, it may be via converting between representations internally. +// +// Like SetSqlQuery, after this is called the query can be executed +// using any of the Execute methods. If the query is expected to be +// executed repeatedly, Prepare should be called first on the statement. +func (st *statement) SetSubstraitPlan(plan []byte) error { + return adbc.Error{ + Code: adbc.StatusNotImplemented, + Msg: "SetSubstraitPlan not yet implemented for BigQuery driver", + } +} + +func (st *statement) getBoundParameterReader() (array.RecordReader, error) { + if st.paramBinding != nil { + return array.NewRecordReader(st.paramBinding.Schema(), []arrow.Record{st.paramBinding}) + } else if st.streamBinding != nil { + return st.streamBinding, nil + } else { + return nil, nil + } +} + +func (st *statement) clearParameters() { + if st.paramBinding != nil { + st.paramBinding.Release() + st.paramBinding = nil + } + if st.streamBinding != nil { + st.streamBinding.Release() + st.streamBinding = nil + } +} + +// SetParameters takes a record batch to send as the parameter bindings when +// executing. It should match the schema from ParameterSchema. +// +// This will call Retain on the record to ensure it doesn't get released out +// from under the statement. Release will be called on a previous binding +// record or reader if it existed, and will be called upon calling Close on the +// PreparedStatement. +func (st *statement) SetParameters(binding arrow.Record) { + st.clearParameters() + st.paramBinding = binding + if st.paramBinding != nil { + st.paramBinding.Retain() + } +} + +// SetRecordReader takes a RecordReader to send as the parameter bindings when +// executing. It should match the schema from ParameterSchema. +// +// This will call Retain on the reader to ensure it doesn't get released out +// from under the statement. Release will be called on a previous binding +// record or reader if it existed, and will be called upon calling Close on the +// PreparedStatement. +func (st *statement) SetRecordReader(binding array.RecordReader) { + st.clearParameters() + st.streamBinding = binding + st.streamBinding.Retain() +} + +// Bind uses an arrow record batch to bind parameters to the query. +// +// This can be used for bulk inserts or for prepared statements. +// The driver will call release on the passed in Record when it is done, +// but it may not do this until the statement is closed or another +// record is bound. +func (st *statement) Bind(_ context.Context, values arrow.Record) error { + st.SetParameters(values) + return nil +} + +// BindStream uses a record batch stream to bind parameters for this +// query. This can be used for bulk inserts or prepared statements. +// +// The driver will call Release on the record reader, but may not do this +// until Close is called. +func (st *statement) BindStream(_ context.Context, stream array.RecordReader) error { + st.SetRecordReader(stream) + return nil +} + +// GetParameterSchema returns an Arrow schema representation of +// the expected parameters to be bound. +// +// This retrieves an Arrow Schema describing the number, names, and +// types of the parameters in a parameterized statement. The fields +// of the schema should be in order of the ordinal position of the +// parameters; named parameters should appear only once. +// +// If the parameter does not have a name, or a name cannot be determined, +// the name of the corresponding field in the schema will be an empty +// string. If the type cannot be determined, the type of the corresponding +// field will be NA (NullType). +// +// This should be called only after calling Prepare. +// +// This should return an error with StatusNotImplemented if the schema +// cannot be determined. +func (st *statement) GetParameterSchema() (*arrow.Schema, error) { + return nil, adbc.Error{ + Code: adbc.StatusNotImplemented, + Msg: "GetParameterSchema not yet implemented for BigQuery driver", + } +} + +// ExecutePartitions executes the current statement and gets the results +// as a partitioned result set. +// +// It returns the Schema of the result set, the collection of partition +// descriptors and the number of rows affected, if known. If unknown, +// the number of rows affected will be -1. +// +// If the driver does not support partitioned results, this will return +// an error with a StatusNotImplemented code. +func (st *statement) ExecutePartitions(ctx context.Context) (*arrow.Schema, adbc.Partitions, int64, error) { + return nil, adbc.Partitions{}, -1, adbc.Error{ + Code: adbc.StatusNotImplemented, + Msg: "ExecutePartitions not yet implemented for BigQuery driver", + } +} diff --git a/go/adbc/go.mod b/go/adbc/go.mod index 206d97e8fe..b5727599a3 100644 --- a/go/adbc/go.mod +++ b/go/adbc/go.mod @@ -20,6 +20,7 @@ module github.com/apache/arrow-adbc/go/adbc go 1.21 require ( + github.com/ClickHouse/clickhouse-go/v2 v2.25.0 github.com/apache/arrow/go/v17 v17.0.0-20240520131450-cc3e2db30094 github.com/bluele/gcache v0.0.2 github.com/golang/protobuf v1.5.4 @@ -40,6 +41,7 @@ require ( github.com/Azure/azure-sdk-for-go/sdk/azcore v1.9.2 // indirect github.com/Azure/azure-sdk-for-go/sdk/internal v1.5.2 // indirect github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.3.0 // indirect + github.com/ClickHouse/ch-go v0.61.5 // indirect github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c // indirect github.com/andybalholm/brotli v1.1.0 // indirect github.com/apache/arrow/go/v15 v15.0.0 // indirect @@ -62,6 +64,8 @@ require ( github.com/dustin/go-humanize v1.0.1 // indirect github.com/dvsekhvalnov/jose2go v1.6.0 // indirect github.com/gabriel-vasile/mimetype v1.4.3 // indirect + github.com/go-faster/city v1.0.1 // indirect + github.com/go-faster/errors v0.7.1 // indirect github.com/goccy/go-json v0.10.2 // indirect github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect github.com/golang-jwt/jwt/v5 v5.2.1 // indirect @@ -78,13 +82,19 @@ require ( github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 // indirect github.com/mtibben/percent v0.2.1 // indirect github.com/ncruces/go-strftime v0.1.9 // indirect + github.com/paulmach/orb v0.11.1 // indirect github.com/pierrec/lz4/v4 v4.1.21 // indirect github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c // indirect + github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec // indirect + github.com/segmentio/asm v1.2.0 // indirect + github.com/shopspring/decimal v1.4.0 // indirect github.com/sirupsen/logrus v1.9.3 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/zeebo/xxh3 v1.0.2 // indirect + go.opentelemetry.io/otel v1.26.0 // indirect + go.opentelemetry.io/otel/trace v1.26.0 // indirect golang.org/x/crypto v0.23.0 // indirect golang.org/x/mod v0.17.0 // indirect golang.org/x/net v0.25.0 // indirect diff --git a/go/adbc/go.sum b/go/adbc/go.sum index b466e403d5..027a1b18eb 100644 --- a/go/adbc/go.sum +++ b/go/adbc/go.sum @@ -14,6 +14,10 @@ github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.3.0 h1:IfFdxTUDiV58iZqPK github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.3.0/go.mod h1:SUZc9YRRHfx2+FAQKNDGrssXehqLpxmwRv2mC/5ntj4= github.com/AzureAD/microsoft-authentication-library-for-go v1.2.1 h1:DzHpqpoJVaCgOUdVHxE8QB52S6NiVdDQvGlny1qvPqA= github.com/AzureAD/microsoft-authentication-library-for-go v1.2.1/go.mod h1:wP83P5OoQ5p6ip3ScPr0BAq0BvuPAvacpEuSzyouqAI= +github.com/ClickHouse/ch-go v0.61.5 h1:zwR8QbYI0tsMiEcze/uIMK+Tz1D3XZXLdNrlaOpeEI4= +github.com/ClickHouse/ch-go v0.61.5/go.mod h1:s1LJW/F/LcFs5HJnuogFMta50kKDO0lf9zzfrbl0RQg= +github.com/ClickHouse/clickhouse-go/v2 v2.25.0 h1:rKscwqgQHzWBTZySZDcHKxgs0Ad+xFULfZvo26W5UlY= +github.com/ClickHouse/clickhouse-go/v2 v2.25.0/go.mod h1:iDTViXk2Fgvf1jn2dbJd1ys+fBkdD1UMRnXlwmhijhQ= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c h1:RGWPOewvKIROun94nF7v2cua9qP+thov/7M50KEoeSU= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c/go.mod h1:X0CRv0ky0k6m906ixxpzmDRLvX58TFUKS2eePweuyxk= github.com/andybalholm/brotli v1.1.0 h1:eLKJA0d02Lf0mVpIDgYnqXcUn0GqVmEFny3VuID1U3M= @@ -77,18 +81,27 @@ github.com/dvsekhvalnov/jose2go v1.6.0 h1:Y9gnSnP4qEI0+/uQkHvFXeD2PLPJeXEL+ySMEA github.com/dvsekhvalnov/jose2go v1.6.0/go.mod h1:QsHjhyTlD/lAVqn/NSbVZmSCGeDehTB/mPZadG+mhXU= github.com/gabriel-vasile/mimetype v1.4.3 h1:in2uUcidCuFcDKtdcBxlR0rJ1+fsokWf+uqxgUFjbI0= github.com/gabriel-vasile/mimetype v1.4.3/go.mod h1:d8uq/6HKRL6CGdk+aubisF/M5GcPfT7nKyLpA0lbSSk= +github.com/go-faster/city v1.0.1 h1:4WAxSZ3V2Ws4QRDrscLEDcibJY8uf41H6AhXDrNDcGw= +github.com/go-faster/city v1.0.1/go.mod h1:jKcUJId49qdW3L1qKHH/3wPeUstCVpVSXTM6vO3VcTw= +github.com/go-faster/errors v0.7.1 h1:MkJTnDoEdi9pDabt1dpWf7AA8/BaSYZqibYyhZ20AYg= +github.com/go-faster/errors v0.7.1/go.mod h1:5ySTjWFiphBs07IKuiL69nxdfd5+fzh1u7FPGZP2quo= github.com/goccy/go-json v0.10.2 h1:CrxCmQqYDkv1z7lO7Wbh2HN93uovUHgrECaO5ZrCXAU= github.com/goccy/go-json v0.10.2/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 h1:ZpnhV/YsD2/4cESfV5+Hoeu/iUR3ruzNvZ+yQfO03a0= github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2/go.mod h1:bBOAhwG1umN6/6ZUMtDFBMQR8jRg9O75tm9K00oMsK4= +github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/golang-jwt/jwt/v5 v5.2.1 h1:OuVbFODueb089Lh128TAcimifWaLhJwVflnrgM17wHk= github.com/golang-jwt/jwt/v5 v5.2.1/go.mod h1:pqrtFR0X4osieyHYxtmOUWsAWrfe1Q5UVIyoH402zdk= +github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= +github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/google/flatbuffers v24.3.25+incompatible h1:CX395cjN9Kke9mmalRoL3d81AtFUxJM+yDthflgJGkI= github.com/google/flatbuffers v24.3.25+incompatible/go.mod h1:1AeVuKshWv4vARoZatz6mlQ0JxURH0Kv5+zNeJKJCa8= +github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/pprof v0.0.0-20221118152302-e6195bd50e26 h1:Xim43kblpZXfIBQsbuBVKCudVG457BR2GZFIz3uw3hQ= @@ -103,12 +116,16 @@ github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9Y github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= github.com/jmespath/go-jmespath/internal/testify v1.5.1 h1:shLQSRRSCCPj3f2gpwzGwWFoC7ycTf1rcQZHOlsJ6N8= github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= +github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= +github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/asmfmt v1.3.2 h1:4Ri7ox3EwapiOjCki+hw14RyKk201CN4rzyCJRFLpK4= github.com/klauspost/asmfmt v1.3.2/go.mod h1:AG8TuvYojzulgDAMCnYn50l/5QV3Bs/tp6j0HLHbNSE= +github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/compress v1.17.8 h1:YcnTYrq7MikUT7k0Yb5eceMmALQPYBW/Xltxn0NAMnU= github.com/klauspost/compress v1.17.8/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= github.com/klauspost/cpuid/v2 v2.2.7 h1:ZWSB3igEs+d0qvnxR/ZBzXVmxkgt8DdzP6m9pfuVLDM= github.com/klauspost/cpuid/v2 v2.2.7/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= +github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= @@ -125,21 +142,31 @@ github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 h1:AMFGa4R4MiIpsp github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8/go.mod h1:mC1jAcsrzbxHt8iiaC+zU4b1ylILSosueou12R++wfY= github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 h1:+n/aFZefKZp7spd8DFdX7uMikMLXX4oubIzJF4kv/wI= github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3/go.mod h1:RagcQ7I8IeTMnF8JTXieKnO4Z6JCsikNEzj0DwauVzE= +github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/mtibben/percent v0.2.1 h1:5gssi8Nqo8QU/r2pynCm+hBQHpkB/uNK7BJCFogWdzs= github.com/mtibben/percent v0.2.1/go.mod h1:KG9uO+SZkUp+VkRHsCdYQV3XSZrrSpR3O9ibNBTZrns= github.com/ncruces/go-strftime v0.1.9 h1:bY0MQC28UADQmHmaF5dgpLmImcShSi2kHU9XLdhx/f4= github.com/ncruces/go-strftime v0.1.9/go.mod h1:Fwc5htZGVVkseilnfgOVb9mKy6w1naJmn9CehxcKcls= github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= +github.com/paulmach/orb v0.11.1 h1:3koVegMC4X/WeiXYz9iswopaTwMem53NzTJuTF20JzU= +github.com/paulmach/orb v0.11.1/go.mod h1:5mULz1xQfs3bmQm63QEJA6lNGujuRafwA5S/EnuLaLU= +github.com/paulmach/protoscan v0.2.1/go.mod h1:SpcSwydNLrxUGSDvXvO0P7g7AuhJ7lcKfDlhJCDw2gY= github.com/pierrec/lz4/v4 v4.1.21 h1:yOVMLb6qSIDP67pl/5F7RepeKYu/VmTyEXvuMI5d9mQ= github.com/pierrec/lz4/v4 v4.1.21/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c h1:+mdjkGKdHQG3305AYmdv1U2eRNDiU2ErMBj1gwrq8eQ= github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c/go.mod h1:7rwL4CYBLnjLxUqIJNnCWiEdr3bn6IUYi15bNlnbCCU= +github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= +github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec h1:W09IVJc94icq4NjY3clb7Lk8O1qJ8BdBEF8z0ibU0rE= github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= -github.com/rogpeppe/go-internal v1.9.0 h1:73kH8U+JUqXU8lRuOHeVHaa/SZPifC7BkcraZVejAe8= -github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs= +github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ= +github.com/rogpeppe/go-internal v1.10.0/go.mod h1:UQnix2H7Ngw/k4C5ijL5+65zddjncjaFoBhdsK/akog= +github.com/segmentio/asm v1.2.0 h1:9BQrFxC+YOHJlTlHGkTrFWf59nbL3XnCoFLTwDCI7ys= +github.com/segmentio/asm v1.2.0/go.mod h1:BqMnlJP91P8d+4ibuonYZw9mfnzI9HfxselHZr5aAcs= +github.com/shopspring/decimal v1.4.0 h1:bxl37RwXBklmTi0C79JfXCEBD1cqqHt0bbgBAGFp81k= +github.com/shopspring/decimal v1.4.0/go.mod h1:gawqmDU56v4yIKSwfBSFip1HdCCXN8/+DMd9qYNcwME= github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ= github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= github.com/snowflakedb/gosnowflake v1.10.0 h1:5hBGKa/jJEhciokzgJcz5xmLNlJ8oUm8vhfu5tg82tM= @@ -147,43 +174,85 @@ github.com/snowflakedb/gosnowflake v1.10.0/go.mod h1:WC4eGUOH3K9w3pLsdwZsdawIwtW github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY= github.com/stretchr/objx v0.5.2/go.mod h1:FRsXN1f5AsAjCGJKqEizvkpNtU+EGNCLh3NxZ/8L+MA= +github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= +github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= +github.com/xdg-go/scram v1.1.1/go.mod h1:RaEWvsqvNKKvBPvcKeFjrG2cJqOkHTiyTpzz23ni57g= +github.com/xdg-go/stringprep v1.0.3/go.mod h1:W3f5j4i+9rC0kuIEJL0ky1VpHXQU3ocBgklLGvcBnW8= +github.com/youmark/pkcs8 v0.0.0-20181117223130-1be2e3e5546d/go.mod h1:rHwXgn7JulP+udvsHwJoVG1YGAP6VLg4y9I5dyZdqmA= github.com/youmark/pkcs8 v0.0.0-20201027041543-1326539a0a0a h1:fZHgsYlfvtyqToslyjUt3VOPF4J7aK/3MPcK7xp3PDk= github.com/youmark/pkcs8 v0.0.0-20201027041543-1326539a0a0a/go.mod h1:ul22v+Nro/R083muKhosV54bj5niojjWZvU8xrevuH4= +github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/zeebo/assert v1.3.0 h1:g7C04CbJuIDKNPFHmsk4hwZDO5O+kntRxzaUoNXj+IQ= github.com/zeebo/assert v1.3.0/go.mod h1:Pq9JiuJQpG8JLJdtkwrJESF0Foym2/D9XMU5ciN/wJ0= github.com/zeebo/xxh3 v1.0.2 h1:xZmwmqxHZA8AI603jOQ0tMqmBr9lPeFwGg6d+xy9DC0= github.com/zeebo/xxh3 v1.0.2/go.mod h1:5NWz9Sef7zIDm2JHfFlcQvNekmcEl9ekUZQQKCYaDcA= +go.mongodb.org/mongo-driver v1.11.4/go.mod h1:PTSz5yu21bkT/wXpkS7WR5f0ddqw5quethTUn9WM+2g= +go.opentelemetry.io/otel v1.26.0 h1:LQwgL5s/1W7YiiRwxf03QGnWLb2HW4pLiAhaA5cZXBs= +go.opentelemetry.io/otel v1.26.0/go.mod h1:UmLkJHUAidDval2EICqBMbnAd0/m2vmpf/dAM+fvFs4= +go.opentelemetry.io/otel/trace v1.26.0 h1:1ieeAUb4y0TE26jUFrCIXKpTuVK7uJGN9/Z/2LP5sQA= +go.opentelemetry.io/otel/trace v1.26.0/go.mod h1:4iDxvGDQuUkHve82hJJ8UqrwswHYsZuWCBllGV2U2y0= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200302210943-78000ba7a073/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.23.0 h1:dIJU/v2J8Mdglj/8rJ6UUOM3Zc9zLZxVZwwxMooUSAI= golang.org/x/crypto v0.23.0/go.mod h1:CKFgDieR+mRhux2Lsu27y0fO304Db0wZe70UKqHu0v8= golang.org/x/exp v0.0.0-20240318143956-a85f2c67cd81 h1:6R2FC06FonbXQ8pK11/PDFY6N6LWlf9KlzibaCapmqc= golang.org/x/exp v0.0.0-20240318143956-a85f2c67cd81/go.mod h1:CQ1k9gNrJ50XIzaKCRR2hssIjF07kZFEiieALBM/ARQ= +golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.17.0 h1:zY54UmvipHiNd+pm+m0x9KhZ9hl1/7QNMyxXbc6ICqA= golang.org/x/mod v0.17.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.25.0 h1:d/OCCoBEUq33pjydKrGQhw7IlUPI2Oylr+8qLx49kac= golang.org/x/net v0.25.0/go.mod h1:JkAGAh7GEvH74S6FOH42FLoXpXbE/aqXSrIQjXgsiwM= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.20.0 h1:Od9JTbYCk261bKm4M/mw7AklTlFYIa0bIp9BgSm1S8Y= golang.org/x/sys v0.20.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.20.0 h1:VnkxpohqXaOBYJtBmEppKUG6mXpi+4O6purfc2+sMhw= golang.org/x/term v0.20.0/go.mod h1:8UkIAJTvZgivsXaD6/pH6U9ecQzZ45awqEOzuCvwpFY= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.15.0 h1:h1V/4gjBv8v9cjcR6+AR5+/cIYK5N/WAgiv4xlsEtAk= golang.org/x/text v0.15.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.21.0 h1:qc0xYgIbsSDt9EyWz05J5wfa7LOVW0YTLOXrqdLAWIw= golang.org/x/tools v0.21.0/go.mod h1:aiJjzUbINMkxbQROHiO6hDPo2LHcIPhhQsa9DLh0yGk= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 h1:+cNy6SZtPcJQH3LJVLOSmiC7MMxXNOb3PU/VUEz+EhU= golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028/go.mod h1:NDW/Ps6MPRej6fsCIbMTohpP40sJ/P/vI1MoTEGwX90= gonum.org/v1/gonum v0.15.0 h1:2lYxjRbTYyxkJxlhC+LvJIx3SsANPdRybu1tGj9/OrQ= @@ -192,9 +261,12 @@ google.golang.org/genproto/googleapis/rpc v0.0.0-20240227224415-6ceb2ff114de h1: google.golang.org/genproto/googleapis/rpc v0.0.0-20240227224415-6ceb2ff114de/go.mod h1:H4O17MA/PE9BsGx3w+a+W2VOLLD1Qf7oJneAoU6WktY= google.golang.org/grpc v1.63.2 h1:MUeiw1B2maTVZthpU5xvASfTh3LDbxHd6IJ6QQVU+xM= google.golang.org/grpc v1.63.2/go.mod h1:WAX/8DgncnokcFUldAxq7GeB5DXHDbMF+lLvDomNkRA= +google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= +google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.34.1 h1:9ddQBjfCyZPOHPUiPxpYESBLc+T8P3E+Vo4IbKZgFWg= google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= diff --git a/go/adbc/pkg/Makefile b/go/adbc/pkg/Makefile index b791c5b7da..73a84bb432 100644 --- a/go/adbc/pkg/Makefile +++ b/go/adbc/pkg/Makefile @@ -26,6 +26,7 @@ else endif DRIVERS := \ + libadbc_driver_clickhouse.$(SUFFIX) \ libadbc_driver_flightsql.$(SUFFIX) \ libadbc_driver_panicdummy.$(SUFFIX) \ libadbc_driver_snowflake.$(SUFFIX) @@ -38,6 +39,7 @@ libadbc_driver_%.$(SUFFIX): % $(RM) $(basename $@).h regenerate: + go run gen/main.go -prefix ClickHouse -o ./clickhouse/ -driver ../driver/clickhouse go run gen/main.go -prefix FlightSQL -o ./flightsql/ -driver ../driver/flightsql go run gen/main.go -prefix PanicDummy -o ./panicdummy/ -driver ../driver/panicdummy go run gen/main.go -prefix Snowflake -o ./snowflake/ -driver ../driver/snowflake diff --git a/go/adbc/pkg/clickhouse/driver.go b/go/adbc/pkg/clickhouse/driver.go new file mode 100644 index 0000000000..87b4d5be50 --- /dev/null +++ b/go/adbc/pkg/clickhouse/driver.go @@ -0,0 +1,1899 @@ +// Code generated by _tmpl/driver.go.tmpl. DO NOT EDIT. + +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF 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. + +//go:build driverlib + +package main + +// ADBC_EXPORTING is required on Windows, or else the symbols +// won't be accessible to the driver manager + +// #cgo CFLAGS: -DADBC_EXPORTING +// #cgo CXXFLAGS: -std=c++11 -DADBC_EXPORTING +// #include "../../drivermgr/adbc.h" +// #include "utils.h" +// #include +// #include +// #include +// +// typedef const char cchar_t; +// typedef const uint8_t cuint8_t; +// typedef const uint32_t cuint32_t; +// typedef const struct AdbcError ConstAdbcError; +// +// int ClickHouseArrayStreamGetSchema(struct ArrowArrayStream*, struct ArrowSchema*); +// int ClickHouseArrayStreamGetNext(struct ArrowArrayStream*, struct ArrowArray*); +// const char* ClickHouseArrayStreamGetLastError(struct ArrowArrayStream*); +// void ClickHouseArrayStreamRelease(struct ArrowArrayStream*); +// +// int ClickHouseArrayStreamGetSchemaTrampoline(struct ArrowArrayStream*, struct ArrowSchema*); +// int ClickHouseArrayStreamGetNextTrampoline(struct ArrowArrayStream*, struct ArrowArray*); +// +// void releasePartitions(struct AdbcPartitions* partitions); +// +import "C" +import ( + "context" + "errors" + "fmt" + "log/slog" + "os" + "runtime" + "runtime/cgo" + "strings" + "sync/atomic" + "unsafe" + + "github.com/apache/arrow-adbc/go/adbc" + "github.com/apache/arrow-adbc/go/adbc/driver/clickhouse" + "github.com/apache/arrow/go/v17/arrow/array" + "github.com/apache/arrow/go/v17/arrow/cdata" + "github.com/apache/arrow/go/v17/arrow/memory" + "github.com/apache/arrow/go/v17/arrow/memory/mallocator" +) + +// Must use malloc() to respect CGO rules +var drv = clickhouse.NewDriver(mallocator.NewMallocator()) + +// Flag set if any method panic()ed - afterwards all calls to driver will fail +// since internal state of driver is unknown +var globalPoison atomic.Bool + +const errPrefix = "[ClickHouse] " +const logLevelEnvVar = "ADBC_DRIVER_CLICKHOUSE_LOG_LEVEL" + +func setErr(err *C.struct_AdbcError, format string, vals ...interface{}) { + if err == nil { + return + } + + if err.release != nil { + C.ClickHouseerrRelease(err) + } + + msg := errPrefix + fmt.Sprintf(format, vals...) + err.message = C.CString(msg) + err.release = (*[0]byte)(C.ClickHouse_release_error) +} + +func setErrWithDetails(err *C.struct_AdbcError, adbcError adbc.Error) { + if err == nil { + return + } + + if err.vendor_code != C.ADBC_ERROR_VENDOR_CODE_PRIVATE_DATA { + setErr(err, adbcError.Msg) + return + } + + cErrPtr := C.malloc(C.sizeof_struct_ClickHouseError) + cErr := (*C.struct_ClickHouseError)(cErrPtr) + cErr.message = C.CString(adbcError.Msg) + err.message = cErr.message + err.release = (*[0]byte)(C.ClickHouseReleaseErrWithDetails) + err.private_data = cErrPtr + + numDetails := len(adbcError.Details) + if numDetails > 0 { + cErr.keys = (**C.cchar_t)(C.calloc(C.size_t(numDetails), C.size_t(unsafe.Sizeof((*C.cchar_t)(nil))))) + cErr.values = (**C.cuint8_t)(C.calloc(C.size_t(numDetails), C.size_t(unsafe.Sizeof((*C.cuint8_t)(nil))))) + cErr.lengths = (*C.size_t)(C.calloc(C.size_t(numDetails), C.sizeof_size_t)) + + keys := fromCArr[*C.cchar_t](cErr.keys, numDetails) + values := fromCArr[*C.cuint8_t](cErr.values, numDetails) + lengths := fromCArr[C.size_t](cErr.lengths, numDetails) + + for i, detail := range adbcError.Details { + keys[i] = C.CString(detail.Key()) + bytes, err := detail.Serialize() + if err != nil { + msg := err.Error() + values[i] = (*C.cuint8_t)(unsafe.Pointer(C.CString(msg))) + lengths[i] = C.size_t(len(msg)) + } else { + values[i] = (*C.cuint8_t)(C.malloc(C.size_t(len(bytes)))) + sink := fromCArr[byte]((*byte)(values[i]), len(bytes)) + copy(sink, bytes) + lengths[i] = C.size_t(len(bytes)) + } + } + } else { + cErr.keys = nil + cErr.values = nil + cErr.lengths = nil + } + cErr.count = C.int(numDetails) +} + +func errToAdbcErr(adbcerr *C.struct_AdbcError, err error) adbc.Status { + if err == nil { + return adbc.StatusOK + } + + var adbcError adbc.Error + if errors.As(err, &adbcError) { + setErrWithDetails(adbcerr, adbcError) + return adbcError.Code + } + + setErr(adbcerr, err.Error()) + return adbc.StatusUnknown +} + +// We panicked; make all API functions error and dump stack traces +func poison(err *C.struct_AdbcError, fname string, e interface{}) C.AdbcStatusCode { + if !globalPoison.Swap(true) { + // Only print stack traces on the first occurrence + buf := make([]byte, 1<<20) + length := runtime.Stack(buf, true) + fmt.Fprintf(os.Stderr, "ClickHouse driver panicked, stack traces:\n%s", buf[:length]) + } + setErr(err, "%s: Go panic in ClickHouse driver (see stderr): %#v", fname, e) + return C.ADBC_STATUS_INTERNAL +} + +// Check environment variables and enable logging if possible. +func initLoggingFromEnv(db adbc.Database) { + logLevel := slog.LevelError + switch strings.ToLower(os.Getenv(logLevelEnvVar)) { + case "debug": + logLevel = slog.LevelDebug + case "info": + logLevel = slog.LevelInfo + case "warn": + case "warning": + logLevel = slog.LevelWarn + case "error": + logLevel = slog.LevelError + case "": + return + default: + printLoggingHelp() + return + } + + h := slog.NewJSONHandler(os.Stderr, &slog.HandlerOptions{ + AddSource: false, + Level: logLevel, + }) + logger := slog.New(h) + + ext, ok := db.(adbc.DatabaseLogging) + if !ok { + logger.Error("ClickHouse does not support logging") + return + } + ext.SetLogger(logger) +} + +func printLoggingHelp() { + fmt.Fprintf(os.Stderr, "ClickHouse: to enable logging, set %s to 'debug', 'info', 'warn', or 'error'", logLevelEnvVar) +} + +// Allocate a new cgo.Handle and store its address in a heap-allocated +// uintptr_t. Experimentally, this was found to be necessary, else +// something (the Go runtime?) would corrupt (garbage-collect?) the +// handle. +func createHandle(hndl cgo.Handle) unsafe.Pointer { + // uintptr_t* hptr = malloc(sizeof(uintptr_t)); + hptr := (*C.uintptr_t)(C.malloc(C.sizeof_uintptr_t)) + // *hptr = (uintptr)hndl; + *hptr = C.uintptr_t(uintptr(hndl)) + return unsafe.Pointer(hptr) +} + +func getFromHandle[T any](ptr unsafe.Pointer) *T { + // uintptr_t* hptr = (uintptr_t*)ptr; + hptr := (*C.uintptr_t)(ptr) + return cgo.Handle((uintptr)(*hptr)).Value().(*T) +} + +func exportStringOption(val string, out *C.char, length *C.size_t) C.AdbcStatusCode { + lenWithTerminator := C.size_t(len(val) + 1) + if lenWithTerminator <= *length { + sink := fromCArr[byte]((*byte)(unsafe.Pointer(out)), int(*length)) + copy(sink, val) + sink[lenWithTerminator] = 0 + } + *length = lenWithTerminator + return C.ADBC_STATUS_OK +} + +func exportBytesOption(val []byte, out *C.uint8_t, length *C.size_t) C.AdbcStatusCode { + if C.size_t(len(val)) <= *length { + sink := fromCArr[byte]((*byte)(out), int(*length)) + copy(sink, val) + } + *length = C.size_t(len(val)) + return C.ADBC_STATUS_OK +} + +type cancellableContext struct { + ctx context.Context + cancel context.CancelFunc +} + +func (c *cancellableContext) newContext() context.Context { + c.cancelContext() + c.ctx, c.cancel = context.WithCancel(context.Background()) + return c.ctx +} + +func (c *cancellableContext) cancelContext() { + if c.cancel != nil { + c.cancel() + } + c.ctx = nil + c.cancel = nil +} + +func checkDBAlloc(db *C.struct_AdbcDatabase, err *C.struct_AdbcError, fname string) bool { + if globalPoison.Load() { + setErr(err, "%s: Go panicked, driver is in unknown state", fname) + return false + } + if db == nil { + setErr(err, "%s: database not allocated", fname) + return false + } + if db.private_data == nil { + setErr(err, "%s: database not allocated", fname) + return false + } + return true +} + +func checkDBInit(db *C.struct_AdbcDatabase, err *C.struct_AdbcError, fname string) *cDatabase { + if !checkDBAlloc(db, err, fname) { + return nil + } + cdb := getFromHandle[cDatabase](db.private_data) + if cdb.db == nil { + setErr(err, "%s: database not initialized", fname) + return nil + } + + return cdb +} + +// Custom ArrowArrayStream export to support ADBC error data in ArrowArrayStream + +type cArrayStream struct { + rdr array.RecordReader + // Must be C-allocated + adbcErr *C.struct_AdbcError + status C.AdbcStatusCode +} + +func (cStream *cArrayStream) maybeError() C.int { + err := cStream.rdr.Err() + if err != nil { + if cStream.adbcErr != nil { + C.ClickHouseerrRelease(cStream.adbcErr) + } else { + cStream.adbcErr = (*C.struct_AdbcError)(C.calloc(1, C.ADBC_ERROR_1_1_0_SIZE)) + } + cStream.adbcErr.vendor_code = C.ADBC_ERROR_VENDOR_CODE_PRIVATE_DATA + cStream.status = C.AdbcStatusCode(errToAdbcErr(cStream.adbcErr, err)) + switch adbc.Status(cStream.status) { + case adbc.StatusUnknown: + return C.EIO + case adbc.StatusNotImplemented: + return C.ENOTSUP + case adbc.StatusNotFound: + return C.ENOENT + case adbc.StatusAlreadyExists: + return C.EEXIST + case adbc.StatusInvalidArgument: + return C.EINVAL + case adbc.StatusInvalidState: + return C.EINVAL + case adbc.StatusInvalidData: + return C.EIO + case adbc.StatusIntegrity: + return C.EIO + case adbc.StatusInternal: + return C.EIO + case adbc.StatusIO: + return C.EIO + case adbc.StatusCancelled: + return C.ECANCELED + case adbc.StatusTimeout: + return C.ETIMEDOUT + case adbc.StatusUnauthenticated: + return C.EACCES + case adbc.StatusUnauthorized: + return C.EACCES + default: + return C.EIO + } + } + return 0 +} + +//export ClickHouseArrayStreamGetLastError +func ClickHouseArrayStreamGetLastError(stream *C.struct_ArrowArrayStream) *C.cchar_t { + if stream == nil || stream.release != (*[0]byte)(C.ClickHouseArrayStreamRelease) || stream.private_data == nil { + return nil + } + cStream := getFromHandle[cArrayStream](stream.private_data) + if cStream.adbcErr != nil { + return cStream.adbcErr.message + } + return nil +} + +//export ClickHouseArrayStreamGetNext +func ClickHouseArrayStreamGetNext(stream *C.struct_ArrowArrayStream, array *C.struct_ArrowArray) C.int { + if stream == nil || stream.release != (*[0]byte)(C.ClickHouseArrayStreamRelease) || stream.private_data == nil { + return C.EINVAL + } + cStream := getFromHandle[cArrayStream](stream.private_data) + if cStream.rdr.Next() { + cdata.ExportArrowRecordBatch(cStream.rdr.Record(), toCdataArray(array), nil) + return 0 + } + array.release = nil + array.private_data = nil + return cStream.maybeError() +} + +//export ClickHouseArrayStreamGetSchema +func ClickHouseArrayStreamGetSchema(stream *C.struct_ArrowArrayStream, schema *C.struct_ArrowSchema) C.int { + if stream == nil || stream.release != (*[0]byte)(C.ClickHouseArrayStreamRelease) || stream.private_data == nil { + return C.EINVAL + } + cStream := getFromHandle[cArrayStream](stream.private_data) + s := cStream.rdr.Schema() + if s == nil { + return cStream.maybeError() + } + cdata.ExportArrowSchema(s, toCdataSchema(schema)) + return 0 +} + +//export ClickHouseArrayStreamRelease +func ClickHouseArrayStreamRelease(stream *C.struct_ArrowArrayStream) { + if stream == nil || stream.release != (*[0]byte)(C.ClickHouseArrayStreamRelease) || stream.private_data == nil { + return + } + h := (*(*cgo.Handle)(stream.private_data)) + + cStream := h.Value().(*cArrayStream) + cStream.rdr.Release() + if cStream.adbcErr != nil { + C.ClickHouseerrRelease(cStream.adbcErr) + C.free(unsafe.Pointer(cStream.adbcErr)) + } + C.free(unsafe.Pointer(stream.private_data)) + stream.private_data = nil + h.Delete() + runtime.GC() +} + +//export ClickHouseErrorFromArrayStream +func ClickHouseErrorFromArrayStream(stream *C.struct_ArrowArrayStream, status *C.AdbcStatusCode) *C.struct_AdbcError { + if stream == nil || stream.release != (*[0]byte)(C.ClickHouseArrayStreamRelease) || stream.private_data == nil { + return nil + } + cStream := getFromHandle[cArrayStream](stream.private_data) + if status != nil { + *status = cStream.status + } + return cStream.adbcErr +} + +func exportRecordReader(rdr array.RecordReader, stream *C.struct_ArrowArrayStream) { + cStream := &cArrayStream{rdr: rdr, status: C.ADBC_STATUS_OK} + stream.get_last_error = (*[0]byte)(C.ClickHouseArrayStreamGetLastError) + stream.get_next = (*[0]byte)(C.ClickHouseArrayStreamGetNextTrampoline) + stream.get_schema = (*[0]byte)(C.ClickHouseArrayStreamGetSchemaTrampoline) + stream.release = (*[0]byte)(C.ClickHouseArrayStreamRelease) + hndl := cgo.NewHandle(cStream) + stream.private_data = createHandle(hndl) + rdr.Retain() +} + +type cDatabase struct { + opts map[string]string + db adbc.Database +} + +//export ClickHouseDatabaseGetOption +func ClickHouseDatabaseGetOption(db *C.struct_AdbcDatabase, key *C.cchar_t, value *C.char, length *C.size_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcDatabaseGetOption", e) + } + }() + cdb := checkDBInit(db, err, "AdbcDatabaseGetOption") + if cdb == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + opts, ok := cdb.db.(adbc.GetSetOptions) + if !ok { + setErr(err, "AdbcDatabaseGetOption: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + val, e := opts.GetOption(C.GoString(key)) + if e != nil { + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + + return exportStringOption(val, value, length) +} + +//export ClickHouseDatabaseGetOptionBytes +func ClickHouseDatabaseGetOptionBytes(db *C.struct_AdbcDatabase, key *C.cchar_t, value *C.uint8_t, length *C.size_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcDatabaseGetOptionBytes", e) + } + }() + cdb := checkDBInit(db, err, "AdbcDatabaseGetOptionBytes") + if cdb == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + opts, ok := cdb.db.(adbc.GetSetOptions) + if !ok { + setErr(err, "AdbcDatabaseGetOptionBytes: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + val, e := opts.GetOptionBytes(C.GoString(key)) + if e != nil { + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + + return exportBytesOption(val, value, length) +} + +//export ClickHouseDatabaseGetOptionDouble +func ClickHouseDatabaseGetOptionDouble(db *C.struct_AdbcDatabase, key *C.cchar_t, value *C.double, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcDatabaseGetOptionDouble", e) + } + }() + cdb := checkDBInit(db, err, "AdbcDatabaseGetOptionDouble") + if cdb == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + opts, ok := cdb.db.(adbc.GetSetOptions) + if !ok { + setErr(err, "AdbcDatabaseGetOptionDouble: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + + val, e := opts.GetOptionDouble(C.GoString(key)) + *value = C.double(val) + return C.AdbcStatusCode(errToAdbcErr(err, e)) +} + +//export ClickHouseDatabaseGetOptionInt +func ClickHouseDatabaseGetOptionInt(db *C.struct_AdbcDatabase, key *C.cchar_t, value *C.int64_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcDatabaseGetOptionInt", e) + } + }() + cdb := checkDBInit(db, err, "AdbcDatabaseGetOptionInt") + if cdb == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + opts, ok := cdb.db.(adbc.GetSetOptions) + if !ok { + setErr(err, "AdbcDatabaseGetOptionInt: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + + val, e := opts.GetOptionInt(C.GoString(key)) + *value = C.int64_t(val) + return C.AdbcStatusCode(errToAdbcErr(err, e)) +} + +//export ClickHouseDatabaseInit +func ClickHouseDatabaseInit(db *C.struct_AdbcDatabase, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcDatabaseInit", e) + } + }() + if !checkDBAlloc(db, err, "AdbcDatabaseInit") { + return C.ADBC_STATUS_INVALID_STATE + } + cdb := getFromHandle[cDatabase](db.private_data) + + if cdb.db != nil { + setErr(err, "AdbcDatabaseInit: database already initialized") + return C.ADBC_STATUS_INVALID_STATE + } + + adb, aerr := drv.NewDatabase(cdb.opts) + if aerr != nil { + return C.AdbcStatusCode(errToAdbcErr(err, aerr)) + } + + initLoggingFromEnv(adb) + + cdb.db = adb + return C.ADBC_STATUS_OK +} + +//export ClickHouseDatabaseNew +func ClickHouseDatabaseNew(db *C.struct_AdbcDatabase, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcDatabaseNew", e) + } + }() + if globalPoison.Load() { + setErr(err, "AdbcDatabaseNew: Go panicked, driver is in unknown state") + return C.ADBC_STATUS_INTERNAL + } + if db.private_data != nil { + setErr(err, "AdbcDatabaseNew: database already allocated") + return C.ADBC_STATUS_INVALID_STATE + } + dbobj := &cDatabase{opts: make(map[string]string)} + hndl := cgo.NewHandle(dbobj) + db.private_data = createHandle(hndl) + return C.ADBC_STATUS_OK +} + +//export ClickHouseDatabaseRelease +func ClickHouseDatabaseRelease(db *C.struct_AdbcDatabase, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcDatabaseInit", e) + } + }() + if !checkDBAlloc(db, err, "AdbcDatabaseRelease") { + return C.ADBC_STATUS_INVALID_STATE + } + h := (*(*cgo.Handle)(db.private_data)) + + cdb := h.Value().(*cDatabase) + if cdb.db != nil { + cdb.db.Close() + cdb.db = nil + } + cdb.opts = nil + if db.private_data != nil { + C.free(unsafe.Pointer(db.private_data)) + db.private_data = nil + } + h.Delete() + // manually trigger GC for two reasons: + // 1. ASAN expects the release callback to be called before + // the process ends, but GC is not deterministic. So by manually + // triggering the GC we ensure the release callback gets called. + // 2. Creates deterministic GC behavior by all Release functions + // triggering a garbage collection + runtime.GC() + return C.ADBC_STATUS_OK +} + +//export ClickHouseDatabaseSetOption +func ClickHouseDatabaseSetOption(db *C.struct_AdbcDatabase, key, value *C.cchar_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcDatabaseSetOption", e) + } + }() + if !checkDBAlloc(db, err, "AdbcDatabaseSetOption") { + return C.ADBC_STATUS_INVALID_STATE + } + cdb := getFromHandle[cDatabase](db.private_data) + + k, v := C.GoString(key), C.GoString(value) + if cdb.db != nil { + opts, ok := cdb.db.(adbc.PostInitOptions) + if !ok { + setErr(err, "AdbcDatabaseSetOption: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + return C.AdbcStatusCode(errToAdbcErr(err, opts.SetOption(k, v))) + } else { + cdb.opts[k] = v + } + + return C.ADBC_STATUS_OK +} + +//export ClickHouseDatabaseSetOptionBytes +func ClickHouseDatabaseSetOptionBytes(db *C.struct_AdbcDatabase, key *C.cchar_t, value *C.cuint8_t, length C.size_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcDatabaseSetOptionBytes", e) + } + }() + cdb := checkDBInit(db, err, "AdbcDatabaseSetOptionBytes") + if cdb == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + opts, ok := cdb.db.(adbc.GetSetOptions) + if !ok { + setErr(err, "AdbcDatabaseSetOptionBytes: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + + return C.AdbcStatusCode(errToAdbcErr(err, opts.SetOptionBytes(C.GoString(key), fromCArr[byte](value, int(length))))) +} + +//export ClickHouseDatabaseSetOptionDouble +func ClickHouseDatabaseSetOptionDouble(db *C.struct_AdbcDatabase, key *C.cchar_t, value C.double, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcDatabaseSetOptionDouble", e) + } + }() + cdb := checkDBInit(db, err, "AdbcDatabaseSetOptionDouble") + if cdb == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + opts, ok := cdb.db.(adbc.GetSetOptions) + if !ok { + setErr(err, "AdbcDatabaseSetOptionDouble: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + + return C.AdbcStatusCode(errToAdbcErr(err, opts.SetOptionDouble(C.GoString(key), float64(value)))) +} + +//export ClickHouseDatabaseSetOptionInt +func ClickHouseDatabaseSetOptionInt(db *C.struct_AdbcDatabase, key *C.cchar_t, value C.int64_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcDatabaseSetOptionInt", e) + } + }() + cdb := checkDBInit(db, err, "AdbcDatabaseSetOptionInt") + if cdb == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + opts, ok := cdb.db.(adbc.GetSetOptions) + if !ok { + setErr(err, "AdbcDatabaseSetOptionInt: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + + return C.AdbcStatusCode(errToAdbcErr(err, opts.SetOptionInt(C.GoString(key), int64(value)))) +} + +type cConn struct { + cancellableContext + + cnxn adbc.Connection + initArgs map[string]string +} + +func checkConnAlloc(cnxn *C.struct_AdbcConnection, err *C.struct_AdbcError, fname string) bool { + if globalPoison.Load() { + setErr(err, "%s: Go panicked, driver is in unknown state", fname) + return false + } + if cnxn == nil { + setErr(err, "%s: connection not allocated", fname) + return false + } + if cnxn.private_data == nil { + setErr(err, "%s: connection not allocated", fname) + return false + } + return true +} + +func checkConnInit(cnxn *C.struct_AdbcConnection, err *C.struct_AdbcError, fname string) *cConn { + if !checkConnAlloc(cnxn, err, fname) { + return nil + } + conn := getFromHandle[cConn](cnxn.private_data) + if conn.cnxn == nil { + setErr(err, "%s: connection not initialized", fname) + return nil + } + + return conn +} + +//export ClickHouseConnectionGetOption +func ClickHouseConnectionGetOption(db *C.struct_AdbcConnection, key *C.cchar_t, value *C.char, length *C.size_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcConnectionGetOption", e) + } + }() + conn := checkConnInit(db, err, "AdbcConnectionGetOption") + if conn == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + opts, ok := conn.cnxn.(adbc.GetSetOptions) + if !ok { + setErr(err, "AdbcConnectionGetOption: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + val, e := opts.GetOption(C.GoString(key)) + if e != nil { + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + + return exportStringOption(val, value, length) +} + +//export ClickHouseConnectionGetOptionBytes +func ClickHouseConnectionGetOptionBytes(db *C.struct_AdbcConnection, key *C.cchar_t, value *C.uint8_t, length *C.size_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcConnectionGetOptionBytes", e) + } + }() + conn := checkConnInit(db, err, "AdbcConnectionGetOptionBytes") + if conn == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + opts, ok := conn.cnxn.(adbc.GetSetOptions) + if !ok { + setErr(err, "AdbcConnectionGetOptionBytes: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + val, e := opts.GetOptionBytes(C.GoString(key)) + if e != nil { + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + + return exportBytesOption(val, value, length) +} + +//export ClickHouseConnectionGetOptionDouble +func ClickHouseConnectionGetOptionDouble(db *C.struct_AdbcConnection, key *C.cchar_t, value *C.double, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcConnectionGetOptionDouble", e) + } + }() + conn := checkConnInit(db, err, "AdbcConnectionGetOptionDouble") + if conn == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + opts, ok := conn.cnxn.(adbc.GetSetOptions) + if !ok { + setErr(err, "AdbcConnectionGetOptionDouble: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + + val, e := opts.GetOptionDouble(C.GoString(key)) + *value = C.double(val) + return C.AdbcStatusCode(errToAdbcErr(err, e)) +} + +//export ClickHouseConnectionGetOptionInt +func ClickHouseConnectionGetOptionInt(db *C.struct_AdbcConnection, key *C.cchar_t, value *C.int64_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcConnectionGetOptionInt", e) + } + }() + conn := checkConnInit(db, err, "AdbcConnectionGetOptionInt") + if conn == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + opts, ok := conn.cnxn.(adbc.GetSetOptions) + if !ok { + setErr(err, "AdbcConnectionGetOptionInt: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + + val, e := opts.GetOptionInt(C.GoString(key)) + *value = C.int64_t(val) + return C.AdbcStatusCode(errToAdbcErr(err, e)) +} + +//export ClickHouseConnectionNew +func ClickHouseConnectionNew(cnxn *C.struct_AdbcConnection, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcConnectionNew", e) + } + }() + if globalPoison.Load() { + setErr(err, "AdbcConnectionNew: Go panicked, driver is in unknown state") + return C.ADBC_STATUS_INTERNAL + } + if cnxn.private_data != nil { + setErr(err, "AdbcConnectionNew: connection already allocated") + return C.ADBC_STATUS_INVALID_STATE + } + + hndl := cgo.NewHandle(&cConn{}) + cnxn.private_data = createHandle(hndl) + return C.ADBC_STATUS_OK +} + +//export ClickHouseConnectionSetOption +func ClickHouseConnectionSetOption(cnxn *C.struct_AdbcConnection, key, val *C.cchar_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcConnectionSetOption", e) + } + }() + if !checkConnAlloc(cnxn, err, "AdbcConnectionSetOption") { + return C.ADBC_STATUS_INVALID_STATE + } + conn := getFromHandle[cConn](cnxn.private_data) + + if conn.cnxn == nil { + // not yet initialized + k, v := C.GoString(key), C.GoString(val) + if conn.initArgs == nil { + conn.initArgs = map[string]string{} + } + conn.initArgs[k] = v + return C.ADBC_STATUS_OK + } + + opts, ok := conn.cnxn.(adbc.PostInitOptions) + if !ok { + setErr(err, "AdbcConnectionSetOption: not supported post-init") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + return C.AdbcStatusCode(errToAdbcErr(err, opts.SetOption(C.GoString(key), C.GoString(val)))) +} + +//export ClickHouseConnectionSetOptionBytes +func ClickHouseConnectionSetOptionBytes(db *C.struct_AdbcConnection, key *C.cchar_t, value *C.cuint8_t, length C.size_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcConnectionSetOptionBytes", e) + } + }() + conn := checkConnInit(db, err, "AdbcConnectionSetOptionBytes") + if conn == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + opts, ok := conn.cnxn.(adbc.GetSetOptions) + if !ok { + setErr(err, "AdbcConnectionSetOptionBytes: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + + return C.AdbcStatusCode(errToAdbcErr(err, opts.SetOptionBytes(C.GoString(key), fromCArr[byte](value, int(length))))) +} + +//export ClickHouseConnectionSetOptionDouble +func ClickHouseConnectionSetOptionDouble(db *C.struct_AdbcConnection, key *C.cchar_t, value C.double, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcConnectionSetOptionDouble", e) + } + }() + conn := checkConnInit(db, err, "AdbcConnectionSetOptionDouble") + if conn == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + opts, ok := conn.cnxn.(adbc.GetSetOptions) + if !ok { + setErr(err, "AdbcConnectionSetOptionDouble: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + + return C.AdbcStatusCode(errToAdbcErr(err, opts.SetOptionDouble(C.GoString(key), float64(value)))) +} + +//export ClickHouseConnectionSetOptionInt +func ClickHouseConnectionSetOptionInt(db *C.struct_AdbcConnection, key *C.cchar_t, value C.int64_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcConnectionSetOptionInt", e) + } + }() + conn := checkConnInit(db, err, "AdbcConnectionSetOptionInt") + if conn == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + opts, ok := conn.cnxn.(adbc.GetSetOptions) + if !ok { + setErr(err, "AdbcConnectionSetOptionInt: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + + return C.AdbcStatusCode(errToAdbcErr(err, opts.SetOptionInt(C.GoString(key), int64(value)))) +} + +//export ClickHouseConnectionInit +func ClickHouseConnectionInit(cnxn *C.struct_AdbcConnection, db *C.struct_AdbcDatabase, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcConnectionInit", e) + } + }() + if !checkConnAlloc(cnxn, err, "AdbcConnectionInit") { + return C.ADBC_STATUS_INVALID_STATE + } + conn := getFromHandle[cConn](cnxn.private_data) + + if conn.cnxn != nil { + setErr(err, "AdbcConnectionInit: connection already initialized") + return C.ADBC_STATUS_INVALID_STATE + } + cdb := checkDBInit(db, err, "AdbcConnectionInit") + if cdb == nil { + return C.ADBC_STATUS_INVALID_STATE + } + c, e := cdb.db.Open(context.Background()) + if e != nil { + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + conn.cnxn = c + + if len(conn.initArgs) > 0 { + // C allow SetOption before Init, Go doesn't allow options to Open so set them now + opts, ok := conn.cnxn.(adbc.PostInitOptions) + if !ok { + setErr(err, "AdbcConnectionInit: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + + for k, v := range conn.initArgs { + rawCode := errToAdbcErr(err, opts.SetOption(k, v)) + if rawCode != adbc.StatusOK { + return C.AdbcStatusCode(rawCode) + } + } + conn.initArgs = nil + } + + return C.ADBC_STATUS_OK +} + +//export ClickHouseConnectionRelease +func ClickHouseConnectionRelease(cnxn *C.struct_AdbcConnection, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcConnectionRelease", e) + } + }() + if !checkConnAlloc(cnxn, err, "AdbcConnectionRelease") { + return C.ADBC_STATUS_INVALID_STATE + } + h := (*(*cgo.Handle)(cnxn.private_data)) + + conn := h.Value().(*cConn) + defer func() { + conn.cancelContext() + conn.cnxn = nil + C.free(cnxn.private_data) + cnxn.private_data = nil + h.Delete() + // manually trigger GC for two reasons: + // 1. ASAN expects the release callback to be called before + // the process ends, but GC is not deterministic. So by manually + // triggering the GC we ensure the release callback gets called. + // 2. Creates deterministic GC behavior by all Release functions + // triggering a garbage collection + runtime.GC() + }() + if conn.cnxn == nil { + return C.ADBC_STATUS_OK + } + return C.AdbcStatusCode(errToAdbcErr(err, conn.cnxn.Close())) +} + +func fromCArr[T, CType any](ptr *CType, sz int) []T { + if ptr == nil || sz == 0 { + return nil + } + + return unsafe.Slice((*T)(unsafe.Pointer(ptr)), sz) +} + +func toCdataStream(ptr *C.struct_ArrowArrayStream) *cdata.CArrowArrayStream { + return (*cdata.CArrowArrayStream)(unsafe.Pointer(ptr)) +} + +func toCdataSchema(ptr *C.struct_ArrowSchema) *cdata.CArrowSchema { + return (*cdata.CArrowSchema)(unsafe.Pointer(ptr)) +} + +func toCdataArray(ptr *C.struct_ArrowArray) *cdata.CArrowArray { + return (*cdata.CArrowArray)(unsafe.Pointer(ptr)) +} + +//export ClickHouseConnectionCancel +func ClickHouseConnectionCancel(cnxn *C.struct_AdbcConnection, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcConnectionCancel", e) + } + }() + conn := checkConnInit(cnxn, err, "AdbcConnectionCancel") + if conn == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + conn.cancelContext() + return C.ADBC_STATUS_OK +} + +func toStrPtr(in *C.cchar_t) *string { + if in == nil { + return nil + } + + out := C.GoString((*C.char)(in)) + return &out +} + +func toStrSlice(in **C.cchar_t) []string { + if in == nil { + return nil + } + + sz := unsafe.Sizeof(*in) + + out := make([]string, 0, 1) + for *in != nil { + out = append(out, C.GoString(*in)) + in = (**C.cchar_t)(unsafe.Add(unsafe.Pointer(in), sz)) + } + return out +} + +//export ClickHouseConnectionGetInfo +func ClickHouseConnectionGetInfo(cnxn *C.struct_AdbcConnection, codes *C.cuint32_t, len C.size_t, out *C.struct_ArrowArrayStream, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcConnectionGetInfo", e) + } + }() + conn := checkConnInit(cnxn, err, "AdbcConnectionGetInfo") + if conn == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + infoCodes := fromCArr[adbc.InfoCode](codes, int(len)) + rdr, e := conn.cnxn.GetInfo(conn.newContext(), infoCodes) + if e != nil { + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + + defer rdr.Release() + exportRecordReader(rdr, out) + return C.ADBC_STATUS_OK +} + +//export ClickHouseConnectionGetObjects +func ClickHouseConnectionGetObjects(cnxn *C.struct_AdbcConnection, depth C.int, catalog, dbSchema, tableName *C.cchar_t, tableType **C.cchar_t, columnName *C.cchar_t, + out *C.struct_ArrowArrayStream, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcConnectionGetObjects", e) + } + }() + conn := checkConnInit(cnxn, err, "AdbcConnectionGetObjects") + if conn == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + rdr, e := conn.cnxn.GetObjects(conn.newContext(), adbc.ObjectDepth(depth), toStrPtr(catalog), toStrPtr(dbSchema), toStrPtr(tableName), toStrPtr(columnName), toStrSlice(tableType)) + if e != nil { + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + defer rdr.Release() + exportRecordReader(rdr, out) + return C.ADBC_STATUS_OK +} + +//export ClickHouseConnectionGetStatistics +func ClickHouseConnectionGetStatistics(cnxn *C.struct_AdbcConnection, catalog, dbSchema, tableName *C.cchar_t, approximate C.char, out *C.struct_ArrowArrayStream, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcConnectionGetStatistics", e) + } + }() + conn := checkConnInit(cnxn, err, "AdbcConnectionGetStatistics") + if conn == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + gs, ok := conn.cnxn.(adbc.ConnectionGetStatistics) + if !ok { + setErr(err, "AdbcConnectionGetStatistics: not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + + rdr, e := gs.GetStatistics(conn.newContext(), toStrPtr(catalog), toStrPtr(dbSchema), toStrPtr(tableName), int(approximate) != 0) + if e != nil { + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + + defer rdr.Release() + exportRecordReader(rdr, out) + return C.ADBC_STATUS_OK +} + +//export ClickHouseConnectionGetStatisticNames +func ClickHouseConnectionGetStatisticNames(cnxn *C.struct_AdbcConnection, out *C.struct_ArrowArrayStream, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcConnectionGetStatistics", e) + } + }() + conn := checkConnInit(cnxn, err, "AdbcConnectionGetStatistics") + if conn == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + gs, ok := conn.cnxn.(adbc.ConnectionGetStatistics) + if !ok { + setErr(err, "AdbcConnectionGetStatistics: not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + + rdr, e := gs.GetStatisticNames(conn.newContext()) + if e != nil { + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + defer rdr.Release() + exportRecordReader(rdr, out) + return C.ADBC_STATUS_OK +} + +//export ClickHouseConnectionGetTableSchema +func ClickHouseConnectionGetTableSchema(cnxn *C.struct_AdbcConnection, catalog, dbSchema, tableName *C.cchar_t, schema *C.struct_ArrowSchema, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcConnectionGetTableSchema", e) + } + }() + conn := checkConnInit(cnxn, err, "AdbcConnectionGetTableSchema") + if conn == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + sc, e := conn.cnxn.GetTableSchema(conn.newContext(), toStrPtr(catalog), toStrPtr(dbSchema), C.GoString(tableName)) + if e != nil { + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + cdata.ExportArrowSchema(sc, toCdataSchema(schema)) + return C.ADBC_STATUS_OK +} + +//export ClickHouseConnectionGetTableTypes +func ClickHouseConnectionGetTableTypes(cnxn *C.struct_AdbcConnection, out *C.struct_ArrowArrayStream, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcConnectionGetTableTypes", e) + } + }() + conn := checkConnInit(cnxn, err, "AdbcConnectionGetTableTypes") + if conn == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + rdr, e := conn.cnxn.GetTableTypes(conn.newContext()) + if e != nil { + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + defer rdr.Release() + exportRecordReader(rdr, out) + return C.ADBC_STATUS_OK +} + +//export ClickHouseConnectionReadPartition +func ClickHouseConnectionReadPartition(cnxn *C.struct_AdbcConnection, serialized *C.cuint8_t, serializedLen C.size_t, out *C.struct_ArrowArrayStream, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcConnectionReadPartition", e) + } + }() + conn := checkConnInit(cnxn, err, "AdbcConnectionReadPartition") + if conn == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + rdr, e := conn.cnxn.ReadPartition(conn.newContext(), fromCArr[byte](serialized, int(serializedLen))) + if e != nil { + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + defer rdr.Release() + exportRecordReader(rdr, out) + return C.ADBC_STATUS_OK +} + +//export ClickHouseConnectionCommit +func ClickHouseConnectionCommit(cnxn *C.struct_AdbcConnection, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcConnectionCommit", e) + } + }() + conn := checkConnInit(cnxn, err, "AdbcConnectionCommit") + if conn == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + return C.AdbcStatusCode(errToAdbcErr(err, conn.cnxn.Commit(conn.newContext()))) +} + +//export ClickHouseConnectionRollback +func ClickHouseConnectionRollback(cnxn *C.struct_AdbcConnection, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcConnectionRollback", e) + } + }() + conn := checkConnInit(cnxn, err, "AdbcConnectionRollback") + if conn == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + return C.AdbcStatusCode(errToAdbcErr(err, conn.cnxn.Rollback(conn.newContext()))) +} + +type cStmt struct { + cancellableContext + + stmt adbc.Statement +} + +func checkStmtAlloc(stmt *C.struct_AdbcStatement, err *C.struct_AdbcError, fname string) bool { + if globalPoison.Load() { + setErr(err, "%s: Go panicked, driver is in unknown state", fname) + return false + } + if stmt == nil { + setErr(err, "%s: statement not allocated", fname) + return false + } + if stmt.private_data == nil { + setErr(err, "%s: statement not allocated", fname) + return false + } + return true +} + +func checkStmtInit(stmt *C.struct_AdbcStatement, err *C.struct_AdbcError, fname string) *cStmt { + if !checkStmtAlloc(stmt, err, fname) { + return nil + } + cStmt := getFromHandle[cStmt](stmt.private_data) + if cStmt.stmt == nil { + setErr(err, "%s: statement not allocated", fname) + return nil + } + return cStmt +} + +//export ClickHouseStatementGetOption +func ClickHouseStatementGetOption(db *C.struct_AdbcStatement, key *C.cchar_t, value *C.char, length *C.size_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcStatementGetOption", e) + } + }() + st := checkStmtInit(db, err, "AdbcStatementGetOption") + if st == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + opts, ok := st.stmt.(adbc.GetSetOptions) + if !ok { + setErr(err, "AdbcStatementGetOption: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + val, e := opts.GetOption(C.GoString(key)) + if e != nil { + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + + return exportStringOption(val, value, length) +} + +//export ClickHouseStatementGetOptionBytes +func ClickHouseStatementGetOptionBytes(db *C.struct_AdbcStatement, key *C.cchar_t, value *C.uint8_t, length *C.size_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcStatementGetOptionBytes", e) + } + }() + st := checkStmtInit(db, err, "AdbcStatementGetOptionBytes") + if st == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + opts, ok := st.stmt.(adbc.GetSetOptions) + if !ok { + setErr(err, "AdbcStatementGetOptionBytes: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + val, e := opts.GetOptionBytes(C.GoString(key)) + if e != nil { + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + + return exportBytesOption(val, value, length) +} + +//export ClickHouseStatementGetOptionDouble +func ClickHouseStatementGetOptionDouble(db *C.struct_AdbcStatement, key *C.cchar_t, value *C.double, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcStatementGetOptionDouble", e) + } + }() + st := checkStmtInit(db, err, "AdbcStatementGetOptionDouble") + if st == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + opts, ok := st.stmt.(adbc.GetSetOptions) + if !ok { + setErr(err, "AdbcStatementGetOptionDouble: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + + val, e := opts.GetOptionDouble(C.GoString(key)) + *value = C.double(val) + return C.AdbcStatusCode(errToAdbcErr(err, e)) +} + +//export ClickHouseStatementGetOptionInt +func ClickHouseStatementGetOptionInt(db *C.struct_AdbcStatement, key *C.cchar_t, value *C.int64_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcStatementGetOptionInt", e) + } + }() + st := checkStmtInit(db, err, "AdbcStatementGetOptionInt") + if st == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + opts, ok := st.stmt.(adbc.GetSetOptions) + if !ok { + setErr(err, "AdbcStatementGetOptionInt: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + + val, e := opts.GetOptionInt(C.GoString(key)) + *value = C.int64_t(val) + return C.AdbcStatusCode(errToAdbcErr(err, e)) +} + +//export ClickHouseStatementNew +func ClickHouseStatementNew(cnxn *C.struct_AdbcConnection, stmt *C.struct_AdbcStatement, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcStatementNew", e) + } + }() + if globalPoison.Load() { + setErr(err, "AdbcStatementNew: Go panicked, driver is in unknown state") + return C.ADBC_STATUS_INTERNAL + } + if stmt.private_data != nil { + setErr(err, "AdbcStatementNew: statement already allocated") + return C.ADBC_STATUS_INVALID_STATE + } + + conn := checkConnInit(cnxn, err, "AdbcStatementNew") + if conn == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + st, e := conn.cnxn.NewStatement() + if e != nil { + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + + hndl := cgo.NewHandle(&cStmt{stmt: st}) + stmt.private_data = createHandle(hndl) + return C.ADBC_STATUS_OK +} + +//export ClickHouseStatementRelease +func ClickHouseStatementRelease(stmt *C.struct_AdbcStatement, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcStatementRelease", e) + } + }() + if globalPoison.Load() { + setErr(err, "AdbcStatementRelease: Go panicked, driver is in unknown state") + return C.ADBC_STATUS_INTERNAL + } + if !checkStmtAlloc(stmt, err, "AdbcStatementRelease") { + return C.ADBC_STATUS_INVALID_STATE + } + h := (*(*cgo.Handle)(stmt.private_data)) + + st := h.Value().(*cStmt) + defer func() { + st.cancelContext() + st.stmt = nil + C.free(stmt.private_data) + stmt.private_data = nil + h.Delete() + // manually trigger GC for two reasons: + // 1. ASAN expects the release callback to be called before + // the process ends, but GC is not deterministic. So by manually + // triggering the GC we ensure the release callback gets called. + // 2. Creates deterministic GC behavior by all Release functions + // triggering a garbage collection + runtime.GC() + }() + if st.stmt == nil { + return C.ADBC_STATUS_OK + } + return C.AdbcStatusCode(errToAdbcErr(err, st.stmt.Close())) +} + +//export ClickHouseStatementCancel +func ClickHouseStatementCancel(stmt *C.struct_AdbcStatement, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcStatementCancel", e) + } + }() + st := checkStmtInit(stmt, err, "AdbcStatementCancel") + if st == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + st.cancelContext() + return C.ADBC_STATUS_OK +} + +//export ClickHouseStatementPrepare +func ClickHouseStatementPrepare(stmt *C.struct_AdbcStatement, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcStatementPrepare", e) + } + }() + st := checkStmtInit(stmt, err, "AdbcStatementPrepare") + if st == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + return C.AdbcStatusCode(errToAdbcErr(err, st.stmt.Prepare(st.newContext()))) +} + +//export ClickHouseStatementExecuteQuery +func ClickHouseStatementExecuteQuery(stmt *C.struct_AdbcStatement, out *C.struct_ArrowArrayStream, affected *C.int64_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcStatementExecuteQuery", e) + } + }() + st := checkStmtInit(stmt, err, "AdbcStatementExecuteQuery") + if st == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + if out == nil { + n, e := st.stmt.ExecuteUpdate(st.newContext()) + if e != nil { + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + + if affected != nil { + *affected = C.int64_t(n) + } + } else { + rdr, n, e := st.stmt.ExecuteQuery(st.newContext()) + if e != nil { + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + + if affected != nil { + *affected = C.int64_t(n) + } + + defer rdr.Release() + exportRecordReader(rdr, out) + } + return C.ADBC_STATUS_OK +} + +//export ClickHouseStatementExecuteSchema +func ClickHouseStatementExecuteSchema(stmt *C.struct_AdbcStatement, schema *C.struct_ArrowSchema, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcStatementExecuteQuery", e) + } + }() + st := checkStmtInit(stmt, err, "AdbcStatementExecuteQuery") + if st == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + es, ok := st.stmt.(adbc.StatementExecuteSchema) + if !ok { + setErr(err, "AdbcStatementExecuteSchema: not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + + sc, e := es.ExecuteSchema(st.newContext()) + if e != nil { + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + + cdata.ExportArrowSchema(sc, toCdataSchema(schema)) + return C.ADBC_STATUS_OK +} + +//export ClickHouseStatementSetSqlQuery +func ClickHouseStatementSetSqlQuery(stmt *C.struct_AdbcStatement, query *C.cchar_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcStatementSetSqlQuery", e) + } + }() + st := checkStmtInit(stmt, err, "AdbcStatementSetSqlQuery") + if st == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + return C.AdbcStatusCode(errToAdbcErr(err, st.stmt.SetSqlQuery(C.GoString(query)))) +} + +//export ClickHouseStatementSetSubstraitPlan +func ClickHouseStatementSetSubstraitPlan(stmt *C.struct_AdbcStatement, plan *C.cuint8_t, length C.size_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcStatementSetSubstraitPlan", e) + } + }() + st := checkStmtInit(stmt, err, "AdbcStatementSetSubstraitPlan") + if st == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + return C.AdbcStatusCode(errToAdbcErr(err, st.stmt.SetSubstraitPlan(fromCArr[byte](plan, int(length))))) +} + +//export ClickHouseStatementBind +func ClickHouseStatementBind(stmt *C.struct_AdbcStatement, values *C.struct_ArrowArray, schema *C.struct_ArrowSchema, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcStatementBind", e) + } + }() + st := checkStmtInit(stmt, err, "AdbcStatementBind") + if st == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + rec, e := cdata.ImportCRecordBatch(toCdataArray(values), toCdataSchema(schema)) + if e != nil { + // if there was an error, we need to manually release the input + cdata.ReleaseCArrowArray(toCdataArray(values)) + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + defer rec.Release() + + return C.AdbcStatusCode(errToAdbcErr(err, st.stmt.Bind(st.newContext(), rec))) +} + +//export ClickHouseStatementBindStream +func ClickHouseStatementBindStream(stmt *C.struct_AdbcStatement, stream *C.struct_ArrowArrayStream, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcStatementBindStream", e) + } + }() + st := checkStmtInit(stmt, err, "AdbcStatementBindStream") + if st == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + rdr, e := cdata.ImportCRecordReader(toCdataStream(stream), nil) + if e != nil { + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + return C.AdbcStatusCode(errToAdbcErr(err, st.stmt.BindStream(st.newContext(), rdr.(array.RecordReader)))) +} + +//export ClickHouseStatementGetParameterSchema +func ClickHouseStatementGetParameterSchema(stmt *C.struct_AdbcStatement, schema *C.struct_ArrowSchema, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcStatementGetParameterSchema", e) + } + }() + st := checkStmtInit(stmt, err, "AdbcStatementGetParameterSchema") + if st == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + sc, e := st.stmt.GetParameterSchema() + if e != nil { + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + + cdata.ExportArrowSchema(sc, toCdataSchema(schema)) + return C.ADBC_STATUS_OK +} + +//export ClickHouseStatementSetOption +func ClickHouseStatementSetOption(stmt *C.struct_AdbcStatement, key, value *C.cchar_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcStatementSetOption", e) + } + }() + st := checkStmtInit(stmt, err, "AdbcStatementSetOption") + if st == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + return C.AdbcStatusCode(errToAdbcErr(err, st.stmt.SetOption(C.GoString(key), C.GoString(value)))) +} + +//export ClickHouseStatementSetOptionBytes +func ClickHouseStatementSetOptionBytes(db *C.struct_AdbcStatement, key *C.cchar_t, value *C.cuint8_t, length C.size_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcStatementSetOptionBytes", e) + } + }() + st := checkStmtInit(db, err, "AdbcStatementSetOptionBytes") + if st == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + opts, ok := st.stmt.(adbc.GetSetOptions) + if !ok { + setErr(err, "AdbcStatementSetOptionBytes: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + + return C.AdbcStatusCode(errToAdbcErr(err, opts.SetOptionBytes(C.GoString(key), fromCArr[byte](value, int(length))))) +} + +//export ClickHouseStatementSetOptionDouble +func ClickHouseStatementSetOptionDouble(db *C.struct_AdbcStatement, key *C.cchar_t, value C.double, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcStatementSetOptionDouble", e) + } + }() + st := checkStmtInit(db, err, "AdbcStatementSetOptionDouble") + if st == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + opts, ok := st.stmt.(adbc.GetSetOptions) + if !ok { + setErr(err, "AdbcStatementSetOptionDouble: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + + return C.AdbcStatusCode(errToAdbcErr(err, opts.SetOptionDouble(C.GoString(key), float64(value)))) +} + +//export ClickHouseStatementSetOptionInt +func ClickHouseStatementSetOptionInt(db *C.struct_AdbcStatement, key *C.cchar_t, value C.int64_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcStatementSetOptionInt", e) + } + }() + st := checkStmtInit(db, err, "AdbcStatementSetOptionInt") + if st == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + opts, ok := st.stmt.(adbc.GetSetOptions) + if !ok { + setErr(err, "AdbcStatementSetOptionInt: options are not supported") + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + + return C.AdbcStatusCode(errToAdbcErr(err, opts.SetOptionInt(C.GoString(key), int64(value)))) +} + +//export releasePartitions +func releasePartitions(partitions *C.struct_AdbcPartitions) { + if partitions.private_data == nil { + return + } + + C.free(unsafe.Pointer(partitions.partitions)) + C.free(unsafe.Pointer(partitions.partition_lengths)) + C.free(partitions.private_data) + partitions.partitions = nil + partitions.partition_lengths = nil + partitions.private_data = nil +} + +//export ClickHouseStatementExecutePartitions +func ClickHouseStatementExecutePartitions(stmt *C.struct_AdbcStatement, schema *C.struct_ArrowSchema, partitions *C.struct_AdbcPartitions, affected *C.int64_t, err *C.struct_AdbcError) (code C.AdbcStatusCode) { + defer func() { + if e := recover(); e != nil { + code = poison(err, "AdbcStatementExecutePartitions", e) + } + }() + st := checkStmtInit(stmt, err, "AdbcStatementExecutePartitions") + if st == nil { + return C.ADBC_STATUS_INVALID_STATE + } + + sc, part, n, e := st.stmt.ExecutePartitions(st.newContext()) + if e != nil { + return C.AdbcStatusCode(errToAdbcErr(err, e)) + } + + if partitions == nil { + setErr(err, "AdbcStatementExecutePartitions: partitions output struct is null") + return C.ADBC_STATUS_INVALID_ARGUMENT + } + + if affected != nil { + *affected = C.int64_t(n) + } + + if sc != nil && schema != nil { + cdata.ExportArrowSchema(sc, toCdataSchema(schema)) + } + + partitions.num_partitions = C.size_t(part.NumPartitions) + partitions.partitions = (**C.cuint8_t)(C.malloc(C.size_t(unsafe.Sizeof((*C.uint8_t)(nil)) * uintptr(part.NumPartitions)))) + partitions.partition_lengths = (*C.size_t)(C.malloc(C.size_t(unsafe.Sizeof(C.size_t(0)) * uintptr(part.NumPartitions)))) + + // Copy into C-allocated memory to avoid violating CGO rules + totalLen := 0 + for _, p := range part.PartitionIDs { + totalLen += len(p) + } + partitions.private_data = C.malloc(C.size_t(totalLen)) + dst := fromCArr[byte]((*byte)(partitions.private_data), totalLen) + + partIDs := fromCArr[*C.cuint8_t](partitions.partitions, int(partitions.num_partitions)) + partLens := fromCArr[C.size_t](partitions.partition_lengths, int(partitions.num_partitions)) + for i, p := range part.PartitionIDs { + partIDs[i] = (*C.cuint8_t)(&dst[0]) + copy(dst, p) + dst = dst[len(p):] + partLens[i] = C.size_t(len(p)) + } + + partitions.release = (*[0]byte)(C.releasePartitions) + return C.ADBC_STATUS_OK +} + +//export ClickHouseDriverInit +func ClickHouseDriverInit(version C.int, rawDriver *C.void, err *C.struct_AdbcError) C.AdbcStatusCode { + driver := (*C.struct_AdbcDriver)(unsafe.Pointer(rawDriver)) + + switch version { + case C.ADBC_VERSION_1_0_0: + sink := fromCArr[byte]((*byte)(unsafe.Pointer(driver)), C.ADBC_DRIVER_1_0_0_SIZE) + memory.Set(sink, 0) + case C.ADBC_VERSION_1_1_0: + sink := fromCArr[byte]((*byte)(unsafe.Pointer(driver)), C.ADBC_DRIVER_1_1_0_SIZE) + memory.Set(sink, 0) + default: + setErr(err, "Only version 1.0.0/1.1.0 supported, got %d", int(version)) + return C.ADBC_STATUS_NOT_IMPLEMENTED + } + + driver.DatabaseInit = (*[0]byte)(C.ClickHouseDatabaseInit) + driver.DatabaseNew = (*[0]byte)(C.ClickHouseDatabaseNew) + driver.DatabaseRelease = (*[0]byte)(C.ClickHouseDatabaseRelease) + driver.DatabaseSetOption = (*[0]byte)(C.ClickHouseDatabaseSetOption) + + driver.ConnectionNew = (*[0]byte)(C.ClickHouseConnectionNew) + driver.ConnectionInit = (*[0]byte)(C.ClickHouseConnectionInit) + driver.ConnectionRelease = (*[0]byte)(C.ClickHouseConnectionRelease) + driver.ConnectionSetOption = (*[0]byte)(C.ClickHouseConnectionSetOption) + driver.ConnectionGetInfo = (*[0]byte)(C.ClickHouseConnectionGetInfo) + driver.ConnectionGetObjects = (*[0]byte)(C.ClickHouseConnectionGetObjects) + driver.ConnectionGetTableSchema = (*[0]byte)(C.ClickHouseConnectionGetTableSchema) + driver.ConnectionGetTableTypes = (*[0]byte)(C.ClickHouseConnectionGetTableTypes) + driver.ConnectionReadPartition = (*[0]byte)(C.ClickHouseConnectionReadPartition) + driver.ConnectionCommit = (*[0]byte)(C.ClickHouseConnectionCommit) + driver.ConnectionRollback = (*[0]byte)(C.ClickHouseConnectionRollback) + + driver.StatementNew = (*[0]byte)(C.ClickHouseStatementNew) + driver.StatementRelease = (*[0]byte)(C.ClickHouseStatementRelease) + driver.StatementSetOption = (*[0]byte)(C.ClickHouseStatementSetOption) + driver.StatementSetSqlQuery = (*[0]byte)(C.ClickHouseStatementSetSqlQuery) + driver.StatementSetSubstraitPlan = (*[0]byte)(C.ClickHouseStatementSetSubstraitPlan) + driver.StatementBind = (*[0]byte)(C.ClickHouseStatementBind) + driver.StatementBindStream = (*[0]byte)(C.ClickHouseStatementBindStream) + driver.StatementExecuteQuery = (*[0]byte)(C.ClickHouseStatementExecuteQuery) + driver.StatementExecutePartitions = (*[0]byte)(C.ClickHouseStatementExecutePartitions) + driver.StatementGetParameterSchema = (*[0]byte)(C.ClickHouseStatementGetParameterSchema) + driver.StatementPrepare = (*[0]byte)(C.ClickHouseStatementPrepare) + + if version == C.ADBC_VERSION_1_1_0 { + driver.ErrorGetDetailCount = (*[0]byte)(C.ClickHouseErrorGetDetailCount) + driver.ErrorGetDetail = (*[0]byte)(C.ClickHouseErrorGetDetail) + driver.ErrorFromArrayStream = (*[0]byte)(C.ClickHouseErrorFromArrayStream) + + driver.DatabaseGetOption = (*[0]byte)(C.ClickHouseDatabaseGetOption) + driver.DatabaseGetOptionBytes = (*[0]byte)(C.ClickHouseDatabaseGetOptionBytes) + driver.DatabaseGetOptionDouble = (*[0]byte)(C.ClickHouseDatabaseGetOptionDouble) + driver.DatabaseGetOptionInt = (*[0]byte)(C.ClickHouseDatabaseGetOptionInt) + driver.DatabaseSetOptionBytes = (*[0]byte)(C.ClickHouseDatabaseSetOptionBytes) + driver.DatabaseSetOptionDouble = (*[0]byte)(C.ClickHouseDatabaseSetOptionDouble) + driver.DatabaseSetOptionInt = (*[0]byte)(C.ClickHouseDatabaseSetOptionInt) + + driver.ConnectionCancel = (*[0]byte)(C.ClickHouseConnectionCancel) + driver.ConnectionGetOption = (*[0]byte)(C.ClickHouseConnectionGetOption) + driver.ConnectionGetOptionBytes = (*[0]byte)(C.ClickHouseConnectionGetOptionBytes) + driver.ConnectionGetOptionDouble = (*[0]byte)(C.ClickHouseConnectionGetOptionDouble) + driver.ConnectionGetOptionInt = (*[0]byte)(C.ClickHouseConnectionGetOptionInt) + driver.ConnectionGetStatistics = (*[0]byte)(C.ClickHouseConnectionGetStatistics) + driver.ConnectionGetStatisticNames = (*[0]byte)(C.ClickHouseConnectionGetStatisticNames) + driver.ConnectionSetOptionBytes = (*[0]byte)(C.ClickHouseConnectionSetOptionBytes) + driver.ConnectionSetOptionDouble = (*[0]byte)(C.ClickHouseConnectionSetOptionDouble) + driver.ConnectionSetOptionInt = (*[0]byte)(C.ClickHouseConnectionSetOptionInt) + + driver.StatementCancel = (*[0]byte)(C.ClickHouseStatementCancel) + driver.StatementExecuteSchema = (*[0]byte)(C.ClickHouseStatementExecuteSchema) + driver.StatementGetOption = (*[0]byte)(C.ClickHouseStatementGetOption) + driver.StatementGetOptionBytes = (*[0]byte)(C.ClickHouseStatementGetOptionBytes) + driver.StatementGetOptionDouble = (*[0]byte)(C.ClickHouseStatementGetOptionDouble) + driver.StatementGetOptionInt = (*[0]byte)(C.ClickHouseStatementGetOptionInt) + driver.StatementSetOptionBytes = (*[0]byte)(C.ClickHouseStatementSetOptionBytes) + driver.StatementSetOptionDouble = (*[0]byte)(C.ClickHouseStatementSetOptionDouble) + driver.StatementSetOptionInt = (*[0]byte)(C.ClickHouseStatementSetOptionInt) + } + + return C.ADBC_STATUS_OK +} + +func main() {} diff --git a/go/adbc/pkg/clickhouse/utils.c b/go/adbc/pkg/clickhouse/utils.c new file mode 100644 index 0000000000..ada0ecfb73 --- /dev/null +++ b/go/adbc/pkg/clickhouse/utils.c @@ -0,0 +1,445 @@ +// Code generated by _tmpl/utils.c.tmpl. DO NOT EDIT. + +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF 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. + +// clang-format off +//go:build driverlib +// clang-format on + +#include "utils.h" + +#include + +#ifdef __cplusplus +extern "C" { +#endif + +void ClickHouse_release_error(struct AdbcError* error) { + free(error->message); + error->message = NULL; + error->release = NULL; +} + +void ClickHouseReleaseErrWithDetails(struct AdbcError* error) { + if (!error || error->release != ClickHouseReleaseErrWithDetails || + !error->private_data) { + return; + } + + struct ClickHouseError* details = + (struct ClickHouseError*) error->private_data; + for (int i = 0; i < details->count; i++) { + free(details->keys[i]); + free(details->values[i]); + } + free(details->keys); + free(details->values); + free(details->lengths); + free(details); + + free(error->message); + error->message = NULL; + error->release = NULL; + error->private_data = NULL; +} + +int ClickHouseErrorGetDetailCount(const struct AdbcError* error) { + if (!error || error->release != ClickHouseReleaseErrWithDetails || + !error->private_data) { + return 0; + } + + return ((struct ClickHouseError*) error->private_data)->count; +} + +struct AdbcErrorDetail ClickHouseErrorGetDetail(const struct AdbcError* error, + int index) { + if (!error || error->release != ClickHouseReleaseErrWithDetails || + !error->private_data) { + return (struct AdbcErrorDetail){NULL, NULL, 0}; + } + struct ClickHouseError* details = (struct ClickHouseError*) error->private_data; + if (index < 0 || index >= details->count) { + return (struct AdbcErrorDetail){NULL, NULL, 0}; + } + + return (struct AdbcErrorDetail){ + .key = details->keys[index], + .value = details->values[index], + .value_length = details->lengths[index] + }; +} + +int AdbcErrorGetDetailCount(const struct AdbcError* error) { + return ClickHouseErrorGetDetailCount(error); +} + +struct AdbcErrorDetail AdbcErrorGetDetail(const struct AdbcError* error, int index) { + return ClickHouseErrorGetDetail(error, index); +} + +const struct AdbcError* AdbcErrorFromArrayStream(struct ArrowArrayStream* stream, + AdbcStatusCode* status) { + return ClickHouseErrorFromArrayStream(stream, status); +} + +AdbcStatusCode AdbcDatabaseGetOption(struct AdbcDatabase* database, const char* key, + char* value, size_t* length, + struct AdbcError* error) { + return ClickHouseDatabaseGetOption(database, key, value, length, error); +} + +AdbcStatusCode AdbcDatabaseGetOptionBytes(struct AdbcDatabase* database, const char* key, + uint8_t* value, size_t* length, + struct AdbcError* error) { + return ClickHouseDatabaseGetOptionBytes(database, key, value, length, error); +} + +AdbcStatusCode AdbcDatabaseGetOptionDouble(struct AdbcDatabase* database, const char* key, + double* value, struct AdbcError* error) { + return ClickHouseDatabaseGetOptionDouble(database, key, value, error); +} + +AdbcStatusCode AdbcDatabaseGetOptionInt(struct AdbcDatabase* database, const char* key, + int64_t* value, struct AdbcError* error) { + return ClickHouseDatabaseGetOptionInt(database, key, value, error); +} + +AdbcStatusCode AdbcDatabaseInit(struct AdbcDatabase* database, struct AdbcError* error) { + return ClickHouseDatabaseInit(database, error); +} + +AdbcStatusCode AdbcDatabaseNew(struct AdbcDatabase* database, struct AdbcError* error) { + return ClickHouseDatabaseNew(database, error); +} + +AdbcStatusCode AdbcDatabaseRelease(struct AdbcDatabase* database, + struct AdbcError* error) { + return ClickHouseDatabaseRelease(database, error); +} + +AdbcStatusCode AdbcDatabaseSetOption(struct AdbcDatabase* database, const char* key, + const char* value, struct AdbcError* error) { + return ClickHouseDatabaseSetOption(database, key, value, error); +} + +AdbcStatusCode AdbcDatabaseSetOptionBytes(struct AdbcDatabase* database, const char* key, + const uint8_t* value, size_t length, + struct AdbcError* error) { + return ClickHouseDatabaseSetOptionBytes(database, key, value, length, error); +} + +AdbcStatusCode AdbcDatabaseSetOptionDouble(struct AdbcDatabase* database, const char* key, + double value, struct AdbcError* error) { + return ClickHouseDatabaseSetOptionDouble(database, key, value, error); +} + +AdbcStatusCode AdbcDatabaseSetOptionInt(struct AdbcDatabase* database, const char* key, + int64_t value, struct AdbcError* error) { + return ClickHouseDatabaseSetOptionInt(database, key, value, error); +} + +AdbcStatusCode AdbcConnectionCancel(struct AdbcConnection* connection, + struct AdbcError* error) { + return ClickHouseConnectionCancel(connection, error); +} + +AdbcStatusCode AdbcConnectionCommit(struct AdbcConnection* connection, + struct AdbcError* error) { + return ClickHouseConnectionCommit(connection, error); +} + +AdbcStatusCode AdbcConnectionGetInfo(struct AdbcConnection* connection, + const uint32_t* info_codes, size_t info_codes_length, + struct ArrowArrayStream* out, + struct AdbcError* error) { + if (out) memset(out, 0, sizeof(*out)); + return ClickHouseConnectionGetInfo(connection, info_codes, info_codes_length, + out, error); +} + +AdbcStatusCode AdbcConnectionGetObjects(struct AdbcConnection* connection, int depth, + const char* catalog, const char* db_schema, + const char* table_name, const char** table_type, + const char* column_name, + struct ArrowArrayStream* out, + struct AdbcError* error) { + if (out) memset(out, 0, sizeof(*out)); + return ClickHouseConnectionGetObjects(connection, depth, catalog, db_schema, table_name, + table_type, column_name, out, error); +} + +AdbcStatusCode AdbcConnectionGetOption(struct AdbcConnection* connection, const char* key, + char* value, size_t* length, + struct AdbcError* error) { + return ClickHouseConnectionGetOption(connection, key, value, length, error); +} + +AdbcStatusCode AdbcConnectionGetOptionBytes(struct AdbcConnection* connection, + const char* key, uint8_t* value, + size_t* length, struct AdbcError* error) { + return ClickHouseConnectionGetOptionBytes(connection, key, value, length, error); +} + +AdbcStatusCode AdbcConnectionGetOptionDouble(struct AdbcConnection* connection, + const char* key, double* value, + struct AdbcError* error) { + return ClickHouseConnectionGetOptionDouble(connection, key, value, error); +} + +AdbcStatusCode AdbcConnectionGetOptionInt(struct AdbcConnection* connection, + const char* key, int64_t* value, + struct AdbcError* error) { + return ClickHouseConnectionGetOptionInt(connection, key, value, error); +} + +AdbcStatusCode AdbcConnectionGetStatistics(struct AdbcConnection* connection, + const char* catalog, const char* db_schema, + const char* table_name, char approximate, + struct ArrowArrayStream* out, + struct AdbcError* error) { + return ClickHouseConnectionGetStatistics(connection, catalog, db_schema, table_name, + approximate, out, error); +} + +AdbcStatusCode AdbcConnectionGetStatisticNames(struct AdbcConnection* connection, + struct ArrowArrayStream* out, + struct AdbcError* error) { + return ClickHouseConnectionGetStatisticNames(connection, out, error); +} + +AdbcStatusCode AdbcConnectionGetTableSchema(struct AdbcConnection* connection, + const char* catalog, const char* db_schema, + const char* table_name, + struct ArrowSchema* schema, + struct AdbcError* error) { + if (schema) memset(schema, 0, sizeof(*schema)); + return ClickHouseConnectionGetTableSchema(connection, catalog, db_schema, table_name, + schema, error); +} + +AdbcStatusCode AdbcConnectionGetTableTypes(struct AdbcConnection* connection, + struct ArrowArrayStream* out, + struct AdbcError* error) { + if (out) memset(out, 0, sizeof(*out)); + return ClickHouseConnectionGetTableTypes(connection, out, error); +} + +AdbcStatusCode AdbcConnectionInit(struct AdbcConnection* connection, + struct AdbcDatabase* database, + struct AdbcError* error) { + return ClickHouseConnectionInit(connection, database, error); +} + +AdbcStatusCode AdbcConnectionNew(struct AdbcConnection* connection, + struct AdbcError* error) { + return ClickHouseConnectionNew(connection, error); +} + +AdbcStatusCode AdbcConnectionReadPartition(struct AdbcConnection* connection, + const uint8_t* serialized_partition, + size_t serialized_length, + struct ArrowArrayStream* out, + struct AdbcError* error) { + if (out) memset(out, 0, sizeof(*out)); + return ClickHouseConnectionReadPartition(connection, serialized_partition, + serialized_length, out, error); +} + +AdbcStatusCode AdbcConnectionRelease(struct AdbcConnection* connection, + struct AdbcError* error) { + return ClickHouseConnectionRelease(connection, error); +} + +AdbcStatusCode AdbcConnectionRollback(struct AdbcConnection* connection, + struct AdbcError* error) { + return ClickHouseConnectionRollback(connection, error); +} + +AdbcStatusCode AdbcConnectionSetOption(struct AdbcConnection* connection, const char* key, + const char* value, struct AdbcError* error) { + return ClickHouseConnectionSetOption(connection, key, value, error); +} + +AdbcStatusCode AdbcConnectionSetOptionBytes(struct AdbcConnection* connection, + const char* key, const uint8_t* value, + size_t length, struct AdbcError* error) { + return ClickHouseConnectionSetOptionBytes(connection, key, value, length, error); +} + +AdbcStatusCode AdbcConnectionSetOptionDouble(struct AdbcConnection* connection, + const char* key, double value, + struct AdbcError* error) { + return ClickHouseConnectionSetOptionDouble(connection, key, value, error); +} + +AdbcStatusCode AdbcConnectionSetOptionInt(struct AdbcConnection* connection, + const char* key, int64_t value, + struct AdbcError* error) { + return ClickHouseConnectionSetOptionInt(connection, key, value, error); +} + +AdbcStatusCode AdbcStatementCancel(struct AdbcStatement* statement, + struct AdbcError* error) { + return ClickHouseStatementCancel(statement, error); +} + +AdbcStatusCode AdbcStatementBind(struct AdbcStatement* statement, + struct ArrowArray* values, struct ArrowSchema* schema, + struct AdbcError* error) { + return ClickHouseStatementBind(statement, values, schema, error); +} + +AdbcStatusCode AdbcStatementBindStream(struct AdbcStatement* statement, + struct ArrowArrayStream* stream, + struct AdbcError* error) { + return ClickHouseStatementBindStream(statement, stream, error); +} + +AdbcStatusCode AdbcStatementExecutePartitions(struct AdbcStatement* statement, + struct ArrowSchema* schema, + struct AdbcPartitions* partitions, + int64_t* rows_affected, + struct AdbcError* error) { + if (schema) memset(schema, 0, sizeof(*schema)); + if (partitions) memset(partitions, 0, sizeof(*partitions)); + return ClickHouseStatementExecutePartitions(statement, schema, partitions, + rows_affected, error); +} + +AdbcStatusCode AdbcStatementExecuteQuery(struct AdbcStatement* statement, + struct ArrowArrayStream* out, + int64_t* rows_affected, + struct AdbcError* error) { + if (out) memset(out, 0, sizeof(*out)); + return ClickHouseStatementExecuteQuery(statement, out, rows_affected, error); +} + +AdbcStatusCode AdbcStatementExecuteSchema(struct AdbcStatement* statement, + struct ArrowSchema* schema, + struct AdbcError* error) { + if (schema) memset(schema, 0, sizeof(*schema)); + return ClickHouseStatementExecuteSchema(statement, schema, error); +} + +AdbcStatusCode AdbcStatementGetOption(struct AdbcStatement* statement, const char* key, + char* value, size_t* length, + struct AdbcError* error) { + return ClickHouseStatementGetOption(statement, key, value, length, error); +} + +AdbcStatusCode AdbcStatementGetOptionBytes(struct AdbcStatement* statement, + const char* key, uint8_t* value, + size_t* length, struct AdbcError* error) { + return ClickHouseStatementGetOptionBytes(statement, key, value, length, error); +} + +AdbcStatusCode AdbcStatementGetOptionDouble(struct AdbcStatement* statement, + const char* key, double* value, + struct AdbcError* error) { + return ClickHouseStatementGetOptionDouble(statement, key, value, error); +} + +AdbcStatusCode AdbcStatementGetOptionInt(struct AdbcStatement* statement, + const char* key, int64_t* value, + struct AdbcError* error) { + return ClickHouseStatementGetOptionInt(statement, key, value, error); +} + +AdbcStatusCode AdbcStatementGetParameterSchema(struct AdbcStatement* statement, + struct ArrowSchema* schema, + struct AdbcError* error) { + if (schema) memset(schema, 0, sizeof(*schema)); + return ClickHouseStatementGetParameterSchema(statement, schema, error); +} + +AdbcStatusCode AdbcStatementNew(struct AdbcConnection* connection, + struct AdbcStatement* statement, + struct AdbcError* error) { + return ClickHouseStatementNew(connection, statement, error); +} + +AdbcStatusCode AdbcStatementPrepare(struct AdbcStatement* statement, + struct AdbcError* error) { + return ClickHouseStatementPrepare(statement, error); +} + +AdbcStatusCode AdbcStatementRelease(struct AdbcStatement* statement, + struct AdbcError* error) { + return ClickHouseStatementRelease(statement, error); +} + +AdbcStatusCode AdbcStatementSetSqlQuery(struct AdbcStatement* statement, + const char* query, struct AdbcError* error) { + return ClickHouseStatementSetSqlQuery(statement, query, error); +} + +AdbcStatusCode AdbcStatementSetSubstraitPlan(struct AdbcStatement* statement, + const uint8_t* plan, size_t length, + struct AdbcError* error) { + return ClickHouseStatementSetSubstraitPlan(statement, plan, length, error); +} + +AdbcStatusCode AdbcStatementSetOption(struct AdbcStatement* statement, const char* key, + const char* value, struct AdbcError* error) { + return ClickHouseStatementSetOption(statement, key, value, error); +} + +AdbcStatusCode AdbcStatementSetOptionBytes(struct AdbcStatement* statement, + const char* key, const uint8_t* value, + size_t length, struct AdbcError* error) { + return ClickHouseStatementSetOptionBytes(statement, key, value, length, error); +} + +AdbcStatusCode AdbcStatementSetOptionDouble(struct AdbcStatement* statement, + const char* key, double value, + struct AdbcError* error) { + return ClickHouseStatementSetOptionDouble(statement, key, value, error); +} + +AdbcStatusCode AdbcStatementSetOptionInt(struct AdbcStatement* statement, + const char* key, int64_t value, + struct AdbcError* error) { + return ClickHouseStatementSetOptionInt(statement, key, value, error); +} + +ADBC_EXPORT +AdbcStatusCode AdbcDriverInit(int version, void* driver, struct AdbcError* error) { + return ClickHouseDriverInit(version, driver, error); +} + +int ClickHouseArrayStreamGetSchema(struct ArrowArrayStream*, struct ArrowSchema*); +int ClickHouseArrayStreamGetNext(struct ArrowArrayStream*, struct ArrowArray*); + +int ClickHouseArrayStreamGetSchemaTrampoline(struct ArrowArrayStream* stream, + struct ArrowSchema* out) { + // XXX(https://github.com/apache/arrow-adbc/issues/729) + memset(out, 0, sizeof(*out)); + return ClickHouseArrayStreamGetSchema(stream, out); +} + +int ClickHouseArrayStreamGetNextTrampoline(struct ArrowArrayStream* stream, + struct ArrowArray* out) { + // XXX(https://github.com/apache/arrow-adbc/issues/729) + memset(out, 0, sizeof(*out)); + return ClickHouseArrayStreamGetNext(stream, out); +} + +#ifdef __cplusplus +} +#endif diff --git a/go/adbc/pkg/clickhouse/utils.h b/go/adbc/pkg/clickhouse/utils.h new file mode 100644 index 0000000000..49fda74133 --- /dev/null +++ b/go/adbc/pkg/clickhouse/utils.h @@ -0,0 +1,183 @@ +// Code generated by _tmpl/utils.h.tmpl. DO NOT EDIT. + +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF 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. + +// clang-format off +//go:build driverlib +// clang-format on + +#pragma once + +#include +#include "../../drivermgr/adbc.h" + +struct AdbcError* ClickHouseErrorFromArrayStream(struct ArrowArrayStream*, + AdbcStatusCode*); +AdbcStatusCode ClickHouseDatabaseGetOption(struct AdbcDatabase*, const char*, char*, + size_t*, struct AdbcError*); +AdbcStatusCode ClickHouseDatabaseGetOptionBytes(struct AdbcDatabase*, const char*, + uint8_t*, size_t*, struct AdbcError*); +AdbcStatusCode ClickHouseDatabaseGetOptionDouble(struct AdbcDatabase*, const char*, + double*, struct AdbcError*); +AdbcStatusCode ClickHouseDatabaseGetOptionInt(struct AdbcDatabase*, const char*, int64_t*, + struct AdbcError*); +AdbcStatusCode ClickHouseDatabaseInit(struct AdbcDatabase* db, struct AdbcError* err); +AdbcStatusCode ClickHouseDatabaseNew(struct AdbcDatabase* db, struct AdbcError* err); +AdbcStatusCode ClickHouseDatabaseRelease(struct AdbcDatabase* db, struct AdbcError* err); +AdbcStatusCode ClickHouseDatabaseSetOption(struct AdbcDatabase* db, const char* key, + const char* value, struct AdbcError* err); +AdbcStatusCode ClickHouseDatabaseSetOptionBytes(struct AdbcDatabase*, const char*, + const uint8_t*, size_t, + struct AdbcError*); +AdbcStatusCode ClickHouseDatabaseSetOptionDouble(struct AdbcDatabase*, const char*, + double, struct AdbcError*); +AdbcStatusCode ClickHouseDatabaseSetOptionInt(struct AdbcDatabase*, const char*, int64_t, + struct AdbcError*); + +AdbcStatusCode ClickHouseConnectionCancel(struct AdbcConnection*, struct AdbcError*); +AdbcStatusCode ClickHouseConnectionCommit(struct AdbcConnection* cnxn, + struct AdbcError* err); +AdbcStatusCode ClickHouseConnectionGetInfo(struct AdbcConnection* cnxn, + const uint32_t* codes, size_t len, + struct ArrowArrayStream* out, + struct AdbcError* err); +AdbcStatusCode ClickHouseConnectionGetObjects( + struct AdbcConnection* cnxn, int depth, const char* catalog, const char* dbSchema, + const char* tableName, const char** tableType, const char* columnName, + struct ArrowArrayStream* out, struct AdbcError* err); +AdbcStatusCode ClickHouseConnectionGetOption(struct AdbcConnection*, const char*, char*, + size_t*, struct AdbcError*); +AdbcStatusCode ClickHouseConnectionGetOptionBytes(struct AdbcConnection*, const char*, + uint8_t*, size_t*, struct AdbcError*); +AdbcStatusCode ClickHouseConnectionGetOptionDouble(struct AdbcConnection*, const char*, + double*, struct AdbcError*); +AdbcStatusCode ClickHouseConnectionGetOptionInt(struct AdbcConnection*, const char*, + int64_t*, struct AdbcError*); +AdbcStatusCode ClickHouseConnectionGetStatistics(struct AdbcConnection*, const char*, + const char*, const char*, char, + struct ArrowArrayStream*, + struct AdbcError*); +AdbcStatusCode ClickHouseConnectionGetStatisticNames(struct AdbcConnection*, + struct ArrowArrayStream*, + struct AdbcError*); +AdbcStatusCode ClickHouseConnectionGetTableSchema( + struct AdbcConnection* cnxn, const char* catalog, const char* dbSchema, + const char* tableName, struct ArrowSchema* schema, struct AdbcError* err); +AdbcStatusCode ClickHouseConnectionGetTableTypes(struct AdbcConnection* cnxn, + struct ArrowArrayStream* out, + struct AdbcError* err); +AdbcStatusCode ClickHouseConnectionInit(struct AdbcConnection* cnxn, + struct AdbcDatabase* db, struct AdbcError* err); +AdbcStatusCode ClickHouseConnectionNew(struct AdbcConnection* cnxn, + struct AdbcError* err); +AdbcStatusCode ClickHouseConnectionReadPartition(struct AdbcConnection* cnxn, + const uint8_t* serialized, + size_t serializedLen, + struct ArrowArrayStream* out, + struct AdbcError* err); +AdbcStatusCode ClickHouseConnectionRelease(struct AdbcConnection* cnxn, + struct AdbcError* err); +AdbcStatusCode ClickHouseConnectionRollback(struct AdbcConnection* cnxn, + struct AdbcError* err); +AdbcStatusCode ClickHouseConnectionSetOption(struct AdbcConnection* cnxn, const char* key, + const char* val, struct AdbcError* err); +AdbcStatusCode ClickHouseConnectionSetOptionBytes(struct AdbcConnection*, const char*, + const uint8_t*, size_t, + struct AdbcError*); +AdbcStatusCode ClickHouseConnectionSetOptionDouble(struct AdbcConnection*, const char*, + double, struct AdbcError*); +AdbcStatusCode ClickHouseConnectionSetOptionInt(struct AdbcConnection*, const char*, + int64_t, struct AdbcError*); + +AdbcStatusCode ClickHouseStatementBind(struct AdbcStatement* stmt, + struct ArrowArray* values, + struct ArrowSchema* schema, struct AdbcError* err); +AdbcStatusCode ClickHouseStatementBindStream(struct AdbcStatement* stmt, + struct ArrowArrayStream* stream, + struct AdbcError* err); +AdbcStatusCode ClickHouseStatementCancel(struct AdbcStatement*, struct AdbcError*); +AdbcStatusCode ClickHouseStatementExecuteQuery(struct AdbcStatement* stmt, + struct ArrowArrayStream* out, + int64_t* affected, struct AdbcError* err); +AdbcStatusCode ClickHouseStatementExecutePartitions(struct AdbcStatement* stmt, + struct ArrowSchema* schema, + struct AdbcPartitions* partitions, + int64_t* affected, + struct AdbcError* err); +AdbcStatusCode ClickHouseStatementExecuteSchema(struct AdbcStatement*, + struct ArrowSchema*, struct AdbcError*); +AdbcStatusCode ClickHouseStatementGetOption(struct AdbcStatement*, const char*, char*, + size_t*, struct AdbcError*); +AdbcStatusCode ClickHouseStatementGetOptionBytes(struct AdbcStatement*, const char*, + uint8_t*, size_t*, struct AdbcError*); +AdbcStatusCode ClickHouseStatementGetOptionDouble(struct AdbcStatement*, const char*, + double*, struct AdbcError*); +AdbcStatusCode ClickHouseStatementGetOptionInt(struct AdbcStatement*, const char*, + int64_t*, struct AdbcError*); +AdbcStatusCode ClickHouseStatementGetParameterSchema(struct AdbcStatement* stmt, + struct ArrowSchema* schema, + struct AdbcError* err); +AdbcStatusCode ClickHouseStatementNew(struct AdbcConnection* cnxn, + struct AdbcStatement* stmt, struct AdbcError* err); +AdbcStatusCode ClickHouseStatementPrepare(struct AdbcStatement* stmt, + struct AdbcError* err); +AdbcStatusCode ClickHouseStatementRelease(struct AdbcStatement* stmt, + struct AdbcError* err); +AdbcStatusCode ClickHouseStatementSetOption(struct AdbcStatement* stmt, const char* key, + const char* value, struct AdbcError* err); +AdbcStatusCode ClickHouseStatementSetOptionBytes(struct AdbcStatement*, const char*, + const uint8_t*, size_t, + struct AdbcError*); +AdbcStatusCode ClickHouseStatementSetOptionDouble(struct AdbcStatement*, const char*, + double, struct AdbcError*); +AdbcStatusCode ClickHouseStatementSetOptionInt(struct AdbcStatement*, const char*, + int64_t, struct AdbcError*); +AdbcStatusCode ClickHouseStatementSetSqlQuery(struct AdbcStatement* stmt, + const char* query, struct AdbcError* err); +AdbcStatusCode ClickHouseStatementSetSubstraitPlan(struct AdbcStatement* stmt, + const uint8_t* plan, size_t length, + struct AdbcError* err); + +AdbcStatusCode ClickHouseDriverInit(int version, void* rawDriver, struct AdbcError* err); + +static inline void ClickHouseerrRelease(struct AdbcError* error) { + if (error->release) { + error->release(error); + error->release = NULL; + } +} + +void ClickHouse_release_error(struct AdbcError* error); + +struct ClickHouseError { + char* message; + char** keys; + uint8_t** values; + size_t* lengths; + int count; +}; + +void ClickHouseReleaseErrWithDetails(struct AdbcError* error); + +int ClickHouseErrorGetDetailCount(const struct AdbcError* error); +struct AdbcErrorDetail ClickHouseErrorGetDetail(const struct AdbcError* error, int index); + +int ClickHouseArrayStreamGetSchemaTrampoline(struct ArrowArrayStream* stream, + struct ArrowSchema* out); +int ClickHouseArrayStreamGetNextTrampoline(struct ArrowArrayStream* stream, + struct ArrowArray* out);