From 0a5ca5ee82a22fb615f863261ce58e4967d25990 Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Tue, 27 Feb 2024 17:44:54 -0500 Subject: [PATCH 01/19] Truncation vreplication_log message Signed-off-by: Matt Lord --- go/vt/binlog/binlogplayer/mock_dbclient.go | 7 ++ .../tabletmanager/vreplication/utils.go | 22 +++- .../tabletmanager/vreplication/utils_test.go | 104 ++++++++++++++++++ 3 files changed, 131 insertions(+), 2 deletions(-) create mode 100644 go/vt/vttablet/tabletmanager/vreplication/utils_test.go diff --git a/go/vt/binlog/binlogplayer/mock_dbclient.go b/go/vt/binlog/binlogplayer/mock_dbclient.go index abc170ed493..f8552734dd9 100644 --- a/go/vt/binlog/binlogplayer/mock_dbclient.go +++ b/go/vt/binlog/binlogplayer/mock_dbclient.go @@ -244,3 +244,10 @@ func (dc *MockDBClient) ExecuteFetchMulti(query string, maxrows int) ([]*sqltype } return results, nil } + +// RemoveInvariant can be used to customize the behavior of the mock client. +func (dc *MockDBClient) RemoveInvariant(query string) { + dc.expectMu.Lock() + defer dc.expectMu.Unlock() + delete(dc.invariants, query) +} diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils.go b/go/vt/vttablet/tabletmanager/vreplication/utils.go index 21c3a61c9f1..bac5e278ebe 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils.go @@ -98,6 +98,24 @@ func insertLog(dbClient *vdbClient, typ string, vreplID int32, state, message st query = fmt.Sprintf("update %s.vreplication_log set count = count + 1 where id = %d", sidecar.GetIdentifier(), id) } else { buf := sqlparser.NewTrackedBuffer(nil) + // The message column is a TEXT field and thus has a max length of 64KiB (2^16-1) so we truncate that if needed. + // See: https://dev.mysql.com/doc/refman/en/string-type-syntax.html and + // https://dev.mysql.com/doc/refman/en/storage-requirements.html#data-types-storage-reqs-strings + // We perform the truncation in the middle of the message as the end of the message is likely to be the most + // important part as it often explains WHY we e.g. failed to execute an INSERT in the workflow. + maxMessageLen := 65535 + truncationStr := fmt.Sprintf(" ... %s ... ", sqlparser.TruncationText) + if len(message) > maxMessageLen { + mid := (len(message) / 2) - len(truncationStr) + for mid > (maxMessageLen / 2) { + mid = mid / 2 + } + tail := (len(message) - (mid + len(truncationStr))) + 1 + log.Errorf("BEFORE:: Message length: %d, mid: %d, sub: %d", len(message), mid, tail) + message = fmt.Sprintf("%s%s%s", message[:mid], truncationStr, message[tail:]) + log.Errorf("AFTER:: Message length: %d, mid: %d, sub: %d", len(message), mid, tail) + log.Flush() + } buf.Myprintf("insert into %s.vreplication_log(vrepl_id, type, state, message) values(%s, %s, %s, %s)", sidecar.GetIdentifier(), strconv.Itoa(int(vreplID)), encodeString(typ), encodeString(state), encodeString(message)) query = buf.ParsedQuery().Query @@ -108,7 +126,7 @@ func insertLog(dbClient *vdbClient, typ string, vreplID int32, state, message st return nil } -// insertLogWithParams is called when a stream is created. The attributes of the stream are stored as a json string +// insertLogWithParams is called when a stream is created. The attributes of the stream are stored as a json string. func insertLogWithParams(dbClient *vdbClient, action string, vreplID int32, params map[string]string) error { var message string if params != nil { @@ -121,7 +139,7 @@ func insertLogWithParams(dbClient *vdbClient, action string, vreplID int32, para return nil } -// isUnrecoverableError returns true if vreplication cannot recover from the given error and should completely terminate +// isUnrecoverableError returns true if vreplication cannot recover from the given error and should completely terminate. func isUnrecoverableError(err error) bool { if err == nil { return false diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go new file mode 100644 index 00000000000..fbe2576af30 --- /dev/null +++ b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go @@ -0,0 +1,104 @@ +/* +Copyright 2021 The Vitess Authors. + +Licensed 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 vreplication + +import ( + "fmt" + "strings" + "testing" + + "github.com/stretchr/testify/require" + + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/vt/binlog/binlogplayer" + "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/sqlparser" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" +) + +func TestInsertLogTruncation(t *testing.T) { + dbClient := binlogplayer.NewMockDBClient(t) + defer dbClient.Close() + dbClient.RemoveInvariant("insert into _vt.vreplication_log") + stats := binlogplayer.NewStats() + defer stats.Stop() + vdbClient := newVDBClient(dbClient, stats) + defer vdbClient.Close() + vrID := int32(1) + typ := "Testing" + state := binlogdatapb.VReplicationWorkflowState_Error.String() + maxMessageLen := 65535 + truncationStr := fmt.Sprintf(" ... %s ... ", sqlparser.TruncationText) + + insertStmtf := "insert into _vt.vreplication_log(vrepl_id, type, state, message) values(%d, '%s', '%s', %s)" + + tests := []struct { + message string + expectTruncation bool + }{ + { + message: "Simple message that's not truncated", + }, + { + message: "Simple message that needs to be truncated " + strings.Repeat("a", 80000) + " cuz it's long", + expectTruncation: true, + }, + { + message: "Simple message that doesn't need to be truncated " + strings.Repeat("b", 64000) + " cuz it's not quite too long", + }, + { + message: "Message that is just barely short enough " + strings.Repeat("c", maxMessageLen-(len("Message that is just barely short enough ")+len(" so it doesn't get truncated"))) + " so it doesn't get truncated", + }, + { + message: "Message that is just barely too long " + strings.Repeat("d", maxMessageLen-(len("Message that is just barely too long ")+len(" so it gets truncated"))+1) + " so it gets truncated", + expectTruncation: true, + }, + { + message: "Super long message brosef wut r ya doin " + strings.Repeat("e", 60000) + strings.Repeat("f", 60000) + " so maybe don't do that to yourself and your friends", + expectTruncation: true, + }, + { + message: "Super duper long message brosef wut r ya doin " + strings.Repeat("g", 120602) + strings.Repeat("h", 120001) + " so maybe really don't do that to yourself and your friends", + expectTruncation: true, + }, + } + for _, tc := range tests { + t.Run("insertLog", func(t *testing.T) { + var messageOut string + if tc.expectTruncation { + log.Errorf("BEFORE:: Message length: %d", len(tc.message)) + mid := (len(tc.message) / 2) - len(truncationStr) + for mid > (maxMessageLen / 2) { + mid = mid / 2 + } + tail := (len(tc.message) - (mid + len(truncationStr))) + 1 + messageOut = fmt.Sprintf("%s%s%s", tc.message[:mid], truncationStr, tc.message[tail:]) + require.True(t, strings.HasPrefix(messageOut, tc.message[:10])) // Confirm we still have the same beginning + require.True(t, strings.HasSuffix(messageOut, tc.message[len(tc.message)-10:])) // Confirm we still have the same end + require.True(t, strings.Contains(messageOut, truncationStr)) // Confirm we have the truncation text + } else { + messageOut = tc.message + } + require.LessOrEqual(t, len(messageOut), 65535) + dbClient.ExpectRequest(fmt.Sprintf(insertStmtf, vrID, typ, state, encodeString(messageOut)), &sqltypes.Result{}, nil) + err := insertLog(vdbClient, typ, vrID, state, tc.message) + require.NoError(t, err) + dbClient.Wait() + }) + } +} From 8d01c204d2d59d9d3e41fe49ecda60a5457d70a4 Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Tue, 27 Feb 2024 18:25:03 -0500 Subject: [PATCH 02/19] Make vreplication log message failures non-fatal Signed-off-by: Matt Lord --- .../tabletmanager/vreplication/engine.go | 12 +++------ .../tabletmanager/vreplication/utils.go | 24 +++++++---------- .../tabletmanager/vreplication/utils_test.go | 3 +-- .../tabletmanager/vreplication/vcopier.go | 16 +++--------- .../vreplication/vplayer_flaky_test.go | 3 +-- .../tabletmanager/vreplication/vreplicator.go | 26 +++++-------------- 6 files changed, 25 insertions(+), 59 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vreplication/engine.go b/go/vt/vttablet/tabletmanager/vreplication/engine.go index 2b2937056ac..5a31ff62be7 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/engine.go +++ b/go/vt/vttablet/tabletmanager/vreplication/engine.go @@ -433,9 +433,7 @@ func (vre *Engine) exec(query string, runAsAdmin bool) (*sqltypes.Result, error) return nil, err } vre.controllers[id] = ct - if err := insertLogWithParams(vdbc, LogStreamCreate, id, params); err != nil { - return nil, err - } + insertLogWithParams(vdbc, LogStreamCreate, id, params) } return qr, nil case updateQuery: @@ -475,9 +473,7 @@ func (vre *Engine) exec(query string, runAsAdmin bool) (*sqltypes.Result, error) return nil, err } vre.controllers[id] = ct - if err := insertLog(vdbc, LogStateChange, id, params["state"], ""); err != nil { - return nil, err - } + insertLog(vdbc, LogStateChange, id, params["state"], "") } return qr, nil case deleteQuery: @@ -495,9 +491,7 @@ func (vre *Engine) exec(query string, runAsAdmin bool) (*sqltypes.Result, error) ct.Stop() delete(vre.controllers, id) } - if err := insertLogWithParams(vdbc, LogStreamDelete, id, nil); err != nil { - return nil, err - } + insertLogWithParams(vdbc, LogStreamDelete, id, nil) } if err := dbClient.Begin(); err != nil { return nil, err diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils.go b/go/vt/vttablet/tabletmanager/vreplication/utils.go index bac5e278ebe..75ad4b95552 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils.go @@ -82,16 +82,17 @@ func getLastLog(dbClient *vdbClient, vreplID int32) (id int64, typ, state, messa return id, typ, state, message, nil } -func insertLog(dbClient *vdbClient, typ string, vreplID int32, state, message string) error { +func insertLog(dbClient *vdbClient, typ string, vreplID int32, state, message string) { // getLastLog returns the last log for a stream. During insertion, if the type/state/message match we do not insert // a new log but increment the count. This prevents spamming of the log table in case the same message is logged continuously. id, _, lastLogState, lastLogMessage, err := getLastLog(dbClient, vreplID) if err != nil { - return err + log.Errorf("Could not insert vreplication_log record because we failed to get the last log record: %v", err) + return } if typ == LogStateChange && state == lastLogState { // handles case where current state is Running, controller restarts after an error and initializes the state Running - return nil + return } var query string if id > 0 && message == lastLogMessage { @@ -106,37 +107,30 @@ func insertLog(dbClient *vdbClient, typ string, vreplID int32, state, message st maxMessageLen := 65535 truncationStr := fmt.Sprintf(" ... %s ... ", sqlparser.TruncationText) if len(message) > maxMessageLen { - mid := (len(message) / 2) - len(truncationStr) + mid := ((len(message) / 2) - len(truncationStr)) - 1 for mid > (maxMessageLen / 2) { mid = mid / 2 } - tail := (len(message) - (mid + len(truncationStr))) + 1 - log.Errorf("BEFORE:: Message length: %d, mid: %d, sub: %d", len(message), mid, tail) + tail := (len(message) - mid + len(truncationStr)) message = fmt.Sprintf("%s%s%s", message[:mid], truncationStr, message[tail:]) - log.Errorf("AFTER:: Message length: %d, mid: %d, sub: %d", len(message), mid, tail) - log.Flush() } buf.Myprintf("insert into %s.vreplication_log(vrepl_id, type, state, message) values(%s, %s, %s, %s)", sidecar.GetIdentifier(), strconv.Itoa(int(vreplID)), encodeString(typ), encodeString(state), encodeString(message)) query = buf.ParsedQuery().Query } if _, err = dbClient.ExecuteFetch(query, 10000); err != nil { - return fmt.Errorf("could not insert into log table: %v: %v", query, err) + log.Errorf("Could not insert into vreplication_log table: %v: %v", query, err) } - return nil } // insertLogWithParams is called when a stream is created. The attributes of the stream are stored as a json string. -func insertLogWithParams(dbClient *vdbClient, action string, vreplID int32, params map[string]string) error { +func insertLogWithParams(dbClient *vdbClient, action string, vreplID int32, params map[string]string) { var message string if params != nil { obj, _ := json.Marshal(params) message = string(obj) } - if err := insertLog(dbClient, action, vreplID, params["state"], message); err != nil { - return err - } - return nil + insertLog(dbClient, action, vreplID, params["state"], message) } // isUnrecoverableError returns true if vreplication cannot recover from the given error and should completely terminate. diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go index fbe2576af30..b7adc21b364 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go @@ -96,8 +96,7 @@ func TestInsertLogTruncation(t *testing.T) { } require.LessOrEqual(t, len(messageOut), 65535) dbClient.ExpectRequest(fmt.Sprintf(insertStmtf, vrID, typ, state, encodeString(messageOut)), &sqltypes.Result{}, nil) - err := insertLog(vdbClient, typ, vrID, state, tc.message) - require.NoError(t, err) + insertLog(vdbClient, typ, vrID, state, tc.message) dbClient.Wait() }) } diff --git a/go/vt/vttablet/tabletmanager/vreplication/vcopier.go b/go/vt/vttablet/tabletmanager/vreplication/vcopier.go index dfe51f71dbd..d92522bdc80 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vcopier.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vcopier.go @@ -246,10 +246,7 @@ func (vc *vcopier) initTablesForCopy(ctx context.Context) error { if err := vc.vr.setState(binlogdatapb.VReplicationWorkflowState_Copying, ""); err != nil { return err } - if err := vc.vr.insertLog(LogCopyStart, fmt.Sprintf("Copy phase started for %d table(s)", - len(plan.TargetTables))); err != nil { - return err - } + vc.vr.insertLog(LogCopyStart, fmt.Sprintf("Copy phase started for %d table(s)", len(plan.TargetTables))) if vc.vr.supportsDeferredSecondaryKeys() { settings, err := binlogplayer.ReadVRSettings(vc.vr.dbClient, vc.vr.id) @@ -257,20 +254,15 @@ func (vc *vcopier) initTablesForCopy(ctx context.Context) error { return err } if settings.DeferSecondaryKeys { - if err := vc.vr.insertLog(LogCopyStart, fmt.Sprintf("Copy phase temporarily dropping secondary keys for %d table(s)", - len(plan.TargetTables))); err != nil { - return err - } + vc.vr.insertLog(LogCopyStart, fmt.Sprintf("Copy phase temporarily dropping secondary keys for %d table(s)", len(plan.TargetTables))) for _, tableName := range tableNames { if err := vc.vr.stashSecondaryKeys(ctx, tableName); err != nil { return err } } - if err := vc.vr.insertLog(LogCopyStart, + vc.vr.insertLog(LogCopyStart, fmt.Sprintf("Copy phase finished dropping secondary keys and saving post copy actions to restore them for %d table(s)", - len(plan.TargetTables))); err != nil { - return err - } + len(plan.TargetTables))) } } } else { diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go index d9b68d052c3..ac0691dc8bc 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go @@ -2799,8 +2799,7 @@ func TestVReplicationLogs(t *testing.T) { for _, want := range expected { t.Run("", func(t *testing.T) { - err = insertLog(vdbc, LogMessage, 1, binlogdatapb.VReplicationWorkflowState_Running.String(), "message1") - require.NoError(t, err) + insertLog(vdbc, LogMessage, 1, binlogdatapb.VReplicationWorkflowState_Running.String(), "message1") qr, err := env.Mysqld.FetchSuperQuery(context.Background(), query) require.NoError(t, err) require.Equal(t, want, fmt.Sprintf("%v", qr.Rows)) diff --git a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go index daa642c53af..67496ca37f9 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go @@ -288,9 +288,7 @@ func (vr *vreplicator) replicate(ctx context.Context) error { return err } if numTablesToCopy == 0 { - if err := vr.insertLog(LogCopyEnd, fmt.Sprintf("Copy phase completed at gtid %s", settings.StartPos)); err != nil { - return err - } + vr.insertLog(LogCopyEnd, fmt.Sprintf("Copy phase completed at gtid %s", settings.StartPos)) } } case settings.StartPos.IsZero(): @@ -464,14 +462,12 @@ func (vr *vreplicator) setMessage(message string) error { if _, err := vr.dbClient.Execute(query); err != nil { return fmt.Errorf("could not set message: %v: %v", query, err) } - if err := insertLog(vr.dbClient, LogMessage, vr.id, vr.state.String(), message); err != nil { - return err - } + insertLog(vr.dbClient, LogMessage, vr.id, vr.state.String(), message) return nil } -func (vr *vreplicator) insertLog(typ, message string) error { - return insertLog(vr.dbClient, typ, vr.id, vr.state.String(), message) +func (vr *vreplicator) insertLog(typ, message string) { + insertLog(vr.dbClient, typ, vr.id, vr.state.String(), message) } func (vr *vreplicator) setState(state binlogdatapb.VReplicationWorkflowState, message string) error { @@ -489,9 +485,7 @@ func (vr *vreplicator) setState(state binlogdatapb.VReplicationWorkflowState, me if state == vr.state { return nil } - if err := insertLog(vr.dbClient, LogStateChange, vr.id, state.String(), message); err != nil { - return err - } + insertLog(vr.dbClient, LogStateChange, vr.id, state.String(), message) vr.state = state return nil @@ -815,10 +809,7 @@ func (vr *vreplicator) execPostCopyActions(ctx context.Context, tableName string return nil } - if err := vr.insertLog(LogCopyStart, fmt.Sprintf("Executing %d post copy action(s) for %s table", - len(qr.Rows), tableName)); err != nil { - return err - } + vr.insertLog(LogCopyStart, fmt.Sprintf("Executing %d post copy action(s) for %s table", len(qr.Rows), tableName)) // Save our connection ID so we can use it to easily KILL any // running SQL action we may perform later if needed. @@ -1039,10 +1030,7 @@ func (vr *vreplicator) execPostCopyActions(ctx context.Context, tableName string } } - if err := vr.insertLog(LogCopyStart, fmt.Sprintf("Completed all post copy actions for %s table", - tableName)); err != nil { - return err - } + vr.insertLog(LogCopyStart, fmt.Sprintf("Completed all post copy actions for %s table", tableName)) return nil } From 36c7896e83ac46d39e57d88f756d4e0f66ea5aed Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Tue, 27 Feb 2024 18:34:18 -0500 Subject: [PATCH 03/19] Continue on with show command when there are leftover log records Signed-off-by: Matt Lord --- go/vt/vtctl/workflow/server.go | 3 ++- go/vt/vttablet/tabletmanager/vreplication/utils_test.go | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/go/vt/vtctl/workflow/server.go b/go/vt/vtctl/workflow/server.go index fc215b84c22..41873a5533c 100644 --- a/go/vt/vtctl/workflow/server.go +++ b/go/vt/vtctl/workflow/server.go @@ -923,7 +923,8 @@ ORDER BY if stream.Id > streamLog.StreamId { log.Warningf("Found stream log for nonexistent stream: %+v", streamLog) - break + // This can happen on manual/failed workflow cleanup so keep going. + continue } // stream.Id == streamLog.StreamId diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go index b7adc21b364..7b3698c53e7 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go @@ -1,5 +1,5 @@ /* -Copyright 2021 The Vitess Authors. +Copyright 2024 The Vitess Authors. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. From c0012c50fd37454ea66a991d1ab11ed33a810afd Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Tue, 27 Feb 2024 18:44:51 -0500 Subject: [PATCH 04/19] Fix errant changes Signed-off-by: Matt Lord --- go/vt/vttablet/tabletmanager/vreplication/utils.go | 4 ++-- go/vt/vttablet/tabletmanager/vreplication/utils_test.go | 2 -- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils.go b/go/vt/vttablet/tabletmanager/vreplication/utils.go index 75ad4b95552..789c034bfa3 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils.go @@ -107,11 +107,11 @@ func insertLog(dbClient *vdbClient, typ string, vreplID int32, state, message st maxMessageLen := 65535 truncationStr := fmt.Sprintf(" ... %s ... ", sqlparser.TruncationText) if len(message) > maxMessageLen { - mid := ((len(message) / 2) - len(truncationStr)) - 1 + mid := (len(message) / 2) - len(truncationStr) for mid > (maxMessageLen / 2) { mid = mid / 2 } - tail := (len(message) - mid + len(truncationStr)) + tail := (len(message) - (mid + len(truncationStr))) + 1 message = fmt.Sprintf("%s%s%s", message[:mid], truncationStr, message[tail:]) } buf.Myprintf("insert into %s.vreplication_log(vrepl_id, type, state, message) values(%s, %s, %s, %s)", diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go index 7b3698c53e7..cf296d5fb2c 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go @@ -25,7 +25,6 @@ import ( "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/binlog/binlogplayer" - "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/sqlparser" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" @@ -81,7 +80,6 @@ func TestInsertLogTruncation(t *testing.T) { t.Run("insertLog", func(t *testing.T) { var messageOut string if tc.expectTruncation { - log.Errorf("BEFORE:: Message length: %d", len(tc.message)) mid := (len(tc.message) / 2) - len(truncationStr) for mid > (maxMessageLen / 2) { mid = mid / 2 From 91a733e44e27cff11500aeb6d22fa8b23660a7fa Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Tue, 27 Feb 2024 18:49:29 -0500 Subject: [PATCH 05/19] Use a new const for the max length Signed-off-by: Matt Lord --- .../tabletmanager/vreplication/utils.go | 17 +++++++++-------- .../tabletmanager/vreplication/utils_test.go | 9 ++++----- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils.go b/go/vt/vttablet/tabletmanager/vreplication/utils.go index 789c034bfa3..a7af5c483cf 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils.go @@ -33,6 +33,11 @@ import ( const ( vreplicationLogTableName = "vreplication_log" + // This comes from the fact that the message column in the vreplication_log table is of type TEXT. + // See: go/vt/sidecardb/schema/vreplication/vreplication_log.sql + // https://dev.mysql.com/doc/refman/en/string-type-syntax.html and + // https://dev.mysql.com/doc/refman/en/storage-requirements.html#data-types-storage-reqs-strings + maxVReplicationLogMessageLen = 65535 ) const ( @@ -99,16 +104,12 @@ func insertLog(dbClient *vdbClient, typ string, vreplID int32, state, message st query = fmt.Sprintf("update %s.vreplication_log set count = count + 1 where id = %d", sidecar.GetIdentifier(), id) } else { buf := sqlparser.NewTrackedBuffer(nil) - // The message column is a TEXT field and thus has a max length of 64KiB (2^16-1) so we truncate that if needed. - // See: https://dev.mysql.com/doc/refman/en/string-type-syntax.html and - // https://dev.mysql.com/doc/refman/en/storage-requirements.html#data-types-storage-reqs-strings - // We perform the truncation in the middle of the message as the end of the message is likely to be the most - // important part as it often explains WHY we e.g. failed to execute an INSERT in the workflow. - maxMessageLen := 65535 + // We perform the truncation, if needed, in the middle of the message as the end of the message is likely to + // be the most important part as it often explains WHY we e.g. failed to execute an INSERT in the workflow. truncationStr := fmt.Sprintf(" ... %s ... ", sqlparser.TruncationText) - if len(message) > maxMessageLen { + if len(message) > maxVReplicationLogMessageLen { mid := (len(message) / 2) - len(truncationStr) - for mid > (maxMessageLen / 2) { + for mid > (maxVReplicationLogMessageLen / 2) { mid = mid / 2 } tail := (len(message) - (mid + len(truncationStr))) + 1 diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go index cf296d5fb2c..1982edd20d5 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go @@ -41,7 +41,6 @@ func TestInsertLogTruncation(t *testing.T) { vrID := int32(1) typ := "Testing" state := binlogdatapb.VReplicationWorkflowState_Error.String() - maxMessageLen := 65535 truncationStr := fmt.Sprintf(" ... %s ... ", sqlparser.TruncationText) insertStmtf := "insert into _vt.vreplication_log(vrepl_id, type, state, message) values(%d, '%s', '%s', %s)" @@ -61,10 +60,10 @@ func TestInsertLogTruncation(t *testing.T) { message: "Simple message that doesn't need to be truncated " + strings.Repeat("b", 64000) + " cuz it's not quite too long", }, { - message: "Message that is just barely short enough " + strings.Repeat("c", maxMessageLen-(len("Message that is just barely short enough ")+len(" so it doesn't get truncated"))) + " so it doesn't get truncated", + message: "Message that is just barely short enough " + strings.Repeat("c", maxVReplicationLogMessageLen-(len("Message that is just barely short enough ")+len(" so it doesn't get truncated"))) + " so it doesn't get truncated", }, { - message: "Message that is just barely too long " + strings.Repeat("d", maxMessageLen-(len("Message that is just barely too long ")+len(" so it gets truncated"))+1) + " so it gets truncated", + message: "Message that is just barely too long " + strings.Repeat("d", maxVReplicationLogMessageLen-(len("Message that is just barely too long ")+len(" so it gets truncated"))+1) + " so it gets truncated", expectTruncation: true, }, { @@ -81,7 +80,7 @@ func TestInsertLogTruncation(t *testing.T) { var messageOut string if tc.expectTruncation { mid := (len(tc.message) / 2) - len(truncationStr) - for mid > (maxMessageLen / 2) { + for mid > (maxVReplicationLogMessageLen / 2) { mid = mid / 2 } tail := (len(tc.message) - (mid + len(truncationStr))) + 1 @@ -92,7 +91,7 @@ func TestInsertLogTruncation(t *testing.T) { } else { messageOut = tc.message } - require.LessOrEqual(t, len(messageOut), 65535) + require.LessOrEqual(t, len(messageOut), maxVReplicationLogMessageLen) dbClient.ExpectRequest(fmt.Sprintf(insertStmtf, vrID, typ, state, encodeString(messageOut)), &sqltypes.Result{}, nil) insertLog(vdbClient, typ, vrID, state, tc.message) dbClient.Wait() From 4ffa282d2b87ad795413e4b36e84332e597f3529 Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Tue, 27 Feb 2024 19:03:30 -0500 Subject: [PATCH 06/19] Move truncation string to a var Signed-off-by: Matt Lord --- go/vt/vttablet/tabletmanager/vreplication/utils.go | 11 +++++++---- .../vttablet/tabletmanager/vreplication/utils_test.go | 10 ++++------ 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils.go b/go/vt/vttablet/tabletmanager/vreplication/utils.go index a7af5c483cf..07d28adb6fa 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils.go @@ -40,6 +40,10 @@ const ( maxVReplicationLogMessageLen = 65535 ) +// vrepliationLogTruncationStr is the string that is used to indicate that a message has been +// truncated, in the middle, before being inserted into the vreplication_log table. +var vrepliationLogTruncationStr = fmt.Sprintf(" ... %s ... ", sqlparser.TruncationText) + const ( // Enum values for type column in the vreplication_log table. @@ -106,14 +110,13 @@ func insertLog(dbClient *vdbClient, typ string, vreplID int32, state, message st buf := sqlparser.NewTrackedBuffer(nil) // We perform the truncation, if needed, in the middle of the message as the end of the message is likely to // be the most important part as it often explains WHY we e.g. failed to execute an INSERT in the workflow. - truncationStr := fmt.Sprintf(" ... %s ... ", sqlparser.TruncationText) if len(message) > maxVReplicationLogMessageLen { - mid := (len(message) / 2) - len(truncationStr) + mid := (len(message) / 2) - len(vrepliationLogTruncationStr) for mid > (maxVReplicationLogMessageLen / 2) { mid = mid / 2 } - tail := (len(message) - (mid + len(truncationStr))) + 1 - message = fmt.Sprintf("%s%s%s", message[:mid], truncationStr, message[tail:]) + tail := (len(message) - (mid + len(vrepliationLogTruncationStr))) + 1 + message = fmt.Sprintf("%s%s%s", message[:mid], vrepliationLogTruncationStr, message[tail:]) } buf.Myprintf("insert into %s.vreplication_log(vrepl_id, type, state, message) values(%s, %s, %s, %s)", sidecar.GetIdentifier(), strconv.Itoa(int(vreplID)), encodeString(typ), encodeString(state), encodeString(message)) diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go index 1982edd20d5..ca62c5b9371 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go @@ -25,7 +25,6 @@ import ( "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/binlog/binlogplayer" - "vitess.io/vitess/go/vt/sqlparser" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" ) @@ -41,7 +40,6 @@ func TestInsertLogTruncation(t *testing.T) { vrID := int32(1) typ := "Testing" state := binlogdatapb.VReplicationWorkflowState_Error.String() - truncationStr := fmt.Sprintf(" ... %s ... ", sqlparser.TruncationText) insertStmtf := "insert into _vt.vreplication_log(vrepl_id, type, state, message) values(%d, '%s', '%s', %s)" @@ -79,15 +77,15 @@ func TestInsertLogTruncation(t *testing.T) { t.Run("insertLog", func(t *testing.T) { var messageOut string if tc.expectTruncation { - mid := (len(tc.message) / 2) - len(truncationStr) + mid := (len(tc.message) / 2) - len(vrepliationLogTruncationStr) for mid > (maxVReplicationLogMessageLen / 2) { mid = mid / 2 } - tail := (len(tc.message) - (mid + len(truncationStr))) + 1 - messageOut = fmt.Sprintf("%s%s%s", tc.message[:mid], truncationStr, tc.message[tail:]) + tail := (len(tc.message) - (mid + len(vrepliationLogTruncationStr))) + 1 + messageOut = fmt.Sprintf("%s%s%s", tc.message[:mid], vrepliationLogTruncationStr, tc.message[tail:]) require.True(t, strings.HasPrefix(messageOut, tc.message[:10])) // Confirm we still have the same beginning require.True(t, strings.HasSuffix(messageOut, tc.message[len(tc.message)-10:])) // Confirm we still have the same end - require.True(t, strings.Contains(messageOut, truncationStr)) // Confirm we have the truncation text + require.True(t, strings.Contains(messageOut, vrepliationLogTruncationStr)) // Confirm we have the truncation text } else { messageOut = tc.message } From 186cd268ae575517a960bfa698e90286ff2643ae Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Tue, 27 Feb 2024 19:31:42 -0500 Subject: [PATCH 07/19] Add test log output Signed-off-by: Matt Lord --- go/vt/vttablet/tabletmanager/vreplication/utils_test.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go index ca62c5b9371..d0709690e5b 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go @@ -32,7 +32,7 @@ import ( func TestInsertLogTruncation(t *testing.T) { dbClient := binlogplayer.NewMockDBClient(t) defer dbClient.Close() - dbClient.RemoveInvariant("insert into _vt.vreplication_log") + dbClient.RemoveInvariant("insert into _vt.vreplication_log") // Otherwise the insert will be ignored stats := binlogplayer.NewStats() defer stats.Stop() vdbClient := newVDBClient(dbClient, stats) @@ -83,9 +83,10 @@ func TestInsertLogTruncation(t *testing.T) { } tail := (len(tc.message) - (mid + len(vrepliationLogTruncationStr))) + 1 messageOut = fmt.Sprintf("%s%s%s", tc.message[:mid], vrepliationLogTruncationStr, tc.message[tail:]) - require.True(t, strings.HasPrefix(messageOut, tc.message[:10])) // Confirm we still have the same beginning - require.True(t, strings.HasSuffix(messageOut, tc.message[len(tc.message)-10:])) // Confirm we still have the same end - require.True(t, strings.Contains(messageOut, vrepliationLogTruncationStr)) // Confirm we have the truncation text + require.True(t, strings.HasPrefix(messageOut, tc.message[:1024])) // Confirm we still have the same beginning + require.True(t, strings.HasSuffix(messageOut, tc.message[len(tc.message)-1024:])) // Confirm we still have the same end + require.True(t, strings.Contains(messageOut, vrepliationLogTruncationStr)) // Confirm we have the truncation text + t.Logf("Original message length: %d, truncated message length: %d", len(tc.message), len(messageOut)) } else { messageOut = tc.message } From 64502bf3809be57ba1e6444e115b3b7463390890 Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Wed, 28 Feb 2024 09:52:27 -0500 Subject: [PATCH 08/19] Move truncation to textutil function Signed-off-by: Matt Lord --- go/textutil/strings.go | 28 +++++++ go/textutil/strings_test.go | 76 +++++++++++++++++++ go/vt/binlog/binlogplayer/mock_dbclient.go | 7 ++ .../tabletmanager/vreplication/utils.go | 12 +-- .../tabletmanager/vreplication/utils_test.go | 11 +-- 5 files changed, 121 insertions(+), 13 deletions(-) diff --git a/go/textutil/strings.go b/go/textutil/strings.go index ac35541f52f..0b7f0b66095 100644 --- a/go/textutil/strings.go +++ b/go/textutil/strings.go @@ -17,6 +17,7 @@ limitations under the License. package textutil import ( + "fmt" "net/url" "regexp" "strings" @@ -28,6 +29,13 @@ import ( topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) +type TruncationLocation int + +const ( + TruncationLocationMiddle TruncationLocation = iota + TruncationLocationEnd +) + var ( delimitedListRegexp = regexp.MustCompile(`[ ,;]+`) SimulatedNullString = sqltypes.NULL.String() @@ -133,3 +141,23 @@ func Title(s string) string { }, s) } + +func TruncateText(text string, maxLen int, location TruncationLocation, truncationIndicator string) (string, error) { + if len(truncationIndicator)+2 >= maxLen { + return "", fmt.Errorf("the truncation indicator is too long for the provided text") + } + ol := len(text) + if ol <= maxLen { + return text, nil + } + switch location { + case TruncationLocationMiddle: + prefix := int((float64(maxLen) * 0.5) - float64(len(truncationIndicator))) + suffix := int(ol - int(prefix+len(truncationIndicator)) + 1) + return fmt.Sprintf("%s%s%s", text[:prefix], truncationIndicator, text[suffix:]), nil + case TruncationLocationEnd: + return text[:maxLen-len(truncationIndicator)] + truncationIndicator, nil + default: + return "", fmt.Errorf("invalid truncation location: %d", location) + } +} diff --git a/go/textutil/strings_test.go b/go/textutil/strings_test.go index 2b43166831c..ce20cf19f1c 100644 --- a/go/textutil/strings_test.go +++ b/go/textutil/strings_test.go @@ -17,9 +17,11 @@ limitations under the License. package textutil import ( + "strings" "testing" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" topodatapb "vitess.io/vitess/go/vt/proto/topodata" @@ -208,3 +210,77 @@ func TestTitle(t *testing.T) { }) } } + +func TestTruncateText(t *testing.T) { + defaultLocation := TruncationLocationMiddle + defaultMaxLen := 100 + defaultTruncationIndicator := "..." + + tests := []struct { + name string + text string + maxLen int + location TruncationLocation + truncationIndicator string + want string + wantErr string + }{ + { + name: "no truncation", + text: "hello world", + maxLen: defaultMaxLen, + location: defaultLocation, + want: "hello world", + }, + { + name: "no truncation - exact", + text: strings.Repeat("a", defaultMaxLen), + maxLen: defaultMaxLen, + location: defaultLocation, + want: strings.Repeat("a", defaultMaxLen), + }, + { + name: "barely too long - mid", + text: strings.Repeat("a", defaultMaxLen+1), + truncationIndicator: defaultTruncationIndicator, + maxLen: defaultMaxLen, + location: defaultLocation, + want: "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa...aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa", + }, + { + name: "barely too long - end", + text: strings.Repeat("a", defaultMaxLen+1), + truncationIndicator: defaultTruncationIndicator, + maxLen: defaultMaxLen, + location: TruncationLocationEnd, + want: "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa...", + }, + { + name: "too small", + text: strings.Repeat("a", defaultMaxLen), + truncationIndicator: defaultTruncationIndicator, + maxLen: 4, + location: defaultLocation, + wantErr: "the truncation indicator is too long for the provided text", + }, + { + name: "bad location", + text: strings.Repeat("a", defaultMaxLen+1), + truncationIndicator: defaultTruncationIndicator, + maxLen: defaultMaxLen, + location: 100, + wantErr: "invalid truncation location: 100", + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + val, err := TruncateText(tt.text, tt.maxLen, tt.location, tt.truncationIndicator) + if tt.wantErr != "" { + require.EqualError(t, err, tt.wantErr) + } else { + require.NoError(t, err) + require.Equal(t, tt.want, val) + } + }) + } +} diff --git a/go/vt/binlog/binlogplayer/mock_dbclient.go b/go/vt/binlog/binlogplayer/mock_dbclient.go index f8552734dd9..e89c630b4d8 100644 --- a/go/vt/binlog/binlogplayer/mock_dbclient.go +++ b/go/vt/binlog/binlogplayer/mock_dbclient.go @@ -245,6 +245,13 @@ func (dc *MockDBClient) ExecuteFetchMulti(query string, maxrows int) ([]*sqltype return results, nil } +// AddInvariant can be used to customize the behavior of the mock client. +func (dc *MockDBClient) AddInvariant(query string, result *sqltypes.Result) { + dc.expectMu.Lock() + defer dc.expectMu.Unlock() + dc.invariants[query] = result +} + // RemoveInvariant can be used to customize the behavior of the mock client. func (dc *MockDBClient) RemoveInvariant(query string) { dc.expectMu.Lock() diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils.go b/go/vt/vttablet/tabletmanager/vreplication/utils.go index 07d28adb6fa..e53bad4b6d6 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils.go @@ -24,6 +24,7 @@ import ( "vitess.io/vitess/go/constants/sidecar" "vitess.io/vitess/go/mysql/sqlerror" "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/textutil" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vterrors" @@ -42,7 +43,7 @@ const ( // vrepliationLogTruncationStr is the string that is used to indicate that a message has been // truncated, in the middle, before being inserted into the vreplication_log table. -var vrepliationLogTruncationStr = fmt.Sprintf(" ... %s ... ", sqlparser.TruncationText) +var vreplicationLogTruncationStr = fmt.Sprintf(" ... %s ... ", sqlparser.TruncationText) const ( // Enum values for type column in the vreplication_log table. @@ -111,12 +112,11 @@ func insertLog(dbClient *vdbClient, typ string, vreplID int32, state, message st // We perform the truncation, if needed, in the middle of the message as the end of the message is likely to // be the most important part as it often explains WHY we e.g. failed to execute an INSERT in the workflow. if len(message) > maxVReplicationLogMessageLen { - mid := (len(message) / 2) - len(vrepliationLogTruncationStr) - for mid > (maxVReplicationLogMessageLen / 2) { - mid = mid / 2 + message, err = textutil.TruncateText(message, maxVReplicationLogMessageLen, textutil.TruncationLocationMiddle, vreplicationLogTruncationStr) + if err != nil { + log.Errorf("Could not insert vreplication_log record because we failed to truncate the message: %v", err) + return } - tail := (len(message) - (mid + len(vrepliationLogTruncationStr))) + 1 - message = fmt.Sprintf("%s%s%s", message[:mid], vrepliationLogTruncationStr, message[tail:]) } buf.Myprintf("insert into %s.vreplication_log(vrepl_id, type, state, message) values(%s, %s, %s, %s)", sidecar.GetIdentifier(), strconv.Itoa(int(vreplID)), encodeString(typ), encodeString(state), encodeString(message)) diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go index d0709690e5b..ed312146fd5 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go @@ -24,6 +24,7 @@ import ( "github.com/stretchr/testify/require" "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/textutil" "vitess.io/vitess/go/vt/binlog/binlogplayer" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" @@ -77,15 +78,11 @@ func TestInsertLogTruncation(t *testing.T) { t.Run("insertLog", func(t *testing.T) { var messageOut string if tc.expectTruncation { - mid := (len(tc.message) / 2) - len(vrepliationLogTruncationStr) - for mid > (maxVReplicationLogMessageLen / 2) { - mid = mid / 2 - } - tail := (len(tc.message) - (mid + len(vrepliationLogTruncationStr))) + 1 - messageOut = fmt.Sprintf("%s%s%s", tc.message[:mid], vrepliationLogTruncationStr, tc.message[tail:]) + messageOut, err := textutil.TruncateText(tc.message, maxVReplicationLogMessageLen, textutil.TruncationLocationMiddle, vreplicationLogTruncationStr) + require.NoError(t, err) require.True(t, strings.HasPrefix(messageOut, tc.message[:1024])) // Confirm we still have the same beginning require.True(t, strings.HasSuffix(messageOut, tc.message[len(tc.message)-1024:])) // Confirm we still have the same end - require.True(t, strings.Contains(messageOut, vrepliationLogTruncationStr)) // Confirm we have the truncation text + require.True(t, strings.Contains(messageOut, vreplicationLogTruncationStr)) // Confirm we have the truncation text t.Logf("Original message length: %d, truncated message length: %d", len(tc.message), len(messageOut)) } else { messageOut = tc.message From 4296f8a6e5e5623e58d9d9c31d9cd8bc0d85554b Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Wed, 28 Feb 2024 10:30:05 -0500 Subject: [PATCH 09/19] Use new truncation for vreplication.message too Signed-off-by: Matt Lord --- go/vt/vttablet/tabletmanager/vreplication/utils.go | 5 ++--- go/vt/vttablet/tabletmanager/vreplication/utils_test.go | 7 +++++-- go/vt/vttablet/tabletmanager/vreplication/vreplicator.go | 8 ++++++-- 3 files changed, 13 insertions(+), 7 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils.go b/go/vt/vttablet/tabletmanager/vreplication/utils.go index e53bad4b6d6..cd12e4cc9e5 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils.go @@ -35,10 +35,9 @@ import ( const ( vreplicationLogTableName = "vreplication_log" // This comes from the fact that the message column in the vreplication_log table is of type TEXT. - // See: go/vt/sidecardb/schema/vreplication/vreplication_log.sql - // https://dev.mysql.com/doc/refman/en/string-type-syntax.html and - // https://dev.mysql.com/doc/refman/en/storage-requirements.html#data-types-storage-reqs-strings maxVReplicationLogMessageLen = 65535 + // This comes from the fact that the message column in the vreplication table is varbinary(1000). + maxVReplicationMessageLen = 1000 ) // vrepliationLogTruncationStr is the string that is used to indicate that a message has been diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go index ed312146fd5..5eb9d7ddd34 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go @@ -76,9 +76,12 @@ func TestInsertLogTruncation(t *testing.T) { } for _, tc := range tests { t.Run("insertLog", func(t *testing.T) { - var messageOut string + var ( + messageOut string + err error + ) if tc.expectTruncation { - messageOut, err := textutil.TruncateText(tc.message, maxVReplicationLogMessageLen, textutil.TruncationLocationMiddle, vreplicationLogTruncationStr) + messageOut, err = textutil.TruncateText(tc.message, maxVReplicationLogMessageLen, textutil.TruncationLocationMiddle, vreplicationLogTruncationStr) require.NoError(t, err) require.True(t, strings.HasPrefix(messageOut, tc.message[:1024])) // Confirm we still have the same beginning require.True(t, strings.HasSuffix(messageOut, tc.message[len(tc.message)-1024:])) // Confirm we still have the same end diff --git a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go index 67496ca37f9..224a1141d77 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go @@ -29,6 +29,7 @@ import ( "vitess.io/vitess/go/mysql/replication" "vitess.io/vitess/go/mysql/sqlerror" "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/textutil" "vitess.io/vitess/go/timer" "vitess.io/vitess/go/vt/binlog/binlogplayer" "vitess.io/vitess/go/vt/log" @@ -450,8 +451,11 @@ func (vr *vreplicator) readSettings(ctx context.Context, dbClient *vdbClient) (s return settings, numTablesToCopy, nil } -func (vr *vreplicator) setMessage(message string) error { - message = binlogplayer.MessageTruncate(message) +func (vr *vreplicator) setMessage(message string) (err error) { + message, err = textutil.TruncateText(message, maxVReplicationMessageLen, textutil.TruncationLocationMiddle, vreplicationLogTruncationStr) + if err != nil { + return err + } vr.stats.History.Add(&binlogplayer.StatsHistoryRecord{ Time: time.Now(), Message: message, From 1f808645b0e4b65042b2ebf437257d0f7387d80e Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Wed, 28 Feb 2024 10:41:04 -0500 Subject: [PATCH 10/19] Minor changes from self review Signed-off-by: Matt Lord --- go/textutil/strings.go | 9 ++++++--- go/textutil/strings_test.go | 1 + go/vt/vttablet/tabletmanager/vreplication/utils.go | 7 +++++-- go/vt/vttablet/tabletmanager/vreplication/utils_test.go | 2 +- go/vt/vttablet/tabletmanager/vreplication/vreplicator.go | 2 +- 5 files changed, 14 insertions(+), 7 deletions(-) diff --git a/go/textutil/strings.go b/go/textutil/strings.go index 0b7f0b66095..33b5bd5dc7e 100644 --- a/go/textutil/strings.go +++ b/go/textutil/strings.go @@ -142,14 +142,17 @@ func Title(s string) string { s) } +// TruncateText truncates the provided text to the specified length using the +// provided indicator in place of the truncated text in the specified location +// of the original. func TruncateText(text string, maxLen int, location TruncationLocation, truncationIndicator string) (string, error) { - if len(truncationIndicator)+2 >= maxLen { - return "", fmt.Errorf("the truncation indicator is too long for the provided text") - } ol := len(text) if ol <= maxLen { return text, nil } + if len(truncationIndicator)+2 >= maxLen { + return "", fmt.Errorf("the truncation indicator is too long for the provided text") + } switch location { case TruncationLocationMiddle: prefix := int((float64(maxLen) * 0.5) - float64(len(truncationIndicator))) diff --git a/go/textutil/strings_test.go b/go/textutil/strings_test.go index ce20cf19f1c..ea940605fd1 100644 --- a/go/textutil/strings_test.go +++ b/go/textutil/strings_test.go @@ -280,6 +280,7 @@ func TestTruncateText(t *testing.T) { } else { require.NoError(t, err) require.Equal(t, tt.want, val) + require.LessOrEqual(t, len(val), tt.maxLen) } }) } diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils.go b/go/vt/vttablet/tabletmanager/vreplication/utils.go index cd12e4cc9e5..6ea647608a2 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils.go @@ -34,6 +34,9 @@ import ( const ( vreplicationLogTableName = "vreplication_log" + // Truncate values in the middle to preserve the end of the message which typically contains the + // error text. + truncationLocation = textutil.TruncationLocationMiddle // This comes from the fact that the message column in the vreplication_log table is of type TEXT. maxVReplicationLogMessageLen = 65535 // This comes from the fact that the message column in the vreplication table is varbinary(1000). @@ -41,7 +44,7 @@ const ( ) // vrepliationLogTruncationStr is the string that is used to indicate that a message has been -// truncated, in the middle, before being inserted into the vreplication_log table. +// truncated before being inserted into one of the vreplication sidecar tables. var vreplicationLogTruncationStr = fmt.Sprintf(" ... %s ... ", sqlparser.TruncationText) const ( @@ -111,7 +114,7 @@ func insertLog(dbClient *vdbClient, typ string, vreplID int32, state, message st // We perform the truncation, if needed, in the middle of the message as the end of the message is likely to // be the most important part as it often explains WHY we e.g. failed to execute an INSERT in the workflow. if len(message) > maxVReplicationLogMessageLen { - message, err = textutil.TruncateText(message, maxVReplicationLogMessageLen, textutil.TruncationLocationMiddle, vreplicationLogTruncationStr) + message, err = textutil.TruncateText(message, maxVReplicationLogMessageLen, truncationLocation, vreplicationLogTruncationStr) if err != nil { log.Errorf("Could not insert vreplication_log record because we failed to truncate the message: %v", err) return diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go index 5eb9d7ddd34..dd0bec4c6c1 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go @@ -81,7 +81,7 @@ func TestInsertLogTruncation(t *testing.T) { err error ) if tc.expectTruncation { - messageOut, err = textutil.TruncateText(tc.message, maxVReplicationLogMessageLen, textutil.TruncationLocationMiddle, vreplicationLogTruncationStr) + messageOut, err = textutil.TruncateText(tc.message, maxVReplicationLogMessageLen, truncationLocation, vreplicationLogTruncationStr) require.NoError(t, err) require.True(t, strings.HasPrefix(messageOut, tc.message[:1024])) // Confirm we still have the same beginning require.True(t, strings.HasSuffix(messageOut, tc.message[len(tc.message)-1024:])) // Confirm we still have the same end diff --git a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go index 224a1141d77..deb23aeb3f9 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go @@ -452,7 +452,7 @@ func (vr *vreplicator) readSettings(ctx context.Context, dbClient *vdbClient) (s } func (vr *vreplicator) setMessage(message string) (err error) { - message, err = textutil.TruncateText(message, maxVReplicationMessageLen, textutil.TruncationLocationMiddle, vreplicationLogTruncationStr) + message, err = textutil.TruncateText(message, maxVReplicationMessageLen, truncationLocation, vreplicationLogTruncationStr) if err != nil { return err } From 1a603b932b833f9a3bcfa4db18f43f5e3c2fca45 Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Wed, 28 Feb 2024 11:21:28 -0500 Subject: [PATCH 11/19] Bug fix in end truncation Signed-off-by: Matt Lord --- go/textutil/strings.go | 2 +- go/textutil/strings_test.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/go/textutil/strings.go b/go/textutil/strings.go index 33b5bd5dc7e..ddbdac9c318 100644 --- a/go/textutil/strings.go +++ b/go/textutil/strings.go @@ -159,7 +159,7 @@ func TruncateText(text string, maxLen int, location TruncationLocation, truncati suffix := int(ol - int(prefix+len(truncationIndicator)) + 1) return fmt.Sprintf("%s%s%s", text[:prefix], truncationIndicator, text[suffix:]), nil case TruncationLocationEnd: - return text[:maxLen-len(truncationIndicator)] + truncationIndicator, nil + return text[:maxLen-(len(truncationIndicator)+1)] + truncationIndicator, nil default: return "", fmt.Errorf("invalid truncation location: %d", location) } diff --git a/go/textutil/strings_test.go b/go/textutil/strings_test.go index ea940605fd1..2ba9851b71c 100644 --- a/go/textutil/strings_test.go +++ b/go/textutil/strings_test.go @@ -253,7 +253,7 @@ func TestTruncateText(t *testing.T) { truncationIndicator: defaultTruncationIndicator, maxLen: defaultMaxLen, location: TruncationLocationEnd, - want: "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa...", + want: "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa...", }, { name: "too small", From 647de05008fa393ab2d9cb98d9089e30fcba09cc Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Wed, 28 Feb 2024 12:06:04 -0500 Subject: [PATCH 12/19] Move related truncation method over as well Signed-off-by: Matt Lord --- go/vt/binlog/binlogplayer/dbclient.go | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/go/vt/binlog/binlogplayer/dbclient.go b/go/vt/binlog/binlogplayer/dbclient.go index bc96e690b76..9902beb0af5 100644 --- a/go/vt/binlog/binlogplayer/dbclient.go +++ b/go/vt/binlog/binlogplayer/dbclient.go @@ -25,6 +25,7 @@ import ( "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/mysql/sqlerror" "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/textutil" "vitess.io/vitess/go/vt/dbconfigs" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/sqlparser" @@ -129,10 +130,14 @@ func LogError(msg string, err error) { // LimitString truncates string to specified size func LimitString(s string, limit int) string { - if len(s) > limit { + ls, err := textutil.TruncateText(s, limit, textutil.TruncationLocationMiddle, "...") + if err != nil { // Fallback to simple truncation + if len(s) <= limit { + return s + } return s[:limit] } - return s + return ls } func (dc *dbClientImpl) ExecuteFetch(query string, maxrows int) (*sqltypes.Result, error) { From e1de47c2ee6c25bf9b11956f716d56901d5c80e1 Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Wed, 28 Feb 2024 12:12:28 -0500 Subject: [PATCH 13/19] Correct var name typo (non functional change) Signed-off-by: Matt Lord --- go/vt/binlog/binlogplayer/dbclient.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/go/vt/binlog/binlogplayer/dbclient.go b/go/vt/binlog/binlogplayer/dbclient.go index 9902beb0af5..14d80e93b06 100644 --- a/go/vt/binlog/binlogplayer/dbclient.go +++ b/go/vt/binlog/binlogplayer/dbclient.go @@ -130,14 +130,14 @@ func LogError(msg string, err error) { // LimitString truncates string to specified size func LimitString(s string, limit int) string { - ls, err := textutil.TruncateText(s, limit, textutil.TruncationLocationMiddle, "...") + ts, err := textutil.TruncateText(s, limit, textutil.TruncationLocationMiddle, "...") if err != nil { // Fallback to simple truncation if len(s) <= limit { return s } return s[:limit] } - return ls + return ts } func (dc *dbClientImpl) ExecuteFetch(query string, maxrows int) (*sqltypes.Result, error) { From 5ed22d816e114803126da28163e2ee2479f9a3fe Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Wed, 28 Feb 2024 12:18:45 -0500 Subject: [PATCH 14/19] Nitting myself to deth Signed-off-by: Matt Lord --- go/textutil/strings.go | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/go/textutil/strings.go b/go/textutil/strings.go index ddbdac9c318..1b22231afa2 100644 --- a/go/textutil/strings.go +++ b/go/textutil/strings.go @@ -142,24 +142,24 @@ func Title(s string) string { s) } -// TruncateText truncates the provided text to the specified length using the -// provided indicator in place of the truncated text in the specified location -// of the original. -func TruncateText(text string, maxLen int, location TruncationLocation, truncationIndicator string) (string, error) { +// TruncateText truncates the provided text, if needed, to the specified maximum +// length using the provided truncation indicator in place of the truncated text +// in the specified location of the original string. +func TruncateText(text string, limit int, location TruncationLocation, indicator string) (string, error) { ol := len(text) - if ol <= maxLen { + if ol <= limit { return text, nil } - if len(truncationIndicator)+2 >= maxLen { + if len(indicator)+2 >= limit { return "", fmt.Errorf("the truncation indicator is too long for the provided text") } switch location { case TruncationLocationMiddle: - prefix := int((float64(maxLen) * 0.5) - float64(len(truncationIndicator))) - suffix := int(ol - int(prefix+len(truncationIndicator)) + 1) - return fmt.Sprintf("%s%s%s", text[:prefix], truncationIndicator, text[suffix:]), nil + prefix := int((float64(limit) * 0.5) - float64(len(indicator))) + suffix := int(ol - int(prefix+len(indicator)) + 1) + return fmt.Sprintf("%s%s%s", text[:prefix], indicator, text[suffix:]), nil case TruncationLocationEnd: - return text[:maxLen-(len(truncationIndicator)+1)] + truncationIndicator, nil + return text[:limit-(len(indicator)+1)] + indicator, nil default: return "", fmt.Errorf("invalid truncation location: %d", location) } From c5198d5d0d2e3baf5212ed1964a31d1b12c74753 Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Wed, 28 Feb 2024 12:23:45 -0500 Subject: [PATCH 15/19] Reduce changes Signed-off-by: Matt Lord --- go/vt/vttablet/tabletmanager/vreplication/utils.go | 2 -- go/vt/vttablet/tabletmanager/vreplication/vreplicator.go | 6 +----- 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils.go b/go/vt/vttablet/tabletmanager/vreplication/utils.go index 6ea647608a2..d31d3365920 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils.go @@ -39,8 +39,6 @@ const ( truncationLocation = textutil.TruncationLocationMiddle // This comes from the fact that the message column in the vreplication_log table is of type TEXT. maxVReplicationLogMessageLen = 65535 - // This comes from the fact that the message column in the vreplication table is varbinary(1000). - maxVReplicationMessageLen = 1000 ) // vrepliationLogTruncationStr is the string that is used to indicate that a message has been diff --git a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go index deb23aeb3f9..8a01cf7c8ed 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go @@ -29,7 +29,6 @@ import ( "vitess.io/vitess/go/mysql/replication" "vitess.io/vitess/go/mysql/sqlerror" "vitess.io/vitess/go/sqltypes" - "vitess.io/vitess/go/textutil" "vitess.io/vitess/go/timer" "vitess.io/vitess/go/vt/binlog/binlogplayer" "vitess.io/vitess/go/vt/log" @@ -452,10 +451,7 @@ func (vr *vreplicator) readSettings(ctx context.Context, dbClient *vdbClient) (s } func (vr *vreplicator) setMessage(message string) (err error) { - message, err = textutil.TruncateText(message, maxVReplicationMessageLen, truncationLocation, vreplicationLogTruncationStr) - if err != nil { - return err - } + message = binlogplayer.MessageTruncate(message) vr.stats.History.Add(&binlogplayer.StatsHistoryRecord{ Time: time.Now(), Message: message, From 021f06033ed560395eb22f91621f542ef2e272c1 Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Wed, 28 Feb 2024 12:30:10 -0500 Subject: [PATCH 16/19] Unify truncation indicator Signed-off-by: Matt Lord --- go/vt/binlog/binlogplayer/binlog_player.go | 3 +++ go/vt/vttablet/tabletmanager/vreplication/utils.go | 7 ++----- go/vt/vttablet/tabletmanager/vreplication/utils_test.go | 4 ++-- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/go/vt/binlog/binlogplayer/binlog_player.go b/go/vt/binlog/binlogplayer/binlog_player.go index 6bdc2d70d2d..c415029424f 100644 --- a/go/vt/binlog/binlogplayer/binlog_player.go +++ b/go/vt/binlog/binlogplayer/binlog_player.go @@ -45,6 +45,7 @@ import ( "vitess.io/vitess/go/stats" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/servenv" + "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/throttler" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" @@ -67,6 +68,8 @@ var ( BlplBatchTransaction = "BatchTransaction" ) +var TruncationIndicator = fmt.Sprintf(" ... %s ... ", sqlparser.TruncationText) + // Stats is the internal stats of a player. It is a different // structure that is passed in so stats can be collected over the life // of multiple individual players. diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils.go b/go/vt/vttablet/tabletmanager/vreplication/utils.go index d31d3365920..0037a89f06e 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils.go @@ -25,6 +25,7 @@ import ( "vitess.io/vitess/go/mysql/sqlerror" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/textutil" + "vitess.io/vitess/go/vt/binlog/binlogplayer" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vterrors" @@ -41,10 +42,6 @@ const ( maxVReplicationLogMessageLen = 65535 ) -// vrepliationLogTruncationStr is the string that is used to indicate that a message has been -// truncated before being inserted into one of the vreplication sidecar tables. -var vreplicationLogTruncationStr = fmt.Sprintf(" ... %s ... ", sqlparser.TruncationText) - const ( // Enum values for type column in the vreplication_log table. @@ -112,7 +109,7 @@ func insertLog(dbClient *vdbClient, typ string, vreplID int32, state, message st // We perform the truncation, if needed, in the middle of the message as the end of the message is likely to // be the most important part as it often explains WHY we e.g. failed to execute an INSERT in the workflow. if len(message) > maxVReplicationLogMessageLen { - message, err = textutil.TruncateText(message, maxVReplicationLogMessageLen, truncationLocation, vreplicationLogTruncationStr) + message, err = textutil.TruncateText(message, maxVReplicationLogMessageLen, truncationLocation, binlogplayer.TruncationIndicator) if err != nil { log.Errorf("Could not insert vreplication_log record because we failed to truncate the message: %v", err) return diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go index dd0bec4c6c1..a1775caffbe 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go @@ -81,11 +81,11 @@ func TestInsertLogTruncation(t *testing.T) { err error ) if tc.expectTruncation { - messageOut, err = textutil.TruncateText(tc.message, maxVReplicationLogMessageLen, truncationLocation, vreplicationLogTruncationStr) + messageOut, err = textutil.TruncateText(tc.message, maxVReplicationLogMessageLen, truncationLocation, binlogplayer.TruncationIndicator) require.NoError(t, err) require.True(t, strings.HasPrefix(messageOut, tc.message[:1024])) // Confirm we still have the same beginning require.True(t, strings.HasSuffix(messageOut, tc.message[len(tc.message)-1024:])) // Confirm we still have the same end - require.True(t, strings.Contains(messageOut, vreplicationLogTruncationStr)) // Confirm we have the truncation text + require.True(t, strings.Contains(messageOut, binlogplayer.TruncationIndicator)) // Confirm we have the truncation text t.Logf("Original message length: %d, truncated message length: %d", len(tc.message), len(messageOut)) } else { messageOut = tc.message From 793c7a19063b6fd79fc876e21caf94a2fc27924b Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Wed, 28 Feb 2024 12:35:06 -0500 Subject: [PATCH 17/19] Unify truncation location and indicator btwn vreplication & binlogplayer Signed-off-by: Matt Lord --- go/vt/binlog/binlogplayer/binlog_player.go | 5 +++++ go/vt/binlog/binlogplayer/dbclient.go | 2 +- go/vt/vttablet/tabletmanager/vreplication/utils.go | 5 +---- go/vt/vttablet/tabletmanager/vreplication/utils_test.go | 2 +- 4 files changed, 8 insertions(+), 6 deletions(-) diff --git a/go/vt/binlog/binlogplayer/binlog_player.go b/go/vt/binlog/binlogplayer/binlog_player.go index c415029424f..9a3b3f9c59f 100644 --- a/go/vt/binlog/binlogplayer/binlog_player.go +++ b/go/vt/binlog/binlogplayer/binlog_player.go @@ -43,6 +43,7 @@ import ( "vitess.io/vitess/go/protoutil" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/stats" + "vitess.io/vitess/go/textutil" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/servenv" "vitess.io/vitess/go/vt/sqlparser" @@ -66,6 +67,10 @@ var ( BlplTransaction = "Transaction" // BlplBatchTransaction is the key for the stats map. BlplBatchTransaction = "BatchTransaction" + + // Truncate values in the middle to preserve the end of the message which + // typically contains the error text. + TruncationLocation = textutil.TruncationLocationMiddle ) var TruncationIndicator = fmt.Sprintf(" ... %s ... ", sqlparser.TruncationText) diff --git a/go/vt/binlog/binlogplayer/dbclient.go b/go/vt/binlog/binlogplayer/dbclient.go index 14d80e93b06..61789f345c7 100644 --- a/go/vt/binlog/binlogplayer/dbclient.go +++ b/go/vt/binlog/binlogplayer/dbclient.go @@ -130,7 +130,7 @@ func LogError(msg string, err error) { // LimitString truncates string to specified size func LimitString(s string, limit int) string { - ts, err := textutil.TruncateText(s, limit, textutil.TruncationLocationMiddle, "...") + ts, err := textutil.TruncateText(s, limit, TruncationLocation, TruncationIndicator) if err != nil { // Fallback to simple truncation if len(s) <= limit { return s diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils.go b/go/vt/vttablet/tabletmanager/vreplication/utils.go index 0037a89f06e..35ea52b1c51 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils.go @@ -35,9 +35,6 @@ import ( const ( vreplicationLogTableName = "vreplication_log" - // Truncate values in the middle to preserve the end of the message which typically contains the - // error text. - truncationLocation = textutil.TruncationLocationMiddle // This comes from the fact that the message column in the vreplication_log table is of type TEXT. maxVReplicationLogMessageLen = 65535 ) @@ -109,7 +106,7 @@ func insertLog(dbClient *vdbClient, typ string, vreplID int32, state, message st // We perform the truncation, if needed, in the middle of the message as the end of the message is likely to // be the most important part as it often explains WHY we e.g. failed to execute an INSERT in the workflow. if len(message) > maxVReplicationLogMessageLen { - message, err = textutil.TruncateText(message, maxVReplicationLogMessageLen, truncationLocation, binlogplayer.TruncationIndicator) + message, err = textutil.TruncateText(message, maxVReplicationLogMessageLen, binlogplayer.TruncationLocation, binlogplayer.TruncationIndicator) if err != nil { log.Errorf("Could not insert vreplication_log record because we failed to truncate the message: %v", err) return diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go index a1775caffbe..bfe79036f3c 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go @@ -81,7 +81,7 @@ func TestInsertLogTruncation(t *testing.T) { err error ) if tc.expectTruncation { - messageOut, err = textutil.TruncateText(tc.message, maxVReplicationLogMessageLen, truncationLocation, binlogplayer.TruncationIndicator) + messageOut, err = textutil.TruncateText(tc.message, maxVReplicationLogMessageLen, binlogplayer.TruncationLocation, binlogplayer.TruncationIndicator) require.NoError(t, err) require.True(t, strings.HasPrefix(messageOut, tc.message[:1024])) // Confirm we still have the same beginning require.True(t, strings.HasSuffix(messageOut, tc.message[len(tc.message)-1024:])) // Confirm we still have the same end From 4a5fae93cbc52823e8f14f58e9affe56ec2cc1e2 Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Wed, 28 Feb 2024 12:37:36 -0500 Subject: [PATCH 18/19] Remove what are now redundant comments Signed-off-by: Matt Lord --- go/vt/vttablet/tabletmanager/vreplication/utils.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils.go b/go/vt/vttablet/tabletmanager/vreplication/utils.go index 35ea52b1c51..2b80bfb62a2 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils.go @@ -103,8 +103,6 @@ func insertLog(dbClient *vdbClient, typ string, vreplID int32, state, message st query = fmt.Sprintf("update %s.vreplication_log set count = count + 1 where id = %d", sidecar.GetIdentifier(), id) } else { buf := sqlparser.NewTrackedBuffer(nil) - // We perform the truncation, if needed, in the middle of the message as the end of the message is likely to - // be the most important part as it often explains WHY we e.g. failed to execute an INSERT in the workflow. if len(message) > maxVReplicationLogMessageLen { message, err = textutil.TruncateText(message, maxVReplicationLogMessageLen, binlogplayer.TruncationLocation, binlogplayer.TruncationIndicator) if err != nil { From 9a0e78056b6c012ffe933c83fea6e7f526e029cf Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Thu, 29 Feb 2024 07:19:12 -0500 Subject: [PATCH 19/19] Avoid floats and type conversions Signed-off-by: Matt Lord --- go/textutil/strings.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/go/textutil/strings.go b/go/textutil/strings.go index 1b22231afa2..616366f0083 100644 --- a/go/textutil/strings.go +++ b/go/textutil/strings.go @@ -155,8 +155,8 @@ func TruncateText(text string, limit int, location TruncationLocation, indicator } switch location { case TruncationLocationMiddle: - prefix := int((float64(limit) * 0.5) - float64(len(indicator))) - suffix := int(ol - int(prefix+len(indicator)) + 1) + prefix := (limit / 2) - len(indicator) + suffix := (ol - (prefix + len(indicator))) + 1 return fmt.Sprintf("%s%s%s", text[:prefix], indicator, text[suffix:]), nil case TruncationLocationEnd: return text[:limit-(len(indicator)+1)] + indicator, nil