From 20d4a1e3deb1d23bfa416b07e2a58fec9957d080 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Rafa=C5=82=20Strzali=C5=84ski?= Date: Mon, 15 Jul 2024 11:19:38 +0200 Subject: [PATCH] Add optimization pipeline (#521) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Let's introduce the SQL query optimization pipeline. Now there are only two optimizations supported: - truncate dates to 5 minutes, it will limit how many distinct queries we perform - cache aggregation queries ("GROUP BY"), these queries can be slow, and produce a low number of rows, Optimizations are disabled for tests. Currently, performed optimizations are not visible in the "Live View" tab. I will add it in the next PR. --------- Signed-off-by: Rafał Strzaliński Co-authored-by: Grzegorz Piwowarek Co-authored-by: Jacek Migdal --- quesma/clickhouse/quesma_communicator.go | 14 +- quesma/model/query.go | 12 + quesma/optimize/cache_group_by.go | 37 +++ quesma/optimize/pipeline.go | 45 ++++ quesma/optimize/pipeline_test.go | 193 ++++++++++++++++ quesma/optimize/trunc_date.go | 276 +++++++++++++++++++++++ quesma/queryparser/query_translator.go | 1 + quesma/quesma/quesma.go | 6 + quesma/quesma/search.go | 13 +- 9 files changed, 594 insertions(+), 3 deletions(-) create mode 100644 quesma/optimize/cache_group_by.go create mode 100644 quesma/optimize/pipeline.go create mode 100644 quesma/optimize/pipeline_test.go create mode 100644 quesma/optimize/trunc_date.go diff --git a/quesma/clickhouse/quesma_communicator.go b/quesma/clickhouse/quesma_communicator.go index d19dd34ff..5c121e8ae 100644 --- a/quesma/clickhouse/quesma_communicator.go +++ b/quesma/clickhouse/quesma_communicator.go @@ -72,7 +72,7 @@ func (lm *LogManager) ProcessQuery(ctx context.Context, table *Table, query *mod } - rows, err := executeQuery(ctx, lm, query.SelectCommand.String(), columns, rowToScan) + rows, err := executeQuery(ctx, lm, query, columns, rowToScan) if err == nil { for _, row := range rows { @@ -121,9 +121,11 @@ func (lm *LogManager) explainQuery(ctx context.Context, query string, elapsed ti } } -func executeQuery(ctx context.Context, lm *LogManager, queryAsString string, fields []string, rowToScan []interface{}) ([]model.QueryResultRow, error) { +func executeQuery(ctx context.Context, lm *LogManager, query *model.Query, fields []string, rowToScan []interface{}) ([]model.QueryResultRow, error) { span := lm.phoneHomeAgent.ClickHouseQueryDuration().Begin() + queryAsString := query.SelectCommand.String() + // We drop privileges for the query // // https://clickhouse.com/docs/en/operations/settings/permissions-for-queries @@ -133,6 +135,14 @@ func executeQuery(ctx context.Context, lm *LogManager, queryAsString string, fie settings["readonly"] = "1" settings["allow_ddl"] = "0" + if query.OptimizeHints != nil { + for k, v := range query.OptimizeHints.Settings { + settings[k] = v + } + + queryAsString = queryAsString + "\n-- optimizations: " + strings.Join(query.OptimizeHints.OptimizationsPerformed, ", ") + "\n" + } + ctx = clickhouse.Context(ctx, clickhouse.WithSettings(settings)) rows, err := lm.Query(ctx, queryAsString) diff --git a/quesma/model/query.go b/quesma/model/query.go index f63ae301c..cc2c0ece5 100644 --- a/quesma/model/query.go +++ b/quesma/model/query.go @@ -11,10 +11,18 @@ const ( noLimit = 0 ) +// QueryOptimizeHints contains hints for query execution, e.g., performance settings, temporary table usage +type QueryOptimizeHints struct { + Settings map[string]any + OptimizationsPerformed []string +} + type ( Query struct { SelectCommand SelectCommand // The representation of SELECT query + OptimizeHints *QueryOptimizeHints // it can be optional + Type QueryType TableName string @@ -44,6 +52,10 @@ type ( } ) +func NewQueryExecutionHints() *QueryOptimizeHints { + return &QueryOptimizeHints{Settings: make(map[string]any)} +} + func NewSortColumn(field string, direction OrderByDirection) OrderByExpr { return NewOrderByExpr([]Expr{NewColumnRef(field)}, direction) } diff --git a/quesma/optimize/cache_group_by.go b/quesma/optimize/cache_group_by.go new file mode 100644 index 000000000..c9c5fc2d7 --- /dev/null +++ b/quesma/optimize/cache_group_by.go @@ -0,0 +1,37 @@ +// Copyright Quesma, licensed under the Elastic License 2.0. +// SPDX-License-Identifier: Elastic-2.0 +package optimize + +import "quesma/model" + +// cacheGroupByQueries - a transformer that suggests db to cache the query results +// +// It's done by adding settings to the query +// +// https://clickhouse.com/docs/en/operations/query-cache +// +// Cached queries can be examined with: +// +// select * from system.query_cache +// +// Cache can be dropped with +// +// SYSTEM DROP QUERY CACHE +// + +type cacheGroupByQueries struct { +} + +func (s *cacheGroupByQueries) Transform(queries []*model.Query) ([]*model.Query, error) { + + for _, query := range queries { + + // TODO add better detection + // TODO add CTE here + if len(query.SelectCommand.GroupBy) > 0 { + query.OptimizeHints.Settings["use_query_cache"] = true + query.OptimizeHints.OptimizationsPerformed = append(query.OptimizeHints.OptimizationsPerformed, "cacheGroupByQueries") + } + } + return queries, nil +} diff --git a/quesma/optimize/pipeline.go b/quesma/optimize/pipeline.go new file mode 100644 index 000000000..09d6c26b2 --- /dev/null +++ b/quesma/optimize/pipeline.go @@ -0,0 +1,45 @@ +// Copyright Quesma, licensed under the Elastic License 2.0. +// SPDX-License-Identifier: Elastic-2.0 +package optimize + +import ( + "quesma/model" + "quesma/plugins" + "time" +) + +// OptimizePipeline - a transformer that optimizes queries +type OptimizePipeline struct { + optimizations []plugins.QueryTransformer +} + +func NewOptimizePipeline() plugins.QueryTransformer { + + return &OptimizePipeline{ + optimizations: []plugins.QueryTransformer{ + &truncateDate{truncateTo: 5 * time.Minute}, + &cacheGroupByQueries{}, + }, + } +} + +func (s *OptimizePipeline) Transform(queries []*model.Query) ([]*model.Query, error) { + + // add hints if not present + for _, query := range queries { + if query.OptimizeHints == nil { + query.OptimizeHints = model.NewQueryExecutionHints() + } + } + + // run optimizations on queries + for _, optimization := range s.optimizations { + var err error + queries, err = optimization.Transform(queries) + if err != nil { + return nil, err + } + } + + return queries, nil +} diff --git a/quesma/optimize/pipeline_test.go b/quesma/optimize/pipeline_test.go new file mode 100644 index 000000000..fab31364b --- /dev/null +++ b/quesma/optimize/pipeline_test.go @@ -0,0 +1,193 @@ +// Copyright Quesma, licensed under the Elastic License 2.0. +// SPDX-License-Identifier: Elastic-2.0 +package optimize + +import ( + "fmt" + "github.com/stretchr/testify/assert" + "quesma/model" + "testing" +) + +func Test_cacheGroupBy(t *testing.T) { + + tests := []struct { + name string + shouldCache bool + query model.SelectCommand + }{ + { + "select all", + false, + model.SelectCommand{ + Columns: []model.Expr{model.NewColumnRef("*")}, + FromClause: model.NewTableRef("foo"), + }, + }, + + { + "select a, count() from foo group by 1", + true, + model.SelectCommand{ + Columns: []model.Expr{model.NewColumnRef("a"), model.NewFunction("count", model.NewColumnRef("*"))}, + FromClause: model.NewTableRef("foo"), + GroupBy: []model.Expr{model.NewLiteral(1)}, + }, + }, + // Add CTE here + } + + for _, tt := range tests { + + t.Run(tt.name, func(t *testing.T) { + + queries := []*model.Query{ + { + SelectCommand: tt.query, + }, + } + pipeline := NewOptimizePipeline() + optimized, err := pipeline.Transform(queries) + if err != nil { + t.Fatalf("error optimizing query: %v", err) + } + + if len(optimized) != 1 { + t.Fatalf("expected 1 query, got %d", len(optimized)) + } + + var enabled bool + if optimized[0].OptimizeHints.Settings["use_query_cache"] != nil { + enabled = optimized[0].OptimizeHints.Settings["use_query_cache"].(bool) + } + + assert.Truef(t, enabled == tt.shouldCache, "expected use_query_cache to be %v, got %v", tt.shouldCache, enabled) + + }) + + } +} + +func Test_dateTrunc(t *testing.T) { + + date := func(s string) model.Expr { + return model.NewFunction("parseDateTime64BestEffort", model.NewLiteral(fmt.Sprintf("'%s'", s))) + } + + and := func(a, b model.Expr) model.Expr { + return model.NewInfixExpr(a, "and", b) + } + + lt := func(a, b model.Expr) model.Expr { + return model.NewInfixExpr(a, "<", b) + } + + gt := func(a, b model.Expr) model.Expr { + return model.NewInfixExpr(a, ">", b) + } + + col := func(s string) model.Expr { + return model.NewColumnRef(s) + } + + tests := []struct { + name string + query model.SelectCommand + expected model.SelectCommand + }{ + { + "select all", + model.SelectCommand{ + Columns: []model.Expr{model.NewColumnRef("*")}, + FromClause: model.NewTableRef("foo"), + }, + model.SelectCommand{ + Columns: []model.Expr{model.NewColumnRef("*")}, + FromClause: model.NewTableRef("foo"), + }, + }, + + { + "select all where date ", + model.SelectCommand{ + Columns: []model.Expr{model.NewColumnRef("*")}, + FromClause: model.NewTableRef("foo"), + WhereClause: date("2024-06-04T13:08:53.675Z"), + }, + model.SelectCommand{ + Columns: []model.Expr{model.NewColumnRef("*")}, + FromClause: model.NewTableRef("foo"), + WhereClause: date("2024-06-04T13:08:53.675Z"), + }, + }, + + { + "select all where and between dates (>24h)", + model.SelectCommand{ + Columns: []model.Expr{model.NewColumnRef("*")}, + FromClause: model.NewTableRef("foo"), + WhereClause: and(gt(col("a"), date("2024-06-04T13:08:53.675Z")), lt(col("a"), date("2024-06-06T13:10:53.675Z"))), + }, + model.SelectCommand{ + Columns: []model.Expr{model.NewColumnRef("*")}, + FromClause: model.NewTableRef("foo"), + WhereClause: and(gt(col("a"), date("2024-06-04T13:05:00Z")), lt(col("a"), date("2024-06-06T13:15:00Z"))), + }, + }, + + { + "select all where and between dates (<24h)", + model.SelectCommand{ + Columns: []model.Expr{model.NewColumnRef("*")}, + FromClause: model.NewTableRef("foo"), + WhereClause: and(gt(col("a"), date("2024-06-06T10:08:53.675Z")), lt(col("a"), date("2024-06-06T13:10:53.675Z"))), + }, + model.SelectCommand{ + Columns: []model.Expr{model.NewColumnRef("*")}, + FromClause: model.NewTableRef("foo"), + WhereClause: and(gt(col("a"), date("2024-06-06T10:08:53.675Z")), lt(col("a"), date("2024-06-06T13:10:53.675Z"))), + }, + }, + + { + "select a, count() from foo group by 1", + model.SelectCommand{ + Columns: []model.Expr{model.NewColumnRef("a"), model.NewFunction("count", model.NewColumnRef("*"))}, + FromClause: model.NewTableRef("foo"), + GroupBy: []model.Expr{model.NewLiteral(1)}, + }, + model.SelectCommand{ + Columns: []model.Expr{model.NewColumnRef("a"), model.NewFunction("count", model.NewColumnRef("*"))}, + FromClause: model.NewTableRef("foo"), + GroupBy: []model.Expr{model.NewLiteral(1)}, + }, + }, + // Add CTE here + } + + for _, tt := range tests { + + t.Run(tt.name, func(t *testing.T) { + + queries := []*model.Query{ + { + SelectCommand: tt.query, + }, + } + pipeline := NewOptimizePipeline() + optimized, err := pipeline.Transform(queries) + + if err != nil { + t.Fatalf("error optimizing query: %v", err) + } + + if len(optimized) != 1 { + t.Fatalf("expected 1 query, got %d", len(optimized)) + } + + assert.Equal(t, tt.expected, optimized[0].SelectCommand) + + }) + + } +} diff --git a/quesma/optimize/trunc_date.go b/quesma/optimize/trunc_date.go new file mode 100644 index 000000000..b34649597 --- /dev/null +++ b/quesma/optimize/trunc_date.go @@ -0,0 +1,276 @@ +// Copyright Quesma, licensed under the Elastic License 2.0. +// SPDX-License-Identifier: Elastic-2.0 +package optimize + +import ( + "fmt" + "quesma/model" + "strings" + "time" +) + +// truncateDateVisitor - a visitor that truncates dates in the query +// It finds date comparisons like: +// +// column >= '2024-06-04T13:08:53.675Z' and column <= '2024-06-06T13:10:53.675Z' +// +// and truncates the dates to the nearest 5 minutes (or any other duration), resulting in: +// +// column >= '2024-06-04T13:05:00.000Z' and column <= '2024-06-06T13:15:00.000Z' +// +// Note: Truncation is done only if the difference between the dates is more than 24 hours. + +type truncateDateVisitor struct { + truncateTo time.Duration + truncated bool +} + +func (v *truncateDateVisitor) visitChildren(args []model.Expr) []model.Expr { + var newArgs []model.Expr + for _, arg := range args { + if arg != nil { + newArgs = append(newArgs, arg.Accept(v).(model.Expr)) + } + } + return newArgs +} + +func (v *truncateDateVisitor) processDateNode(e model.Expr) (string, string, bool) { + + if fn, ok := e.(model.FunctionExpr); ok { + + if fn.Name == "parseDateTime64BestEffort" { + + if len(fn.Args) == 1 { + + //"2024-06-04T13:08:53.675Z" -> "2024-06-04T13:08:00.000Z" + + if date, ok := fn.Args[0].(model.LiteralExpr); ok { + if dateStr, ok := date.Value.(string); ok { + + dateStr = strings.Trim(dateStr, "'") + return fn.Name, dateStr, true + } + } + } + } + } + return "", "", false +} + +type dateComparisonNode struct { + column string + op string + fn string + date string +} + +func (v *truncateDateVisitor) processDateComparisonNode(e model.Expr) *dateComparisonNode { + + if op, ok := e.(model.InfixExpr); ok && (op.Op == ">" || op.Op == "<" || op.Op == ">=" || op.Op == "<=") { + if column, ok := op.Left.(model.ColumnRef); ok { + if fn, date, ok := v.processDateNode(op.Right); ok { + return &dateComparisonNode{column: column.ColumnName, op: op.Op, fn: fn, date: date} + } + } + } + return nil +} + +func (v *truncateDateVisitor) compare(column, op string, right model.Expr) model.Expr { + return model.NewInfixExpr(model.NewColumnRef(column), op, right) +} + +func (v *truncateDateVisitor) date(fn, str string) model.Expr { + return model.NewFunction(fn, model.NewLiteral(fmt.Sprintf("'%s'", str))) +} + +func (v *truncateDateVisitor) truncateDate(op string, t time.Time) string { + + truncatedDate := t.Truncate(v.truncateTo) + + if op == "<" || op == "<=" { + truncatedDate = truncatedDate.Add(v.truncateTo) + } + + return truncatedDate.Format(time.RFC3339) +} + +func (v *truncateDateVisitor) VisitLiteral(e model.LiteralExpr) interface{} { + return e +} + +// truncate - truncates the date if the difference between the dates is more than 24 hours +// returns nil if the truncation is not possible or not needed +func (v *truncateDateVisitor) truncate(e model.InfixExpr) interface{} { + + left := v.processDateComparisonNode(e.Left) + right := v.processDateComparisonNode(e.Right) + + if left != nil && right != nil { + + // check if the columns are the same, + if left.column == right.column { + + if leftTime, err := time.Parse(time.RFC3339, left.date); err == nil { + if rightTime, err := time.Parse(time.RFC3339, right.date); err == nil { + + duration := rightTime.Sub(leftTime).Abs() + + // if the duration is more than 24 hours, we can truncate the date + if duration > 24*time.Hour { + + newLeft := v.truncateDate(left.op, leftTime) + newRight := v.truncateDate(right.op, rightTime) + + v.truncated = true + + res := model.NewInfixExpr( + v.compare(left.column, left.op, v.date(left.fn, newLeft)), + e.Op, + v.compare(right.column, right.op, v.date(right.fn, newRight))) + + return res + } + } + } + } + } + + return nil +} + +func (v *truncateDateVisitor) VisitInfix(e model.InfixExpr) interface{} { + + op := strings.ToLower(e.Op) + if op == "and" { + truncatedExpr := v.truncate(e) + + if truncatedExpr != nil { + return truncatedExpr + } + } + + // no truncation + left := e.Left.Accept(v).(model.Expr) + right := e.Right.Accept(v).(model.Expr) + return model.NewInfixExpr(left, e.Op, right) +} + +func (v *truncateDateVisitor) VisitPrefixExpr(e model.PrefixExpr) interface{} { + args := v.visitChildren(e.Args) + return model.NewPrefixExpr(e.Op, args) + +} + +func (v *truncateDateVisitor) VisitFunction(e model.FunctionExpr) interface{} { + args := v.visitChildren(e.Args) + return model.NewFunction(e.Name, args...) +} + +func (v *truncateDateVisitor) VisitColumnRef(e model.ColumnRef) interface{} { + return e +} + +func (v *truncateDateVisitor) VisitNestedProperty(e model.NestedProperty) interface{} { + return model.NestedProperty{ + ColumnRef: e.ColumnRef.Accept(v).(model.ColumnRef), + PropertyName: e.PropertyName.Accept(v).(model.LiteralExpr), + } +} + +func (v *truncateDateVisitor) VisitArrayAccess(e model.ArrayAccess) interface{} { + return model.ArrayAccess{ + ColumnRef: e.ColumnRef.Accept(v).(model.ColumnRef), + Index: e.Index.Accept(v).(model.Expr), + } +} + +func (v *truncateDateVisitor) VisitMultiFunction(e model.MultiFunctionExpr) interface{} { + args := v.visitChildren(e.Args) + return model.MultiFunctionExpr{Name: e.Name, Args: args} +} + +func (v *truncateDateVisitor) VisitString(e model.StringExpr) interface{} { return e } + +func (v *truncateDateVisitor) VisitOrderByExpr(e model.OrderByExpr) interface{} { + exprs := v.visitChildren(e.Exprs) + return model.NewOrderByExpr(exprs, e.Direction) + +} +func (v *truncateDateVisitor) VisitDistinctExpr(e model.DistinctExpr) interface{} { + return model.NewDistinctExpr(e.Expr.Accept(v).(model.Expr)) +} +func (v *truncateDateVisitor) VisitTableRef(e model.TableRef) interface{} { + return model.NewTableRef(e.Name) +} +func (v *truncateDateVisitor) VisitAliasedExpr(e model.AliasedExpr) interface{} { + return model.NewAliasedExpr(e.Expr.Accept(v).(model.Expr), e.Alias) +} +func (v *truncateDateVisitor) VisitWindowFunction(e model.WindowFunction) interface{} { + return model.NewWindowFunction(e.Name, v.visitChildren(e.Args), v.visitChildren(e.PartitionBy), e.OrderBy.Accept(v).(model.OrderByExpr)) +} + +func (v *truncateDateVisitor) VisitSelectCommand(e model.SelectCommand) interface{} { + + // transformation + + var groupBy []model.Expr + + for _, expr := range e.GroupBy { + groupBy = append(groupBy, expr.Accept(v).(model.Expr)) + } + + var columns []model.Expr + for _, expr := range e.Columns { + columns = append(columns, expr.Accept(v).(model.Expr)) + } + + var fromClause model.Expr + if e.FromClause != nil { + fromClause = e.FromClause.Accept(v).(model.Expr) + } + + var whereClause model.Expr + if e.WhereClause != nil { + whereClause = e.WhereClause.Accept(v).(model.Expr) + } + + return model.NewSelectCommand(columns, groupBy, e.OrderBy, + fromClause, whereClause, e.Limit, e.SampleLimit, e.IsDistinct) + +} + +func (v *truncateDateVisitor) VisitParenExpr(p model.ParenExpr) interface{} { + var exprs []model.Expr + for _, expr := range p.Exprs { + exprs = append(exprs, expr.Accept(v).(model.Expr)) + } + return model.NewParenExpr(exprs...) +} + +func (v *truncateDateVisitor) VisitLambdaExpr(e model.LambdaExpr) interface{} { + return model.NewLambdaExpr(e.Args, e.Body.Accept(v).(model.Expr)) +} + +type truncateDate struct { + truncateTo time.Duration +} + +func (s *truncateDate) Transform(queries []*model.Query) ([]*model.Query, error) { + + for k, query := range queries { + visitor := &truncateDateVisitor{} + + visitor.truncateTo = s.truncateTo + + result := query.SelectCommand.Accept(visitor).(*model.SelectCommand) + + // this is just in case if there was no truncation, we keep the original query + if visitor.truncated && result != nil { + queries[k].SelectCommand = *result + query.OptimizeHints.OptimizationsPerformed = append(query.OptimizeHints.OptimizationsPerformed, "truncateDate") + } + } + return queries, nil +} diff --git a/quesma/queryparser/query_translator.go b/quesma/queryparser/query_translator.go index 88e278b91..952d99d3c 100644 --- a/quesma/queryparser/query_translator.go +++ b/quesma/queryparser/query_translator.go @@ -331,6 +331,7 @@ func (cw *ClickhouseQueryTranslator) MakeSearchResponse(queries []*model.Query, Relation: "gte", } } + return response } diff --git a/quesma/quesma/quesma.go b/quesma/quesma/quesma.go index ee5e1ab6f..7f0ee5649 100644 --- a/quesma/quesma/quesma.go +++ b/quesma/quesma/quesma.go @@ -126,6 +126,12 @@ func NewHttpProxy(phoneHomeAgent telemetry.PhoneHomeAgent, logManager *clickhous queryRunner.DateMathRenderer = queryparser.DateMathExpressionFormatLiteral + // TODO this should be configurable somehow + // + // tests should not be run with optimization enabled by default + // TODO: Enable it in YAML + // queryRunner.EnableQueryOptimization() + router := configureRouter(config, schemaRegistry, logManager, quesmaManagementConsole, phoneHomeAgent, queryRunner) return &Quesma{ telemetryAgent: phoneHomeAgent, diff --git a/quesma/quesma/search.go b/quesma/quesma/search.go index cc3d4f494..43b603a51 100644 --- a/quesma/quesma/search.go +++ b/quesma/quesma/search.go @@ -12,6 +12,7 @@ import ( "quesma/end_user_errors" "quesma/logger" "quesma/model" + "quesma/optimize" "quesma/plugins" "quesma/plugins/registry" "quesma/queryparser" @@ -68,6 +69,10 @@ type QueryRunner struct { schemaRegistry schema.Registry } +func (q *QueryRunner) EnableQueryOptimization() { + q.transformationPipeline.transformers = append(q.transformationPipeline.transformers, optimize.NewOptimizePipeline()) +} + func NewQueryRunner(lm *clickhouse.LogManager, cfg config.QuesmaConfiguration, im elasticsearch.IndexManagement, qmc *ui.QuesmaManagementConsole, schemaRegistry schema.Registry) *QueryRunner { ctx, cancel := context.WithCancel(context.Background()) @@ -79,7 +84,6 @@ func NewQueryRunner(lm *clickhouse.LogManager, cfg config.QuesmaConfiguration, i &SchemaCheckPass{cfg: cfg.IndexConfig, schemaRegistry: schemaRegistry, logManager: lm}, // this can be a part of another plugin }, }, schemaRegistry: schemaRegistry} - } func NewAsyncQueryContext(ctx context.Context, cancel context.CancelFunc, id string) *AsyncQueryContext { @@ -569,6 +573,13 @@ func (q *QueryRunner) searchWorkerCommon( } sql := query.SelectCommand.String() + // TODO we should return what optimizations were performed + // TODO translatedQueryBody should be a struct (sql, optimizations, query time, etc) + // + //if query.OptimizeHints != nil { + // sql = sql + "\n-- optimizations: " + strings.Join(query.OptimizeHints.OptimizationsPerformed, ", ") + "\n" + //} + logger.InfoWithCtx(ctx).Msgf("SQL: %s", sql) translatedQueryBody[i] = []byte(sql)