diff --git a/go/vt/mysqlctl/backup.go b/go/vt/mysqlctl/backup.go index e1c5d04bd81..9a19175164a 100644 --- a/go/vt/mysqlctl/backup.go +++ b/go/vt/mysqlctl/backup.go @@ -37,7 +37,6 @@ import ( "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/vterrors" - stats "vitess.io/vitess/go/vt/mysqlctl/backupstats" topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) @@ -122,7 +121,7 @@ func registerBackupFlags(fs *pflag.FlagSet) { // - remember if we were replicating, restore the exact same state func Backup(ctx context.Context, params BackupParams) error { if params.Stats == nil { - params.Stats = stats.NoStats() + params.Stats = backupstats.NoStats() } startTs := time.Now() @@ -137,8 +136,8 @@ func Backup(ctx context.Context, params BackupParams) error { // Scope bsStats to selected storage engine. bsStats := params.Stats.Scope( - stats.Component(stats.BackupStorage), - stats.Implementation( + backupstats.Component(backupstats.BackupStorage), + backupstats.Implementation( titleCase(backupstorage.BackupStorageImplementation), ), ) @@ -156,8 +155,8 @@ func Backup(ctx context.Context, params BackupParams) error { // Scope stats to selected backup engine. beParams := params.Copy() beParams.Stats = params.Stats.Scope( - stats.Component(stats.BackupEngine), - stats.Implementation(titleCase(backupEngineImplementation)), + backupstats.Component(backupstats.BackupEngine), + backupstats.Implementation(titleCase(backupEngineImplementation)), ) var be BackupEngine if isIncrementalBackup(beParams) { @@ -192,8 +191,8 @@ func Backup(ctx context.Context, params BackupParams) error { } // The backup worked, so just return the finish error, if any. - stats.DeprecatedBackupDurationS.Set(int64(time.Since(startTs).Seconds())) - params.Stats.Scope(stats.Operation("Backup")).TimedIncrement(time.Since(startTs)) + backupstats.DeprecatedBackupDurationS.Set(int64(time.Since(startTs).Seconds())) + params.Stats.Scope(backupstats.Operation("Backup")).TimedIncrement(time.Since(startTs)) return finishErr } @@ -359,7 +358,7 @@ func ensureRestoredGTIDPurgedMatchesManifest(ctx context.Context, manifest *Back // and returns ErrNoBackup. Any other error is returned. func Restore(ctx context.Context, params RestoreParams) (*BackupManifest, error) { if params.Stats == nil { - params.Stats = stats.NoStats() + params.Stats = backupstats.NoStats() } startTs := time.Now() @@ -373,8 +372,8 @@ func Restore(ctx context.Context, params RestoreParams) (*BackupManifest, error) // Scope bsStats to selected storage engine. bsStats := params.Stats.Scope( - stats.Component(backupstats.BackupStorage), - stats.Implementation( + backupstats.Component(backupstats.BackupStorage), + backupstats.Implementation( titleCase(backupstorage.BackupStorageImplementation), ), ) @@ -428,8 +427,8 @@ func Restore(ctx context.Context, params RestoreParams) (*BackupManifest, error) // Scope stats to selected backup engine. reParams := params.Copy() reParams.Stats = params.Stats.Scope( - stats.Component(backupstats.BackupEngine), - stats.Implementation(titleCase(backupEngineImplementation)), + backupstats.Component(backupstats.BackupEngine), + backupstats.Implementation(titleCase(backupEngineImplementation)), ) manifest, err := re.ExecuteRestore(ctx, reParams, bh) if err != nil { @@ -487,8 +486,8 @@ func Restore(ctx context.Context, params RestoreParams) (*BackupManifest, error) return nil, err } - stats.DeprecatedRestoreDurationS.Set(int64(time.Since(startTs).Seconds())) - params.Stats.Scope(stats.Operation("Restore")).TimedIncrement(time.Since(startTs)) + backupstats.DeprecatedRestoreDurationS.Set(int64(time.Since(startTs).Seconds())) + params.Stats.Scope(backupstats.Operation("Restore")).TimedIncrement(time.Since(startTs)) params.Logger.Infof("Restore: complete") return manifest, nil } diff --git a/go/vt/mysqlctl/backupstats/stats.go b/go/vt/mysqlctl/backupstats/stats.go index 1d4b643a0e9..6f64dec864f 100644 --- a/go/vt/mysqlctl/backupstats/stats.go +++ b/go/vt/mysqlctl/backupstats/stats.go @@ -21,7 +21,6 @@ import ( "time" "vitess.io/vitess/go/stats" - vtstats "vitess.io/vitess/go/stats" ) // Stats is a reporting interface meant to be shared among backup and restore @@ -52,9 +51,9 @@ type Stats interface { type noStats struct{} type scopedStats struct { - bytes *vtstats.CountersWithMultiLabels - count *vtstats.CountersWithMultiLabels - durationNs *vtstats.CountersWithMultiLabels + bytes *stats.CountersWithMultiLabels + count *stats.CountersWithMultiLabels + durationNs *stats.CountersWithMultiLabels labelValues []string } diff --git a/go/vt/mysqlctl/builtinbackupengine.go b/go/vt/mysqlctl/builtinbackupengine.go index a311e700b68..de1e4ab679b 100644 --- a/go/vt/mysqlctl/builtinbackupengine.go +++ b/go/vt/mysqlctl/builtinbackupengine.go @@ -44,15 +44,15 @@ import ( "vitess.io/vitess/go/vt/logutil" stats "vitess.io/vitess/go/vt/mysqlctl/backupstats" "vitess.io/vitess/go/vt/mysqlctl/backupstorage" - "vitess.io/vitess/go/vt/proto/mysqlctl" - mysqlctlpb "vitess.io/vitess/go/vt/proto/mysqlctl" - tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" - "vitess.io/vitess/go/vt/proto/vtrpc" "vitess.io/vitess/go/vt/servenv" "vitess.io/vitess/go/vt/topo" "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vttablet/tmclient" + + mysqlctlpb "vitess.io/vitess/go/vt/proto/mysqlctl" + tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" + "vitess.io/vitess/go/vt/proto/vtrpc" ) const ( @@ -331,7 +331,7 @@ func (be *BuiltinBackupEngine) executeIncrementalBackup(ctx context.Context, par // It's also nice for incremental backups that are taken on _other_ tablets, so that they don't need to understand what exactly was purged // on _this_ tablet. They don't care, all they want to know is "what GTIDSet can we get from this". incrementalBackupToPosition.GTIDSet = incrementalBackupToPosition.GTIDSet.Union(gtidPurged.GTIDSet) - req := &mysqlctl.ReadBinlogFilesTimestampsRequest{} + req := &mysqlctlpb.ReadBinlogFilesTimestampsRequest{} for _, binlogFile := range binaryLogsToBackup { fe := FileEntry{Base: backupBinlogDir, Name: binlogFile} fullPath, err := fe.fullPath(params.Cnf) diff --git a/go/vt/schema/online_ddl_test.go b/go/vt/schema/online_ddl_test.go index 1a29c921a4f..dbcad5454dc 100644 --- a/go/vt/schema/online_ddl_test.go +++ b/go/vt/schema/online_ddl_test.go @@ -322,14 +322,6 @@ func TestNewOnlineDDLs(t *testing.T) { } func TestNewOnlineDDLsForeignKeys(t *testing.T) { - type expect struct { - sqls []string - notDDL bool - parseError bool - isError bool - expectErrorText string - isView bool - } queries := []string{ "alter table corder add FOREIGN KEY my_fk(customer_id) references customer(customer_id)", "create table t1 (id int primary key, i int, foreign key (i) references parent(id))", diff --git a/go/vt/sqlparser/ast_funcs.go b/go/vt/sqlparser/ast_funcs.go index 7ca1b7e92e3..3d8d027f12a 100644 --- a/go/vt/sqlparser/ast_funcs.go +++ b/go/vt/sqlparser/ast_funcs.go @@ -1996,17 +1996,6 @@ func (node *ColName) CompliantName() string { return node.Name.CompliantName() } -// isExprAliasForCurrentTimeStamp returns true if the Expr provided is an alias for CURRENT_TIMESTAMP -func isExprAliasForCurrentTimeStamp(expr Expr) bool { - switch node := expr.(type) { - case *FuncExpr: - return node.Name.EqualString("current_timestamp") || node.Name.EqualString("now") || node.Name.EqualString("localtimestamp") || node.Name.EqualString("localtime") - case *CurTimeFuncExpr: - return node.Name.EqualString("current_timestamp") || node.Name.EqualString("now") || node.Name.EqualString("localtimestamp") || node.Name.EqualString("localtime") - } - return false -} - // AtCount represents the '@' count in IdentifierCI type AtCount int @@ -2171,19 +2160,6 @@ func (s SelectExprs) AllAggregation() bool { return true } -func isExprLiteral(expr Expr) bool { - switch expr := expr.(type) { - case *Literal: - return true - case BoolVal: - return true - case *UnaryExpr: - return isExprLiteral(expr.Expr) - default: - return false - } -} - // RemoveKeyspaceFromColName removes the Qualifier.Qualifier on all ColNames in the expression tree func RemoveKeyspaceFromColName(expr Expr) { RemoveKeyspace(expr) diff --git a/go/vt/sqlparser/ast_rewriting_test.go b/go/vt/sqlparser/ast_rewriting_test.go index 677c0d582b2..c116960d139 100644 --- a/go/vt/sqlparser/ast_rewriting_test.go +++ b/go/vt/sqlparser/ast_rewriting_test.go @@ -43,7 +43,6 @@ type myTestCase struct { udv int autocommit, clientFoundRows, skipQueryPlanCache, socket, queryTimeout bool sqlSelectLimit, transactionMode, workload, version, versionComment bool - txIsolation bool } func TestRewrites(in *testing.T) { diff --git a/go/vt/topo/memorytopo/file.go b/go/vt/topo/memorytopo/file.go index 0abfc56cb80..0007203799f 100644 --- a/go/vt/topo/memorytopo/file.go +++ b/go/vt/topo/memorytopo/file.go @@ -262,7 +262,6 @@ func (c *Conn) Delete(ctx context.Context, filePath string, version topo.Version // Check if it's a directory. if n.isDirectory() { - //lint:ignore ST1005 Delete is a function name return fmt.Errorf("delete(%v, %v) failed: it's a directory", c.cell, filePath) } diff --git a/go/vt/vtadmin/cluster/discovery/discovery_static_file_test.go b/go/vt/vtadmin/cluster/discovery/discovery_static_file_test.go index 8fa049f8540..344ee32863d 100644 --- a/go/vt/vtadmin/cluster/discovery/discovery_static_file_test.go +++ b/go/vt/vtadmin/cluster/discovery/discovery_static_file_test.go @@ -23,7 +23,6 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "vitess.io/vitess/go/vt/proto/vtadmin" vtadminpb "vitess.io/vitess/go/vt/proto/vtadmin" ) @@ -54,7 +53,7 @@ func TestDiscoverVTGate(t *testing.T) { }] } `), - expected: &vtadmin.VTGate{ + expected: &vtadminpb.VTGate{ Hostname: "127.0.0.1:12345", }, }, @@ -292,7 +291,7 @@ func TestDiscoverVtctld(t *testing.T) { }] } `), - expected: &vtadmin.Vtctld{ + expected: &vtadminpb.Vtctld{ Hostname: "127.0.0.1:12345", }, }, diff --git a/go/vt/vtctl/vdiff2.go b/go/vt/vtctl/vdiff2.go index 9dab6c30517..7cd1c7e00ca 100644 --- a/go/vt/vtctl/vdiff2.go +++ b/go/vt/vtctl/vdiff2.go @@ -165,9 +165,6 @@ func commandVDiff2(ctx context.Context, wr *wrangler.Wrangler, subFlags *pflag.F return fmt.Errorf("invalid action '%s'; %s", action, usage) } - type ErrorResponse struct { - Error string - } output, err := wr.VDiff2(ctx, keyspace, workflowName, action, actionArg, vdiffUUID.String(), options) if err != nil { log.Errorf("vdiff2 returning with error: %v", err) diff --git a/go/vt/vtgate/buffer/shard_buffer.go b/go/vt/vtgate/buffer/shard_buffer.go index 321e9b90f73..ae33aabb399 100644 --- a/go/vt/vtgate/buffer/shard_buffer.go +++ b/go/vt/vtgate/buffer/shard_buffer.go @@ -95,14 +95,6 @@ type shardBuffer struct { state bufferState // queue is the list of buffered requests (ordered by arrival). queue []*entry - // externallyReparented is the maximum value of all seen - // "StreamHealthResponse.TabletexternallyReparentedTimestamp" values across - // all PRIMARY tablets of this shard. - // In practice, it is a) the last time the shard was reparented or b) the last - // time the TabletExternallyReparented RPC was called on the tablet to confirm - // that the tablet is the current PRIMARY. - // We assume the value is a Unix timestamp in seconds. - externallyReparented int64 // lastStart is the last time we saw the start of a failover. lastStart time.Time // lastEnd is the last time we saw the end of a failover. @@ -513,37 +505,6 @@ func (sb *shardBuffer) recordKeyspaceEvent(alias *topodatapb.TabletAlias, stillS sb.stopBufferingLocked(reason, msg) } -func (sb *shardBuffer) recordExternallyReparentedTimestamp(timestamp int64, alias *topodatapb.TabletAlias) { - // Fast path (read lock): Check if new timestamp is higher. - sb.mu.RLock() - if timestamp <= sb.externallyReparented { - // Do nothing. Equal values are reported if the primary has not changed. - // Smaller values can be reported during the failover by the old primary - // after the new primary already took over. - sb.mu.RUnlock() - return - } - sb.mu.RUnlock() - - // New timestamp is higher. Stop buffering if running. - sb.mu.Lock() - defer sb.mu.Unlock() - - // Re-check value after acquiring write lock. - if timestamp <= sb.externallyReparented { - return - } - - sb.externallyReparented = timestamp - if !topoproto.TabletAliasEqual(alias, sb.currentPrimary) { - if sb.currentPrimary != nil { - sb.lastReparent = sb.timeNow() - } - sb.currentPrimary = alias - } - sb.stopBufferingLocked(stopFailoverEndDetected, "failover end detected") -} - func (sb *shardBuffer) stopBufferingDueToMaxDuration() { sb.mu.Lock() defer sb.mu.Unlock() diff --git a/go/vt/vtgate/endtoend/vstream_test.go b/go/vt/vtgate/endtoend/vstream_test.go index 73086137242..c96f84faff1 100644 --- a/go/vt/vtgate/endtoend/vstream_test.go +++ b/go/vt/vtgate/endtoend/vstream_test.go @@ -25,20 +25,19 @@ import ( "sync" "testing" - "vitess.io/vitess/go/mysql/collations" - vtgatepb "vitess.io/vitess/go/vt/proto/vtgate" - "github.com/stretchr/testify/require" "google.golang.org/protobuf/proto" "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/mysql/collations" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/vtgate/vtgateconn" + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" - "vitess.io/vitess/go/vt/proto/query" querypb "vitess.io/vitess/go/vt/proto/query" topodatapb "vitess.io/vitess/go/vt/proto/topodata" - "vitess.io/vitess/go/vt/vtgate/vtgateconn" + vtgatepb "vitess.io/vitess/go/vt/proto/vtgate" ) func initialize(ctx context.Context, t *testing.T) (*vtgateconn.VTGateConn, *mysql.Conn, *mysql.Conn, func()) { @@ -151,7 +150,7 @@ func TestVStream(t *testing.T) { Keyspace: "ks", Shard: "-80", RowChanges: []*binlogdatapb.RowChange{{ - After: &query.Row{ + After: &querypb.Row{ Lengths: []int64{1, 1}, Values: []byte("11"), }, @@ -177,7 +176,7 @@ func TestVStreamCopyBasic(t *testing.T) { } lastPK := sqltypes.Result{ - Fields: []*query.Field{{Name: "id1", Type: query.Type_INT32}}, + Fields: []*querypb.Field{{Name: "id1", Type: querypb.Type_INT32}}, Rows: [][]sqltypes.Value{{sqltypes.NewInt32(4)}}, } qr := sqltypes.ResultToProto3(&lastPK) @@ -405,7 +404,7 @@ func TestVStreamCopyResume(t *testing.T) { // lastPK is id1=4, meaning we should only copy rows for id1 IN(5,6,7,8,9) lastPK := sqltypes.Result{ - Fields: []*query.Field{{Name: "id1", Type: query.Type_INT64, Charset: collations.CollationBinaryID, Flags: uint32(query.MySqlFlag_NUM_FLAG | query.MySqlFlag_BINARY_FLAG)}}, + Fields: []*querypb.Field{{Name: "id1", Type: querypb.Type_INT64, Charset: collations.CollationBinaryID, Flags: uint32(querypb.MySqlFlag_NUM_FLAG | querypb.MySqlFlag_BINARY_FLAG)}}, Rows: [][]sqltypes.Value{{sqltypes.NewInt64(4)}}, } tableLastPK := []*binlogdatapb.TableLastPK{{ diff --git a/go/vt/vtgate/evalengine/compiler_test.go b/go/vt/vtgate/evalengine/compiler_test.go index e27089a44fb..e1e905a6efa 100644 --- a/go/vt/vtgate/evalengine/compiler_test.go +++ b/go/vt/vtgate/evalengine/compiler_test.go @@ -179,19 +179,6 @@ func TestCompilerReference(t *testing.T) { t.Logf("\n%s", track.String()) } -type debugCompiler struct { - t testing.TB -} - -func (d *debugCompiler) Instruction(ins string, args ...any) { - ins = fmt.Sprintf(ins, args...) - d.t.Logf("> %s", ins) -} - -func (d *debugCompiler) Stack(old, new int) { - d.t.Logf("\tsp = %d -> %d", old, new) -} - func TestCompilerSingle(t *testing.T) { var testCases = []struct { expression string diff --git a/go/vt/vtgate/evalengine/eval_temporal.go b/go/vt/vtgate/evalengine/eval_temporal.go index fe7b9ea87f3..13acc5bd290 100644 --- a/go/vt/vtgate/evalengine/eval_temporal.go +++ b/go/vt/vtgate/evalengine/eval_temporal.go @@ -1,8 +1,6 @@ package evalengine import ( - "time" - "vitess.io/vitess/go/hack" "vitess.io/vitess/go/mysql/collations" "vitess.io/vitess/go/mysql/datetime" @@ -140,10 +138,6 @@ func (e *evalTemporal) isZero() bool { return e.dt.IsZero() } -func (e *evalTemporal) toStdTime(loc *time.Location) time.Time { - return e.dt.ToStdTime(loc) -} - func (e *evalTemporal) addInterval(interval *datetime.Interval, strcoll collations.TypedCollation) eval { var tmp *evalTemporal var ok bool diff --git a/go/vt/vtgate/evalengine/testcases/inputs.go b/go/vt/vtgate/evalengine/testcases/inputs.go index 3af5b5b6723..245318529c3 100644 --- a/go/vt/vtgate/evalengine/testcases/inputs.go +++ b/go/vt/vtgate/evalengine/testcases/inputs.go @@ -162,12 +162,6 @@ var regexMatchStrings = []string{ "'c'", "'i'", "'m'", "'n'", "'u'", "'cimnu'", "'cimnuunmic'", } -var regexCounters = []string{ - "NULL", - "0", "1", "5", "100000", - "'2'", "0.4", "0.5", "0x1", -} - const inputPi = "314159265358979323846264338327950288419716939937510582097494459" var inputStrings = []string{ diff --git a/go/vt/vtgate/executor.go b/go/vt/vtgate/executor.go index 971967c0427..07b1ba79f42 100644 --- a/go/vt/vtgate/executor.go +++ b/go/vt/vtgate/executor.go @@ -117,10 +117,6 @@ type Executor struct { // allowScatter will fail planning if set to false and a plan contains any scatter queries allowScatter bool - // truncateErrorLen truncates errors sent to client if they are above this value - // (0 means do not truncate). - truncateErrorLen int - // queryLogger is passed in for logging from this vtgate executor. queryLogger *streamlog.StreamLogger[*logstats.LogStats] } diff --git a/go/vt/vtgate/planbuilder/builder.go b/go/vt/vtgate/planbuilder/builder.go index 5e317e9e9c5..0b090e2472e 100644 --- a/go/vt/vtgate/planbuilder/builder.go +++ b/go/vt/vtgate/planbuilder/builder.go @@ -438,26 +438,6 @@ func (tc *tableCollector) getTables() []string { return tableNames } -func (tc *tableCollector) addVindexTable(t *vindexes.Table) { - if t == nil { - return - } - ks, tbl := "", t.Name.String() - if t.Keyspace != nil { - ks = t.Keyspace.Name - } - tc.addTable(ks, tbl) -} - -func (tc *tableCollector) addAllTables(tables []string) { - if tc.tables == nil { - tc.tables = map[string]any{} - } - for _, tbl := range tables { - tc.tables[tbl] = nil - } -} - func newFlushStmt(stmt *sqlparser.Flush, tables sqlparser.TableNames) *sqlparser.Flush { return &sqlparser.Flush{ IsLocal: stmt.IsLocal, diff --git a/go/vt/vtgate/planbuilder/ddl.go b/go/vt/vtgate/planbuilder/ddl.go index 1557a1d1d8f..f366a169d69 100644 --- a/go/vt/vtgate/planbuilder/ddl.go +++ b/go/vt/vtgate/planbuilder/ddl.go @@ -20,18 +20,6 @@ const ( DifferentDestinations string = "Tables or Views specified in the query do not belong to the same destination" ) -type fkStrategy int - -const ( - fkAllow fkStrategy = iota - fkDisallow -) - -var fkStrategyMap = map[string]fkStrategy{ - "allow": fkAllow, - "disallow": fkDisallow, -} - type fkContraint struct { found bool } diff --git a/go/vt/vtgate/planbuilder/operators/SQL_builder.go b/go/vt/vtgate/planbuilder/operators/SQL_builder.go index ac961095d08..6ea6dd43ff0 100644 --- a/go/vt/vtgate/planbuilder/operators/SQL_builder.go +++ b/go/vt/vtgate/planbuilder/operators/SQL_builder.go @@ -20,7 +20,6 @@ import ( "fmt" "slices" "sort" - "strings" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vterrors" @@ -247,31 +246,6 @@ func (qb *queryBuilder) joinOuterWith(other *queryBuilder, onCondition sqlparser } } -func (qb *queryBuilder) rewriteExprForDerivedTable(expr sqlparser.Expr, dtName string) { - _ = sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) { - col, ok := node.(*sqlparser.ColName) - if !ok { - return true, nil - } - hasTable := qb.hasTable(col.Qualifier.Name.String()) - if hasTable { - col.Qualifier = sqlparser.TableName{ - Name: sqlparser.NewIdentifierCS(dtName), - } - } - return true, nil - }, expr) -} - -func (qb *queryBuilder) hasTable(tableName string) bool { - for _, name := range qb.tableNames { - if strings.EqualFold(tableName, name) { - return true - } - } - return false -} - func (qb *queryBuilder) sortTables() { _ = sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) { sel, isSel := node.(*sqlparser.Select) @@ -319,20 +293,6 @@ func (ts *tableSorter) Swap(i, j int) { ts.sel.From[i], ts.sel.From[j] = ts.sel.From[j], ts.sel.From[i] } -func (h *Horizon) toSQL(qb *queryBuilder) error { - err := stripDownQuery(h.Query, qb.sel) - if err != nil { - return err - } - _ = sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) { - if aliasedExpr, ok := node.(sqlparser.SelectExpr); ok { - removeKeyspaceFromSelectExpr(aliasedExpr) - } - return true, nil - }, qb.sel) - return nil -} - func removeKeyspaceFromSelectExpr(expr sqlparser.SelectExpr) { switch expr := expr.(type) { case *sqlparser.AliasedExpr: diff --git a/go/vt/vtgate/planbuilder/operators/horizon.go b/go/vt/vtgate/planbuilder/operators/horizon.go index c529b7d76b4..da97e74d8cc 100644 --- a/go/vt/vtgate/planbuilder/operators/horizon.go +++ b/go/vt/vtgate/planbuilder/operators/horizon.go @@ -65,42 +65,6 @@ func (h *Horizon) Clone(inputs []ops.Operator) ops.Operator { } } -// findOutputColumn returns the index on which the given name is found in the slice of -// *sqlparser.SelectExprs of the derivedTree. The *sqlparser.SelectExpr must be of type -// *sqlparser.AliasedExpr and match the given name. -// If name is not present but the query's select expressions contain a *sqlparser.StarExpr -// the function will return no error and an index equal to -1. -// If name is not present and the query does not have a *sqlparser.StarExpr, the function -// will return an unknown column error. -func (h *Horizon) findOutputColumn(name *sqlparser.ColName) (int, error) { - hasStar := false - for j, exp := range sqlparser.GetFirstSelect(h.Query).SelectExprs { - switch exp := exp.(type) { - case *sqlparser.AliasedExpr: - if !exp.As.IsEmpty() && exp.As.Equal(name.Name) { - return j, nil - } - if exp.As.IsEmpty() { - col, ok := exp.Expr.(*sqlparser.ColName) - if !ok { - return 0, vterrors.VT12001("complex expression needs column alias: %s", sqlparser.String(exp)) - } - if name.Name.Equal(col.Name) { - return j, nil - } - } - case *sqlparser.StarExpr: - hasStar = true - } - } - - // we have found a star but no matching *sqlparser.AliasedExpr, thus we return -1 with no error. - if hasStar { - return -1, nil - } - return 0, vterrors.VT03014(name.Name.String(), "field list") -} - // IsMergeable is not a great name for this function. Suggestions for a better one are welcome! // This function will return false if the derived table inside it has to run on the vtgate side, and so can't be merged with subqueries // This logic can also be used to check if this is a derived table that can be had on the left hand side of a vtgate join. @@ -250,10 +214,6 @@ func (h *Horizon) getQP(ctx *plancontext.PlanningContext) (*QueryProjection, err return h.QP, nil } -func (h *Horizon) setQP(qp *QueryProjection) { - h.QP = qp -} - func (h *Horizon) ShortDescription() string { return h.Alias } diff --git a/go/vt/vtgate/planbuilder/operators/querygraph.go b/go/vt/vtgate/planbuilder/operators/querygraph.go index 0d0d3f4e28a..f384607fe10 100644 --- a/go/vt/vtgate/planbuilder/operators/querygraph.go +++ b/go/vt/vtgate/planbuilder/operators/querygraph.go @@ -92,23 +92,6 @@ func newQueryGraph() *QueryGraph { return &QueryGraph{} } -func (qg *QueryGraph) collectPredicates(ctx *plancontext.PlanningContext, sel *sqlparser.Select) error { - predicates := sqlparser.SplitAndExpression(nil, sel.Where.Expr) - - for _, predicate := range predicates { - qg.collectPredicate(ctx, predicate) - } - return nil -} - -func (qg *QueryGraph) getPredicateByDeps(ts semantics.TableSet) ([]sqlparser.Expr, bool) { - for _, join := range qg.innerJoins { - if join.deps == ts { - return join.exprs, true - } - } - return nil, false -} func (qg *QueryGraph) addJoinPredicates(ctx *plancontext.PlanningContext, ts semantics.TableSet, predicate sqlparser.Expr) { for _, join := range qg.innerJoins { if join.deps == ts { diff --git a/go/vt/vtgate/planbuilder/operators/sharded_routing.go b/go/vt/vtgate/planbuilder/operators/sharded_routing.go index b1740903da1..13f28005ea8 100644 --- a/go/vt/vtgate/planbuilder/operators/sharded_routing.go +++ b/go/vt/vtgate/planbuilder/operators/sharded_routing.go @@ -557,29 +557,6 @@ func (tr *ShardedRouting) hasVindex(column *sqlparser.ColName) bool { return false } -// Reset all vindex predicates on this route and re-build their options from -// the list of seen routing predicates. -func (tr *ShardedRouting) resetRoutingSelections(ctx *plancontext.PlanningContext) error { - tr.RouteOpCode = engine.Scatter - tr.Selected = nil - for i, vp := range tr.VindexPreds { - tr.VindexPreds[i] = &VindexPlusPredicates{ColVindex: vp.ColVindex, TableID: vp.TableID} - } - - var routing Routing = tr - for _, predicate := range tr.SeenPredicates { - var err error - routing, err = UpdateRoutingLogic(ctx, predicate, routing) - if err != nil { - return err - } - } - if routing != tr { - return vterrors.VT13001("uh-oh. we ended up with a different type of routing") - } - return nil -} - func (tr *ShardedRouting) SelectedVindex() vindexes.Vindex { if tr.Selected == nil { return nil diff --git a/go/vt/vtgate/planbuilder/operators/union.go b/go/vt/vtgate/planbuilder/operators/union.go index ce98060526b..f6258ebee2d 100644 --- a/go/vt/vtgate/planbuilder/operators/union.go +++ b/go/vt/vtgate/planbuilder/operators/union.go @@ -36,7 +36,6 @@ type Union struct { unionColumns sqlparser.SelectExprs unionColumnsAsAlisedExprs []*sqlparser.AliasedExpr - offsetPlanned bool } func newUnion(srcs []ops.Operator, sourceSelects []sqlparser.SelectExprs, columns sqlparser.SelectExprs, distinct bool) *Union { diff --git a/go/vt/vtgate/planbuilder/ordered_aggregate.go b/go/vt/vtgate/planbuilder/ordered_aggregate.go index 3bacc7d3b27..6163900e674 100644 --- a/go/vt/vtgate/planbuilder/ordered_aggregate.go +++ b/go/vt/vtgate/planbuilder/ordered_aggregate.go @@ -52,7 +52,6 @@ var _ logicalPlan = (*orderedAggregate)(nil) // } type orderedAggregate struct { resultsBuilder - extraDistinct *sqlparser.ColName // aggregates specifies the aggregation parameters for each // aggregation function: function opcode and input column number. diff --git a/go/vt/vtgate/planbuilder/pullout_subquery.go b/go/vt/vtgate/planbuilder/pullout_subquery.go index c276c86b426..06c24cad1f5 100644 --- a/go/vt/vtgate/planbuilder/pullout_subquery.go +++ b/go/vt/vtgate/planbuilder/pullout_subquery.go @@ -31,7 +31,6 @@ var _ logicalPlan = (*pulloutSubquery)(nil) // This gets built if a subquery is not correlated and can // therefore can be pulled out and executed upfront. type pulloutSubquery struct { - order int subquery logicalPlan underlying logicalPlan eSubquery *engine.PulloutSubquery diff --git a/go/vt/vtgate/semantics/early_rewriter.go b/go/vt/vtgate/semantics/early_rewriter.go index b3553a2de73..7dfdbf78247 100644 --- a/go/vt/vtgate/semantics/early_rewriter.go +++ b/go/vt/vtgate/semantics/early_rewriter.go @@ -586,27 +586,3 @@ func (e *expanderState) storeExpandInfo(tbl TableInfo, tblName sqlparser.TableNa } e.expandedColumns[tblName] = append(e.expandedColumns[tblName], colName) } - -// createAliasedExpr creates an AliasedExpr with a ColName and an optional alias based on the given ColumnInfo. -// We need table qualifiers if there are more than one table in the FROM clause. -// If we are adding qualifiers, we also add an alias so the qualifiers do not show -// up in the result. For example, SELECT * FROM t1, t2 -> SELECT t1.col AS col, t2.col AS col ... -func (e *expanderState) createAliasedExpr( - col ColumnInfo, - tbl TableInfo, - tblName sqlparser.TableName, -) *sqlparser.AliasedExpr { - tableAliased := !tbl.GetExpr().As.IsEmpty() - withQualifier := e.needsQualifier || tableAliased - var colName *sqlparser.ColName - var alias sqlparser.IdentifierCI - if withQualifier { - colName = sqlparser.NewColNameWithQualifier(col.Name, tblName) - } else { - colName = sqlparser.NewColName(col.Name) - } - if e.needsQualifier { - alias = sqlparser.NewIdentifierCI(col.Name) - } - return &sqlparser.AliasedExpr{Expr: colName, As: alias} -} diff --git a/go/vt/vtgate/semantics/info_schema_gen_test.go b/go/vt/vtgate/semantics/info_schema_gen_test.go index c5fe0123852..61241d96653 100644 --- a/go/vt/vtgate/semantics/info_schema_gen_test.go +++ b/go/vt/vtgate/semantics/info_schema_gen_test.go @@ -70,69 +70,6 @@ func TestGenerateInfoSchemaMap(t *testing.T) { } var ( - informationSchemaTables57 = []string{ - "CHARACTER_SETS", - "COLLATION_CHARACTER_SET_APPLICABILITY", - "COLLATIONS", - "COLUMN_PRIVILEGES", - "COLUMNS", - "ENGINES", - "EVENTS", - "FILES", - "GLOBAL_STATUS", - "GLOBAL_VARIABLES", - "INNODB_BUFFER_PAGE", - "INNODB_BUFFER_PAGE_LRU", - "INNODB_BUFFER_POOL_STATS", - "INNODB_CMP", - "INNODB_CMP_PER_INDEX", - "INNODB_CMP_PER_INDEX_RESET", - "INNODB_CMP_RESET", - "INNODB_CMPMEM", - "INNODB_CMPMEM_RESET", - "INNODB_FT_BEING_DELETED", - "INNODB_FT_CONFIG", - "INNODB_FT_DEFAULT_STOPWORD", - "INNODB_FT_DELETED", - "INNODB_FT_INDEX_CACHE", - "INNODB_FT_INDEX_TABLE", - "INNODB_LOCK_WAITS", - "INNODB_LOCKS", - "INNODB_METRICS", - "INNODB_SYS_COLUMNS", - "INNODB_SYS_DATAFILES", - "INNODB_SYS_FIELDS", - "INNODB_SYS_FOREIGN", - "INNODB_SYS_FOREIGN_COLS", - "INNODB_SYS_INDEXES", - "INNODB_SYS_TABLES", - "INNODB_SYS_TABLESPACES", - "INNODB_SYS_TABLESTATS", - "INNODB_SYS_VIRTUAL", - "INNODB_TEMP_TABLE_INFO", - "INNODB_TRX", - "KEY_COLUMN_USAGE", - "OPTIMIZER_TRACE", - "PARAMETERS", - "PARTITIONS", - "PLUGINS", - "PROCESSLIST", - "PROFILING", - "REFERENTIAL_CONSTRAINTS", - "ROUTINES", - "SCHEMA_PRIVILEGES", - "SCHEMATA", - "SESSION_STATUS", - "SESSION_VARIABLES", - "STATISTICS", - "TABLE_CONSTRAINTS", - "TABLE_PRIVILEGES", - "TABLES", - "TABLESPACES", - "TRIGGERS", - "USER_PRIVILEGES", - "VIEWS", - } informationSchemaTables80 = []string{ "ADMINISTRABLE_ROLE_AUTHORIZATIONS", "APPLICABLE_ROLES", diff --git a/go/vt/vttablet/tabletserver/vstreamer/main_flaky_test.go b/go/vt/vttablet/tabletserver/vstreamer/main_flaky_test.go index e2cdec9b3f1..442c998b71c 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/main_flaky_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/main_flaky_test.go @@ -32,9 +32,8 @@ import ( ) var ( - engine *Engine - env *testenv.Env - schemaDir string + engine *Engine + env *testenv.Env ignoreKeyspaceShardInFieldAndRowEvents bool )