diff --git a/pkg/bench/rttanalysis/orm_queries_bench_test.go b/pkg/bench/rttanalysis/orm_queries_bench_test.go index a6ce65f69927..421368cc42da 100644 --- a/pkg/bench/rttanalysis/orm_queries_bench_test.go +++ b/pkg/bench/rttanalysis/orm_queries_bench_test.go @@ -361,8 +361,6 @@ LEFT JOIN LATERAL ) columns ON true;`, }, - // Once https://github.com/cockroachdb/cockroach/issues/88885 is resolved, - // the previous test case should be identical to this one. { Name: "hasura column descriptions modified", Setup: "CREATE TABLE t(a INT PRIMARY KEY)", diff --git a/pkg/jobs/jobs_test.go b/pkg/jobs/jobs_test.go index ec9c655eda65..940d5f254e60 100644 --- a/pkg/jobs/jobs_test.go +++ b/pkg/jobs/jobs_test.go @@ -2324,7 +2324,7 @@ func TestJobInTxn(t *testing.T) { // Accessed atomically. var hasRun int32 - var job *jobs.Job + var jobID jobspb.JobID defer sql.ClearPlanHooks() // Piggy back on BACKUP to be able to create a succeeding test job. @@ -2338,7 +2338,7 @@ func TestJobInTxn(t *testing.T) { } fn := func(ctx context.Context, _ []sql.PlanNode, _ chan<- tree.Datums) error { var err error - job, err = execCtx.ExtendedEvalContext().QueueJob(ctx, execCtx.InternalSQLTxn(), jobs.Record{ + jobID = execCtx.ExtendedEvalContext().QueueJob(&jobs.Record{ Description: st.String(), Details: jobspb.BackupDetails{}, Progress: jobspb.BackupProgress{}, @@ -2378,7 +2378,7 @@ func TestJobInTxn(t *testing.T) { } fn := func(ctx context.Context, _ []sql.PlanNode, _ chan<- tree.Datums) error { var err error - job, err = execCtx.ExtendedEvalContext().QueueJob(ctx, execCtx.InternalSQLTxn(), jobs.Record{ + jobID = execCtx.ExtendedEvalContext().QueueJob(&jobs.Record{ Description: "RESTORE", Details: jobspb.RestoreDetails{}, Progress: jobspb.RestoreProgress{}, @@ -2413,14 +2413,14 @@ func TestJobInTxn(t *testing.T) { // If we rollback then the job should not run require.NoError(t, txn.Rollback()) registry := s.JobRegistry().(*jobs.Registry) - _, err = registry.LoadJob(ctx, job.ID()) + _, err = registry.LoadJob(ctx, jobID) require.Error(t, err, "the job should not exist after the txn is rolled back") require.True(t, jobs.HasJobNotFoundError(err)) sqlRunner := sqlutils.MakeSQLRunner(sqlDB) // Just in case the job was scheduled let's wait for it to finish // to avoid a race. - sqlRunner.Exec(t, "SHOW JOB WHEN COMPLETE $1", job.ID()) + sqlRunner.Exec(t, "SHOW JOB WHEN COMPLETE $1", jobID) require.Equal(t, int32(0), atomic.LoadInt32(&hasRun), "job has run in transaction before txn commit") require.True(t, timeutil.Since(start) < jobs.DefaultAdoptInterval, "job should have been adopted immediately") @@ -2437,7 +2437,7 @@ func TestJobInTxn(t *testing.T) { // Committing will block and wait for all jobs to run. require.NoError(t, txn.Commit()) registry := s.JobRegistry().(*jobs.Registry) - j, err := registry.LoadJob(ctx, job.ID()) + j, err := registry.LoadJob(ctx, jobID) require.NoError(t, err, "queued job not found") require.NotEqual(t, int32(0), atomic.LoadInt32(&hasRun), "job scheduled in transaction did not run") diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index bc13834ea888..68cdd44d2c2d 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -44,6 +44,7 @@ go_library( "completions.go", "conn_executor.go", "conn_executor_exec.go", + "conn_executor_jobs.go", "conn_executor_prepare.go", "conn_executor_savepoints.go", "conn_executor_show_commit_timestamp.go", @@ -134,6 +135,7 @@ go_library( "inverted_join.go", "job_exec_context.go", "job_exec_context_test_util.go", + "jobs_collection.go", "join.go", "join_predicate.go", "join_token.go", diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 70c8cf522d83..0e51778ee8a4 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -21,7 +21,6 @@ import ( "time" "unicode/utf8" - "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/multitenant" @@ -1018,9 +1017,8 @@ func (s *Server) newConnExecutor( ex.extraTxnState.descCollection = s.cfg.CollectionFactory.NewCollection( ctx, descs.WithDescriptorSessionDataProvider(dsdp), descs.WithMonitor(ex.sessionMon), ) - ex.extraTxnState.jobs = new(jobsCollection) + ex.extraTxnState.jobs = newTxnJobsCollection() ex.extraTxnState.txnRewindPos = -1 - ex.extraTxnState.schemaChangeJobRecords = make(map[descpb.ID]*jobs.Record) ex.extraTxnState.schemaChangerState = &SchemaChangerState{ mode: ex.sessionData().NewSchemaChangerMode, } @@ -1275,18 +1273,7 @@ type connExecutor struct { // descCollection collects descriptors used by the current transaction. descCollection *descs.Collection - // jobs accumulates jobs staged for execution inside the transaction. - // Staging happens when executing statements that are implemented with a - // job. The jobs are staged via the function QueueJob in - // pkg/sql/planner.go. The staged jobs are executed once the transaction - // that staged them commits. - jobs *jobsCollection - - // schemaChangeJobRecords is a map of descriptor IDs to job Records. - // Used in createOrUpdateSchemaChangeJob so we can check if a job has been - // queued up for the given ID. The cache remains valid only for the current - // transaction and it is cleared after the transaction is committed. - schemaChangeJobRecords map[descpb.ID]*jobs.Record + jobs *txnJobsCollection // firstStmtExecuted indicates that the first statement inside this // transaction has been executed. @@ -1714,9 +1701,6 @@ func (ex *connExecutor) resetExtraTxnState(ctx context.Context, ev txnEvent) { } } else { ex.extraTxnState.descCollection.ReleaseAll(ctx) - for k := range ex.extraTxnState.schemaChangeJobRecords { - delete(ex.extraTxnState.schemaChangeJobRecords, k) - } ex.extraTxnState.jobs.reset() ex.extraTxnState.schemaChangerState = &SchemaChangerState{ mode: ex.sessionData().NewSchemaChangerMode, @@ -2824,15 +2808,14 @@ func (ex *connExecutor) initEvalCtx(ctx context.Context, evalCtx *extendedEvalCo DescIDGenerator: ex.getDescIDGenerator(), RangeStatsFetcher: p.execCfg.RangeStatsFetcher, }, - Tracing: &ex.sessionTracing, - MemMetrics: &ex.memMetrics, - Descs: ex.extraTxnState.descCollection, - TxnModesSetter: ex, - Jobs: ex.extraTxnState.jobs, - SchemaChangeJobRecords: ex.extraTxnState.schemaChangeJobRecords, - statsProvider: ex.server.sqlStats, - indexUsageStats: ex.indexUsageStats, - statementPreparer: ex, + Tracing: &ex.sessionTracing, + MemMetrics: &ex.memMetrics, + Descs: ex.extraTxnState.descCollection, + TxnModesSetter: ex, + jobs: ex.extraTxnState.jobs, + statsProvider: ex.server.sqlStats, + indexUsageStats: ex.indexUsageStats, + statementPreparer: ex, } evalCtx.copyFromExecCfg(ex.server.cfg) } @@ -3077,13 +3060,27 @@ func (ex *connExecutor) txnStateTransitionsApplyWrapper( } ex.notifyStatsRefresherOfNewTables(ex.Ctx()) + // If there is any descriptor has new version. We want to make sure there is + // only one version of the descriptor in all nodes. In schema changer jobs, + // `WaitForOneVersion` has been called for the descriptors included in jobs. + // So we just need to do this for descriptors not in jobs. + // We need to get descriptor IDs in jobs before jobs are run because we have + // operations in declarative schema changer removing descriptor IDs from job + // payload as it's done with the descriptors. + descIDsInJobs, err := ex.descIDsInSchemaChangeJobs() + if err != nil { + return advanceInfo{}, err + } ex.statsCollector.PhaseTimes().SetSessionPhaseTime(sessionphase.SessionStartPostCommitJob, timeutil.Now()) if err := ex.server.cfg.JobRegistry.Run( - ex.ctxHolder.connCtx, *ex.extraTxnState.jobs, + ex.ctxHolder.connCtx, ex.extraTxnState.jobs.created, ); err != nil { handleErr(err) } ex.statsCollector.PhaseTimes().SetSessionPhaseTime(sessionphase.SessionEndPostCommitJob, timeutil.Now()) + if err := ex.waitOneVersionForNewVersionDescriptorsWithoutJobs(descIDsInJobs); err != nil { + return advanceInfo{}, err + } fallthrough case txnRestart, txnRollback: @@ -3395,7 +3392,7 @@ func (ex *connExecutor) runPreCommitStages(ctx context.Context) error { scs.state = after scs.jobID = jobID if jobID != jobspb.InvalidJobID { - ex.extraTxnState.jobs.add(jobID) + ex.extraTxnState.jobs.addCreatedJobID(jobID) log.Infof(ctx, "queued new schema change job %d using the new schema changer", jobID) } return nil diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index a410f3866f3b..c0d8bdf33ada 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -1059,12 +1059,15 @@ func (ex *connExecutor) commitSQLTransactionInternal(ctx context.Context) error // createJobs creates jobs for the records cached in schemaChangeJobRecords // during this transaction. func (ex *connExecutor) createJobs(ctx context.Context) error { - if len(ex.extraTxnState.schemaChangeJobRecords) == 0 { + if !ex.extraTxnState.jobs.hasAnyToCreate() { return nil } var records []*jobs.Record - for _, record := range ex.extraTxnState.schemaChangeJobRecords { - records = append(records, record) + if err := ex.extraTxnState.jobs.forEachToCreate(func(jobRecord *jobs.Record) error { + records = append(records, jobRecord) + return nil + }); err != nil { + return err } jobIDs, err := ex.server.cfg.JobRegistry.CreateJobsWithTxn( ctx, ex.planner.InternalSQLTxn(), records, @@ -1072,7 +1075,7 @@ func (ex *connExecutor) createJobs(ctx context.Context) error { if err != nil { return err } - ex.planner.extendedEvalCtx.Jobs.add(jobIDs...) + ex.planner.extendedEvalCtx.jobs.addCreatedJobID(jobIDs...) return nil } diff --git a/pkg/sql/conn_executor_jobs.go b/pkg/sql/conn_executor_jobs.go new file mode 100644 index 000000000000..c2ddb695618c --- /dev/null +++ b/pkg/sql/conn_executor_jobs.go @@ -0,0 +1,124 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sql + +import ( + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/errors" +) + +// waitOneVersionForNewVersionDescriptorsWithoutJobs is to used wait until all +// descriptors with new versions to converge to one version in the cluster. +// `descIDsInJobs` are collected with `descIDsInSchemaChangeJobs`. We need to do +// this to make sure all descriptors mutated are at one version when the schema +// change finish in the user transaction. In schema change jobs, we do similar +// thing for affected descriptors. But, in some scenario, jobs are not created +// for mutated descriptors. +func (ex *connExecutor) waitOneVersionForNewVersionDescriptorsWithoutJobs( + descIDsInJobs catalog.DescriptorIDSet, +) error { + withNewVersion, err := ex.extraTxnState.descCollection.GetOriginalPreviousIDVersionsForUncommitted() + if err != nil { + return err + } + for _, idVersion := range withNewVersion { + if descIDsInJobs.Contains(idVersion.ID) { + continue + } + if _, err := WaitToUpdateLeases(ex.Ctx(), ex.planner.LeaseMgr(), idVersion.ID); err != nil { + // In most cases (normal schema changes), deleted descriptor should have + // been handled by jobs. So, normally we won't hit into the situation of + // wait for one version of a deleted descriptor. However, we need catch + // ErrDescriptorNotFound here because we have a special case of descriptor + // repairing where we delete descriptors directly and never record the ids + // in jobs payload or details. + if errors.Is(err, catalog.ErrDescriptorNotFound) { + continue + } + return err + } + } + return nil +} + +// descIDsInSchemaChangeJobs returns all descriptor IDs with which schema change +// jobs in this transaction will perform. Within schema change jobs, we also +// wait until the whole cluster only has leases on the latest version of these +// descriptors, and we would like to also "wait for one version" for descriptors +// with new versions but not included in any schema change jobs. +func (ex *connExecutor) descIDsInSchemaChangeJobs() (catalog.DescriptorIDSet, error) { + // Get descriptor IDs from legacy schema changer jobs. + var descIDsInJobs catalog.DescriptorIDSet + if err := ex.extraTxnState.jobs.forEachToCreate(func(jobRecord *jobs.Record) error { + switch t := jobRecord.Details.(type) { + case jobspb.SchemaChangeDetails: + // In most cases, the field DescriptorIDs contains descriptor IDs the + // schema change directly affects. Like it could be a table ID if an index + // is created, or it could be a list of schema IDs when dropping a group + // of schemas. + // But it can be confusing sometimes in two types of schema change jobs: + // (1) dropping a database: + // In this scenario, the DescriptorIDs is the ids of tables in this + // database that will be dropped together. And the DroppedDatabaseID field + // is the actual ID of the database that will be dropped. + // (2) any other changes on a database (but not drop): + // For example, when renaming a schema, database's list of all schemas + // need to be updated, and we create a job for this kind of database + // changes. DescriptorIDs is empty in this case and the DescID field in + // the job + // detail is the actual database ID. + for _, descID := range jobRecord.DescriptorIDs { + descIDsInJobs.Add(descID) + } + for _, tbl := range t.DroppedTables { + descIDsInJobs.Add(tbl.ID) + } + for _, id := range t.DroppedTypes { + descIDsInJobs.Add(id) + } + for _, id := range t.DroppedSchemas { + descIDsInJobs.Add(id) + } + descIDsInJobs.Add(t.DroppedDatabaseID) + descIDsInJobs.Add(t.DescID) + } + return nil + }); err != nil { + return catalog.DescriptorIDSet{}, err + } + + // If there is no declarative schema changer job, then we are done. Otherwise, + // we need to check which descriptor has the jobID in its schema change state. + if ex.extraTxnState.schemaChangerState.jobID == jobspb.InvalidJobID { + return descIDsInJobs, nil + } + // Get descriptor IDs with declarative schema changer jobs. + withNewVersion, err := ex.extraTxnState.descCollection.GetOriginalPreviousIDVersionsForUncommitted() + if err != nil { + return catalog.DescriptorIDSet{}, err + } + for _, idVersion := range withNewVersion { + if descIDsInJobs.Contains(idVersion.ID) { + continue + } + desc, err := ex.extraTxnState.descCollection.ByID(ex.state.mu.txn).Get().Desc(ex.Ctx(), idVersion.ID) + if err != nil { + return catalog.DescriptorIDSet{}, err + } + state := desc.GetDeclarativeSchemaChangerState() + if state != nil && state.JobID != jobspb.InvalidJobID { + descIDsInJobs.Add(idVersion.ID) + } + } + return descIDsInJobs, nil +} diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 353a2508889e..7ef2aae5aaaa 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -1111,14 +1111,17 @@ func makeJobsTableRows( } defer cleanup(ctx) - sessionJobs := make([]*jobs.Record, 0, len(p.extendedEvalCtx.SchemaChangeJobRecords)) + sessionJobs := make([]*jobs.Record, 0, p.extendedEvalCtx.jobs.numToCreate()) uniqueJobs := make(map[*jobs.Record]struct{}) - for _, job := range p.extendedEvalCtx.SchemaChangeJobRecords { + if err := p.extendedEvalCtx.jobs.forEachToCreate(func(job *jobs.Record) error { if _, ok := uniqueJobs[job]; ok { - continue + return nil } sessionJobs = append(sessionJobs, job) uniqueJobs[job] = struct{}{} + return nil + }); err != nil { + return matched, err } // Loop while we need to skip a row. @@ -5300,10 +5303,10 @@ func collectMarshaledJobMetadataMap( if err := it.Close(); err != nil { return nil, err } - for _, record := range p.ExtendedEvalContext().SchemaChangeJobRecords { + if err := p.ExtendedEvalContext().jobs.forEachToCreate(func(record *jobs.Record) error { progressBytes, payloadBytes, err := getPayloadAndProgressFromJobsRecord(p, record) if err != nil { - return nil, err + return err } mj := marshaledJobMetadata{ status: tree.NewDString(string(record.RunningStatus)), @@ -5311,6 +5314,9 @@ func collectMarshaledJobMetadataMap( progressBytes: progressBytes, } m[record.JobID] = mj + return nil + }); err != nil { + return nil, err } return m, nil } diff --git a/pkg/sql/database.go b/pkg/sql/database.go index 6bf506ee155e..0bc108bcee67 100644 --- a/pkg/sql/database.go +++ b/pkg/sql/database.go @@ -73,9 +73,7 @@ func (p *planner) renameDatabase( func (p *planner) writeNonDropDatabaseChange( ctx context.Context, desc *dbdesc.Mutable, jobDesc string, ) error { - if err := p.createNonDropDatabaseChangeJob(ctx, desc.ID, jobDesc); err != nil { - return err - } + p.createNonDropDatabaseChangeJob(ctx, desc.ID, jobDesc) b := p.Txn().NewBatch() if err := p.writeDatabaseChangeToBatch(ctx, desc, b); err != nil { return err diff --git a/pkg/sql/drop_database.go b/pkg/sql/drop_database.go index d7c837b0a9b3..190defc9bb24 100644 --- a/pkg/sql/drop_database.go +++ b/pkg/sql/drop_database.go @@ -173,16 +173,14 @@ func (n *dropDatabaseNode) startExec(params runParams) error { } } - if err := p.createDropDatabaseJob( + p.createDropDatabaseJob( ctx, n.dbDesc.GetID(), schemasIDsToDelete, n.d.getDroppedTableDetails(), n.d.typesToDelete, tree.AsStringWithFQNames(n.n, params.Ann()), - ); err != nil { - return err - } + ) n.dbDesc.SetDropped() b := p.txn.NewBatch() diff --git a/pkg/sql/drop_schema.go b/pkg/sql/drop_schema.go index c2a44c495ac6..6254a5a51aa1 100644 --- a/pkg/sql/drop_schema.go +++ b/pkg/sql/drop_schema.go @@ -161,15 +161,12 @@ func (n *dropSchemaNode) startExec(params runParams) error { } // Create the job to drop the schema. - if err := p.createDropSchemaJob( - params.ctx, + p.createDropSchemaJob( schemaIDs, n.d.getDroppedTableDetails(), n.d.typesToDelete, tree.AsStringWithFQNames(n.n, params.Ann()), - ); err != nil { - return err - } + ) // Log Drop Schema event. This is an auditable log event and is recorded // in the same transaction as table descriptor update. @@ -230,18 +227,17 @@ func (p *planner) dropSchemaImpl( } func (p *planner) createDropSchemaJob( - ctx context.Context, schemas []descpb.ID, tableDropDetails []jobspb.DroppedTableDetails, typesToDrop []*typedesc.Mutable, jobDesc string, -) error { +) { typeIDs := make([]descpb.ID, 0, len(typesToDrop)) for _, t := range typesToDrop { typeIDs = append(typeIDs, t.ID) } - _, err := p.extendedEvalCtx.QueueJob(ctx, p.InternalSQLTxn(), jobs.Record{ + p.extendedEvalCtx.QueueJob(&jobs.Record{ Description: jobDesc, Username: p.User(), DescriptorIDs: schemas, @@ -257,7 +253,6 @@ func (p *planner) createDropSchemaJob( Progress: jobspb.SchemaChangeProgress{}, NonCancelable: true, }) - return err } func (n *dropSchemaNode) Next(params runParams) (bool, error) { return false, nil } diff --git a/pkg/sql/drop_table.go b/pkg/sql/drop_table.go index 10c80902fa90..13c1de1d9a40 100644 --- a/pkg/sql/drop_table.go +++ b/pkg/sql/drop_table.go @@ -415,8 +415,8 @@ func (p *planner) markTableMutationJobsSuccessful( // in a batch only when the transaction commits. So, if a job's record exists // in the cache, we can simply delete that record from cache because the // job is not created yet. - if record, exists := p.ExtendedEvalContext().SchemaChangeJobRecords[tableDesc.ID]; exists && record.JobID == jobID { - delete(p.ExtendedEvalContext().SchemaChangeJobRecords, tableDesc.ID) + if record, exists := p.ExtendedEvalContext().jobs.uniqueToCreate[tableDesc.ID]; exists && record.JobID == jobID { + delete(p.ExtendedEvalContext().jobs.uniqueToCreate, tableDesc.ID) continue } mutationJob, err := p.execCfg.JobRegistry.LoadJobWithTxn(ctx, jobID, p.InternalSQLTxn()) diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 1e223a6cec5b..adcd27f79640 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -34,7 +34,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/featureflag" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/jobs" - "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/keyvisualizer" "github.com/cockroachdb/cockroach/pkg/kv" @@ -2200,16 +2199,6 @@ func (r *SessionRegistry) SerializeAll() []serverpb.Session { // into a serverpb.Session. Exported for testing. const MaxSQLBytes = 1000 -type jobsCollection []jobspb.JobID - -func (jc *jobsCollection) add(ids ...jobspb.JobID) { - *jc = append(*jc, ids...) -} - -func (jc *jobsCollection) reset() { - *jc = nil -} - // truncateStatementStringForTelemetry truncates the string // representation of a statement to a maximum length, so as to not // create unduly large logging and error payloads. diff --git a/pkg/sql/grant_revoke.go b/pkg/sql/grant_revoke.go index e4f6a554bde0..bf9d7ee13114 100644 --- a/pkg/sql/grant_revoke.go +++ b/pkg/sql/grant_revoke.go @@ -338,10 +338,7 @@ func (n *changeDescriptorBackedPrivilegesNode) startExec(params runParams) error if err := p.writeDatabaseChangeToBatch(ctx, d, b); err != nil { return err } - if err := p.createNonDropDatabaseChangeJob(ctx, d.ID, - fmt.Sprintf("updating privileges for database %d", d.ID)); err != nil { - return err - } + p.createNonDropDatabaseChangeJob(ctx, d.ID, fmt.Sprintf("updating privileges for database %d", d.ID)) for _, grantee := range n.grantees { privs := eventDetails // copy the granted/revoked privilege list. privs.Grantee = grantee.Normalized() diff --git a/pkg/sql/internal.go b/pkg/sql/internal.go index a63fe263e567..f58d08125cfb 100644 --- a/pkg/sql/internal.go +++ b/pkg/sql/internal.go @@ -17,7 +17,6 @@ import ( "sync" "time" - "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" @@ -26,7 +25,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catsessiondata" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" "github.com/cockroachdb/cockroach/pkg/sql/isql" @@ -302,7 +300,6 @@ func (ie *InternalExecutor) newConnExecutorWithTxn( if ie.extraTxnState != nil { ex.extraTxnState.descCollection = ie.extraTxnState.descCollection ex.extraTxnState.fromOuterTxn = true - ex.extraTxnState.schemaChangeJobRecords = ie.extraTxnState.schemaChangeJobRecords ex.extraTxnState.jobs = ie.extraTxnState.jobs ex.extraTxnState.schemaChangerState = ie.extraTxnState.schemaChangerState ex.extraTxnState.shouldResetSyntheticDescriptors = shouldResetSyntheticDescriptors @@ -1005,13 +1002,6 @@ func (ie *InternalExecutor) execInternal( return r, nil } -// ReleaseSchemaChangeJobRecords is to release the schema change job records. -func (ie *InternalExecutor) releaseSchemaChangeJobRecords() { - for k := range ie.extraTxnState.schemaChangeJobRecords { - delete(ie.extraTxnState.schemaChangeJobRecords, k) - } -} - // commitTxn is to commit the txn bound to the internal executor. // It should only be used in CollectionFactory.TxnWithExecutor(). func (ie *InternalExecutor) commitTxn(ctx context.Context) error { @@ -1239,11 +1229,10 @@ func (ncl *noopClientLock) RTrim(_ context.Context, pos CmdPos) { // executor in that it may lead to surprising bugs whereby we forget to add // fields here and keep them in sync. type extraTxnState struct { - txn *kv.Txn - descCollection *descs.Collection - jobs *jobsCollection - schemaChangeJobRecords map[descpb.ID]*jobs.Record - schemaChangerState *SchemaChangerState + txn *kv.Txn + descCollection *descs.Collection + jobs *txnJobsCollection + schemaChangerState *SchemaChangerState } // InternalDB stored information needed to construct a new @@ -1361,11 +1350,10 @@ func (ief *InternalDB) newInternalExecutorWithTxn( mon: ief.monitor, memMetrics: ief.memMetrics, extraTxnState: &extraTxnState{ - txn: txn, - descCollection: descCol, - jobs: new(jobsCollection), - schemaChangeJobRecords: make(map[descpb.ID]*jobs.Record), - schemaChangerState: schemaChangerState, + txn: txn, + descCollection: descCol, + jobs: newTxnJobsCollection(), + schemaChangerState: schemaChangerState, }, } populateMinimalSessionData(sd) @@ -1374,12 +1362,11 @@ func (ief *InternalDB) newInternalExecutorWithTxn( commitTxnFunc := func(ctx context.Context) error { defer func() { ie.extraTxnState.jobs.reset() - ie.releaseSchemaChangeJobRecords() }() if err := ie.commitTxn(ctx); err != nil { return err } - return ie.s.cfg.JobRegistry.Run(ctx, *ie.extraTxnState.jobs) + return ie.s.cfg.JobRegistry.Run(ctx, ie.extraTxnState.jobs.created) } return ie, commitTxnFunc diff --git a/pkg/sql/jobs_collection.go b/pkg/sql/jobs_collection.go new file mode 100644 index 000000000000..45aa3642ba06 --- /dev/null +++ b/pkg/sql/jobs_collection.go @@ -0,0 +1,89 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sql + +import ( + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" +) + +type jobsCollection []jobspb.JobID + +func (jc *jobsCollection) add(ids ...jobspb.JobID) { + *jc = append(*jc, ids...) +} + +func (jc *jobsCollection) reset() { + *jc = nil +} + +// txnJobsCollection is used to collect information of all jobs created in a +// transaction. It's also used as a cache of job records created outside +// declarative schema changer. +type txnJobsCollection struct { + // created represents a list of job IDs that has been created and queued to + // system.jobs. + created jobsCollection + // uniqueToCreate contains job records unique to a descriptor ID. Typically, + // this kind of jobs are created when mutating relations, we only allow one + // job for a relation in one transaction. These jobs will be created and + // queued at commit time. + uniqueToCreate map[descpb.ID]*jobs.Record + // uniqueToCreate contains job records that are not unique to a descriptor + // IDs. These jobs will be created and queued at commit time. + nonUniqueToCreate []*jobs.Record +} + +func newTxnJobsCollection() *txnJobsCollection { + ret := &txnJobsCollection{ + uniqueToCreate: make(map[descpb.ID]*jobs.Record), + } + return ret +} + +func (j *txnJobsCollection) addCreatedJobID(jobID ...jobspb.JobID) { + j.created.add(jobID...) +} + +func (j *txnJobsCollection) addNonUniqueJobToCreate(jobRecord *jobs.Record) { + j.nonUniqueToCreate = append(j.nonUniqueToCreate, jobRecord) +} + +func (j *txnJobsCollection) reset() { + j.created.reset() + for id := range j.uniqueToCreate { + delete(j.uniqueToCreate, id) + } + j.nonUniqueToCreate = nil +} + +func (j *txnJobsCollection) numToCreate() int { + return len(j.uniqueToCreate) + len(j.nonUniqueToCreate) +} + +func (j *txnJobsCollection) hasAnyToCreate() bool { + return j.numToCreate() > 0 +} + +func (j *txnJobsCollection) forEachToCreate(fn func(jobRecord *jobs.Record) error) error { + for _, r := range j.uniqueToCreate { + if err := fn(r); err != nil { + return err + } + } + for _, r := range j.nonUniqueToCreate { + if err := fn(r); err != nil { + return err + } + } + return nil +} diff --git a/pkg/sql/opt/exec/execbuilder/testdata/tpch_vec b/pkg/sql/opt/exec/execbuilder/testdata/tpch_vec index f79ad2f450c8..878b2c0c45ca 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/tpch_vec +++ b/pkg/sql/opt/exec/execbuilder/testdata/tpch_vec @@ -20551,25 +20551,24 @@ EXPLAIN (VEC) SELECT s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_ │ └ Node 1 └ *colexec.topKSorter - └ *colexecsel.selEQFloat64Float64Op - └ *colexec.hashAggregator - └ *colexecjoin.hashJoiner - ├ *rowexec.joinReader - │ └ *colexecjoin.hashJoiner - │ ├ *rowexec.joinReader - │ │ └ *colexecsel.selSuffixBytesBytesConstOp - │ │ └ *colexecsel.selEQInt64Int64ConstOp - │ │ └ *colfetcher.ColBatchScan - │ └ *rowexec.joinReader - │ └ *rowexec.joinReader - │ └ *rowexec.joinReader - │ └ *rowexec.joinReader - │ └ *colexecsel.selEQBytesBytesConstOp - │ └ *colfetcher.ColBatchScan - └ *rowexec.joinReader - └ *rowexec.joinReader - └ *colexecsel.selEQBytesBytesConstOp - └ *colfetcher.ColBatchScan + └ *rowexec.joinReader + └ *colexecsel.selEQFloat64Float64Op + └ *colexec.hashAggregator + └ *colexecjoin.hashJoiner + ├ *colexecjoin.hashJoiner + │ ├ *colfetcher.ColBatchScan + │ └ *rowexec.joinReader + │ └ *rowexec.joinReader + │ └ *rowexec.joinReader + │ └ *rowexec.joinReader + │ └ *colexecsel.selEQBytesBytesConstOp + │ └ *colfetcher.ColBatchScan + └ *colexecjoin.hashJoiner + ├ *colfetcher.ColBatchScan + └ *rowexec.joinReader + └ *rowexec.joinReader + └ *colexecsel.selEQBytesBytesConstOp + └ *colfetcher.ColBatchScan # Query 3 query T diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q02 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q02 index 34aaefcca64e..d396cd2281a8 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q02 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q02 @@ -73,541 +73,503 @@ project ├── save-table-name: q2_project_1 ├── columns: s_acctbal:17(float!null) s_name:13(char!null) n_name:29(char!null) p_partkey:1(int!null) p_mfgr:3(char!null) s_address:14(varchar!null) s_phone:16(char!null) s_comment:18(varchar!null) ├── cardinality: [0 - 100] - ├── stats: [rows=1, distinct(1)=1, null(1)=0, distinct(3)=1, null(3)=0, distinct(13)=1, null(13)=0, distinct(14)=1, null(14)=0, distinct(16)=1, null(16)=0, distinct(17)=1, null(17)=0, distinct(18)=1, null(18)=0, distinct(29)=1, null(29)=0] + ├── stats: [rows=1.000013, distinct(1)=1, null(1)=0, distinct(3)=0.906664, null(3)=0, distinct(13)=0.632263, null(13)=0, distinct(14)=0.632263, null(14)=0, distinct(16)=0.632263, null(16)=0, distinct(17)=0.632263, null(17)=0, distinct(18)=0.632263, null(18)=0, distinct(29)=0.632263, null(29)=0] ├── fd: (1)-->(3) ├── ordering: -17,+29,+13,+1 - └── top-k - ├── save-table-name: q2_top_k_2 - ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) s_name:13(char!null) s_address:14(varchar!null) s_phone:16(char!null) s_acctbal:17(float!null) s_comment:18(varchar!null) ps_partkey:21(int!null) ps_suppkey:22(int!null) ps_supplycost:24(float!null) n_name:29(char!null) min:66(float!null) - ├── internal-ordering: -17,+29,+13,+(1|21) - ├── k: 100 + └── limit + ├── save-table-name: q2_limit_2 + ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) p_type:5(varchar!null) p_size:6(int!null) s_name:13(char!null) s_address:14(varchar!null) s_phone:16(char!null) s_acctbal:17(float!null) s_comment:18(varchar!null) ps_partkey:21(int!null) ps_suppkey:22(int!null) ps_supplycost:24(float!null) n_name:29(char!null) min:66(float!null) + ├── internal-ordering: -17,+29,+13,+(1|21) opt(6) ├── cardinality: [0 - 100] - ├── stats: [rows=1, distinct(1)=1, null(1)=0, distinct(3)=1, null(3)=0, distinct(13)=1, null(13)=0, distinct(14)=1, null(14)=0, distinct(16)=1, null(16)=0, distinct(17)=1, null(17)=0, distinct(18)=1, null(18)=0, distinct(21)=0.999912, null(21)=0, distinct(22)=0.999982, null(22)=0, distinct(24)=1, null(24)=0, distinct(29)=1, null(29)=0, distinct(66)=1, null(66)=0] + ├── stats: [rows=1.000013, distinct(1)=1, null(1)=0, distinct(3)=0.906664, null(3)=0, distinct(5)=0.99706, null(5)=0, distinct(6)=0.632263, null(6)=0, distinct(13)=0.632263, null(13)=0, distinct(14)=0.632263, null(14)=0, distinct(16)=0.632263, null(16)=0, distinct(17)=0.632263, null(17)=0, distinct(18)=0.632263, null(18)=0, distinct(21)=1, null(21)=0, distinct(22)=0.632251, null(22)=0, distinct(24)=0.632263, null(24)=0, distinct(29)=0.632263, null(29)=0, distinct(66)=0.632263, null(66)=0] ├── key: (21,22) - ├── fd: (1)-->(3), (21,22)-->(1,3,13,14,16-18,24,29,66), (1)==(21), (21)==(1), (22)-->(13,14,16-18,29), (24)==(66), (66)==(24) - ├── ordering: -17,+29,+13,+(1|21) [actual: -17,+29,+13,+1] - └── select - ├── save-table-name: q2_select_3 - ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) s_name:13(char!null) s_address:14(varchar!null) s_phone:16(char!null) s_acctbal:17(float!null) s_comment:18(varchar!null) ps_partkey:21(int!null) ps_suppkey:22(int!null) ps_supplycost:24(float!null) n_name:29(char!null) min:66(float!null) - ├── stats: [rows=1, distinct(1)=1, null(1)=0, distinct(3)=1, null(3)=0, distinct(13)=1, null(13)=0, distinct(14)=1, null(14)=0, distinct(16)=1, null(16)=0, distinct(17)=1, null(17)=0, distinct(18)=1, null(18)=0, distinct(21)=0.999912, null(21)=0, distinct(22)=0.999982, null(22)=0, distinct(24)=1, null(24)=0, distinct(29)=1, null(29)=0, distinct(66)=1, null(66)=0] - ├── key: (21,22) - ├── fd: (1)-->(3), (21,22)-->(1,3,13,14,16-18,24,29,66), (1)==(21), (21)==(1), (22)-->(13,14,16-18,29), (24)==(66), (66)==(24) - ├── group-by (hash) - │ ├── save-table-name: q2_group_by_4 - │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) s_name:13(char!null) s_address:14(varchar!null) s_phone:16(char!null) s_acctbal:17(float!null) s_comment:18(varchar!null) ps_partkey:21(int!null) ps_suppkey:22(int!null) ps_supplycost:24(float!null) n_name:29(char!null) min:66(float!null) - │ ├── grouping columns: ps_partkey:21(int!null) ps_suppkey:22(int!null) - │ ├── stats: [rows=1475.884, distinct(1)=1475.88, null(1)=0, distinct(3)=1475.88, null(3)=0, distinct(13)=1475.88, null(13)=0, distinct(14)=1475.88, null(14)=0, distinct(16)=1475.88, null(16)=0, distinct(17)=1475.88, null(17)=0, distinct(18)=1475.88, null(18)=0, distinct(21)=1171.39, null(21)=0, distinct(22)=1401.91, null(22)=0, distinct(24)=1475.88, null(24)=0, distinct(29)=1475.88, null(29)=0, distinct(66)=1475.88, null(66)=0, distinct(21,22)=1475.88, null(21,22)=0] - │ ├── key: (21,22) - │ ├── fd: (1)-->(3), (21,22)-->(1,3,13,14,16-18,24,29,66), (1)==(21), (21)==(1), (22)-->(13,14,16-18,29) - │ ├── inner-join (hash) - │ │ ├── save-table-name: q2_inner_join_5 - │ │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) p_type:5(varchar!null) p_size:6(int!null) s_suppkey:12(int!null) s_name:13(char!null) s_address:14(varchar!null) s_nationkey:15(int!null) s_phone:16(char!null) s_acctbal:17(float!null) s_comment:18(varchar!null) ps_partkey:21(int!null) ps_suppkey:22(int!null) ps_supplycost:24(float!null) n_nationkey:28(int!null) n_name:29(char!null) n_regionkey:30(int!null) r_regionkey:34(int!null) r_name:35(char!null) ps_partkey:39(int!null) ps_suppkey:40(int!null) ps_supplycost:42(float!null) s_suppkey:46(int!null) s_nationkey:49(int!null) n_nationkey:55(int!null) n_regionkey:57(int!null) r_regionkey:61(int!null) r_name:62(char!null) - │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ ├── stats: [rows=2815.995, distinct(1)=1332.34, null(1)=0, distinct(3)=5, null(3)=0, distinct(5)=150, null(5)=0, distinct(6)=1, null(6)=0, distinct(12)=1401.91, null(12)=0, distinct(13)=1402.47, null(13)=0, distinct(14)=1402.54, null(14)=0, distinct(15)=5, null(15)=0, distinct(16)=1402.54, null(16)=0, distinct(17)=1402.29, null(17)=0, distinct(18)=1402.02, null(18)=0, distinct(21)=1171.39, null(21)=0, distinct(22)=1401.91, null(22)=0, distinct(24)=1471.7, null(24)=0, distinct(28)=5, null(28)=0, distinct(29)=5, null(29)=0, distinct(30)=1, null(30)=0, distinct(34)=1, null(34)=0, distinct(35)=0.996222, null(35)=0, distinct(39)=1332.34, null(39)=0, distinct(40)=1443.92, null(40)=0, distinct(42)=2767.17, null(42)=0, distinct(46)=1443.92, null(46)=0, distinct(49)=5, null(49)=0, distinct(55)=5, null(55)=0, distinct(57)=1, null(57)=0, distinct(61)=1, null(61)=0, distinct(62)=0.996222, null(62)=0, distinct(21,22)=1475.88, null(21,22)=0] - │ │ ├── key: (22,39,46) - │ │ ├── fd: ()-->(6,35,62), (1)-->(3,5), (12)-->(13-18), (21,22)-->(24), (12)==(22), (22)==(12), (28)-->(29,30), (30)==(34), (34)==(30), (15)==(28), (28)==(15), (1)==(21,39), (21)==(1,39), (39,40)-->(42), (46)-->(49), (55)-->(57), (57)==(61), (61)==(57), (49)==(55), (55)==(49), (40)==(46), (46)==(40), (39)==(1,21) - │ │ ├── inner-join (lookup partsupp) - │ │ │ ├── save-table-name: q2_lookup_join_6 - │ │ │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) p_type:5(varchar!null) p_size:6(int!null) s_suppkey:12(int!null) s_name:13(char!null) s_address:14(varchar!null) s_nationkey:15(int!null) s_phone:16(char!null) s_acctbal:17(float!null) s_comment:18(varchar!null) ps_partkey:21(int!null) ps_suppkey:22(int!null) ps_supplycost:24(float!null) n_nationkey:28(int!null) n_name:29(char!null) n_regionkey:30(int!null) r_regionkey:34(int!null) r_name:35(char!null) ps_partkey:39(int!null) ps_suppkey:40(int!null) ps_supplycost:42(float!null) - │ │ │ ├── key columns: [1] = [39] - │ │ │ ├── stats: [rows=4449.128, distinct(1)=1332.34, null(1)=0, distinct(3)=5, null(3)=0, distinct(5)=150, null(5)=0, distinct(6)=1, null(6)=0, distinct(12)=3585.25, null(12)=0, distinct(13)=3590.47, null(13)=0, distinct(14)=3591.21, null(14)=0, distinct(15)=25, null(15)=0, distinct(16)=3591.21, null(16)=0, distinct(17)=3588.76, null(17)=0, distinct(18)=3586.29, null(18)=0, distinct(21)=1332.34, null(21)=0, distinct(22)=3585.25, null(22)=0, distinct(24)=4351.91, null(24)=0, distinct(28)=25, null(28)=0, distinct(29)=25, null(29)=0, distinct(30)=1, null(30)=0, distinct(34)=1, null(34)=0, distinct(35)=1, null(35)=0, distinct(39)=4398.49, null(39)=0, distinct(40)=3585.25, null(40)=0, distinct(42)=4351.91, null(42)=0] - │ │ │ ├── key: (22,39,40) - │ │ │ ├── fd: ()-->(6,35), (1)-->(3,5), (28)-->(29,30), (12)-->(13-18), (21,22)-->(24), (39,40)-->(42), (21)==(1,39), (39)==(1,21), (12)==(22), (22)==(12), (15)==(28), (28)==(15), (30)==(34), (34)==(30), (1)==(21,39) - │ │ │ ├── inner-join (hash) - │ │ │ │ ├── save-table-name: q2_inner_join_7 - │ │ │ │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) p_type:5(varchar!null) p_size:6(int!null) s_suppkey:12(int!null) s_name:13(char!null) s_address:14(varchar!null) s_nationkey:15(int!null) s_phone:16(char!null) s_acctbal:17(float!null) s_comment:18(varchar!null) ps_partkey:21(int!null) ps_suppkey:22(int!null) ps_supplycost:24(float!null) n_nationkey:28(int!null) n_name:29(char!null) n_regionkey:30(int!null) r_regionkey:34(int!null) r_name:35(char!null) - │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ ├── stats: [rows=1930.433, distinct(1)=1332.34, null(1)=0, distinct(3)=5, null(3)=0, distinct(5)=150, null(5)=0, distinct(6)=1, null(6)=0, distinct(12)=1754.22, null(12)=0, distinct(13)=1755.38, null(13)=0, distinct(14)=1755.54, null(14)=0, distinct(15)=5, null(15)=0, distinct(16)=1755.54, null(16)=0, distinct(17)=1755, null(17)=0, distinct(18)=1754.46, null(18)=0, distinct(21)=1332.34, null(21)=0, distinct(22)=1754.22, null(22)=0, distinct(24)=1907.59, null(24)=0, distinct(28)=5, null(28)=0, distinct(29)=5, null(29)=0, distinct(30)=1, null(30)=0, distinct(34)=1, null(34)=0, distinct(35)=0.996222, null(35)=0, distinct(21,22)=1917.25, null(21,22)=0] - │ │ │ │ ├── key: (21,22) - │ │ │ │ ├── fd: ()-->(6,35), (1)-->(3,5), (12)-->(13-18), (21,22)-->(24), (12)==(22), (22)==(12), (28)-->(29,30), (30)==(34), (34)==(30), (15)==(28), (28)==(15), (1)==(21), (21)==(1) - │ │ │ │ ├── inner-join (lookup partsupp) - │ │ │ │ │ ├── save-table-name: q2_lookup_join_8 - │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) p_type:5(varchar!null) p_size:6(int!null) ps_partkey:21(int!null) ps_suppkey:22(int!null) ps_supplycost:24(float!null) - │ │ │ │ │ ├── key columns: [1] = [21] - │ │ │ │ │ ├── stats: [rows=5349.747, distinct(1)=1332.34, null(1)=0, distinct(3)=5, null(3)=0, distinct(5)=150, null(5)=0, distinct(6)=1, null(6)=0, distinct(21)=1332.34, null(21)=0, distinct(22)=4135.03, null(22)=0, distinct(24)=5209.7, null(24)=0] - │ │ │ │ │ ├── key: (21,22) - │ │ │ │ │ ├── fd: ()-->(6), (1)-->(3,5), (21,22)-->(24), (1)==(21), (21)==(1) - │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── save-table-name: q2_select_9 - │ │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) p_type:5(varchar!null) p_size:6(int!null) - │ │ │ │ │ │ ├── stats: [rows=1332.361, distinct(1)=1332.34, null(1)=0, distinct(3)=5, null(3)=0, distinct(5)=150, null(5)=0, distinct(6)=1, null(6)=0] - │ │ │ │ │ │ │ histogram(6)= 0 1332.4 - │ │ │ │ │ │ │ <---- 15 - - │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ ├── fd: ()-->(6), (1)-->(3,5) - │ │ │ │ │ │ ├── scan part - │ │ │ │ │ │ │ ├── save-table-name: q2_scan_10 - │ │ │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) p_type:5(varchar!null) p_size:6(int!null) - │ │ │ │ │ │ │ ├── stats: [rows=200000, distinct(1)=199241, null(1)=0, distinct(3)=5, null(3)=0, distinct(5)=150, null(5)=0, distinct(6)=50, null(6)=0] - │ │ │ │ │ │ │ │ histogram(1)= 0 3.9982 929.57 3.9982 1135.5 3.9982 923.58 3.9982 1036.5 3.9982 964.56 3.9982 953.56 3.9982 899.59 3.9982 1152.5 3.9982 1118.5 3.9982 1137.5 3.9982 1129.5 3.9982 1136.5 3.9982 983.55 3.9982 983.55 3.9982 1028.5 3.9982 1007.5 3.9982 1036.5 3.9982 884.59 3.9982 985.55 3.9982 970.55 3.9982 1036.5 3.9982 943.57 3.9982 1020.5 3.9982 1001.5 3.9982 1001.5 3.9982 954.56 3.9982 1036.5 3.9982 990.54 3.9982 928.57 3.9982 1010.5 3.9982 892.59 3.9982 960.56 3.9982 1059.5 3.9982 947.56 3.9982 906.58 3.9982 935.57 3.9982 860.6 3.9982 971.55 3.9982 1067.5 3.9982 994.54 3.9982 961.56 3.9982 943.57 3.9982 901.59 3.9982 972.55 3.9982 956.56 3.9982 1106.5 3.9982 1152.5 3.9982 967.55 3.9982 943.57 3.9982 916.58 3.9982 1076.5 3.9982 933.57 3.9982 1108.5 3.9982 1081.5 3.9982 975.55 3.9982 1021.5 3.9982 1034.5 3.9982 905.58 3.9982 902.58 3.9982 966.56 3.9982 1080.5 3.9982 927.57 3.9982 936.57 3.9982 1008.5 3.9982 1033.5 3.9982 903.58 3.9982 944.57 3.9982 908.58 3.9982 1008.5 3.9982 1059.5 3.9982 1079.5 3.9982 911.58 3.9982 1107.5 3.9982 992.54 3.9982 975.55 3.9982 1156.5 3.9982 1042.5 3.9982 1072.5 3.9982 916.58 3.9982 1022.5 3.9982 999.54 3.9982 966.56 3.9982 936.57 3.9982 934.57 3.9982 969.55 3.9982 1136.5 3.9982 997.54 3.9982 991.54 3.9982 1002.5 3.9982 1047.5 3.9982 1059.5 3.9982 972.55 3.9982 918.58 3.9982 959.56 3.9982 1083.5 3.9982 934.57 3.9982 900.59 3.9982 970.55 3.9982 952.56 3.9982 1063.5 3.9982 870.6 3.9982 958.56 3.9982 1029.5 3.9982 943.57 3.9982 872.6 3.9982 972.55 3.9982 1009.5 3.9982 875.6 3.9982 1127.5 3.9982 987.55 3.9982 1156.5 3.9982 971.55 3.9982 1155.5 3.9982 930.57 3.9982 1051.5 3.9982 1044.5 3.9982 867.6 3.9982 898.59 3.9982 926.57 3.9982 965.56 3.9982 1027.5 3.9982 993.54 3.9982 927.57 3.9982 973.55 3.9982 934.57 3.9982 951.56 3.9982 1007.5 3.9982 1124.5 3.9982 936.57 3.9982 1050.5 3.9982 1075.5 3.9982 1028.5 3.9982 872.6 3.9982 960.56 3.9982 1014.5 3.9982 1017.5 3.9982 860.6 3.9982 1039.5 3.9982 1059.5 3.9982 921.58 3.9982 936.57 3.9982 1024.5 3.9982 970.55 3.9982 1047.5 3.9982 917.58 3.9982 948.56 3.9982 978.55 3.9982 993.54 3.9982 1121.5 3.9982 944.57 3.9982 1005.5 3.9982 1037.5 3.9982 1261.4 3.9982 1062.5 3.9982 925.57 3.9982 976.55 3.9982 892.59 3.9982 972.55 3.9982 1135.5 3.9982 1044.5 3.9982 959.56 3.9982 990.54 3.9982 993.54 3.9982 1130.5 3.9982 919.58 3.9982 1025.5 3.9982 1001.5 3.9982 974.55 3.9982 1061.5 3.9982 1166.5 3.9982 1017.5 3.9982 1063.5 3.9982 1188.5 3.9982 964.56 3.9982 1047.5 3.9982 1210.4 3.9982 1087.5 3.9982 1151.5 3.9982 1096.5 3.9982 957.56 3.9982 1073.5 3.9982 925.57 3.9982 1051.5 3.9982 930.57 3.9982 1005.5 3.9982 977.55 3.9982 963.56 3.9982 1005.5 3.9982 954.56 3.9982 1025.5 3.9982 1039.5 3.9982 985.55 3.9982 923.58 3.9982 1087.5 3.9982 958.56 3.9982 1066.5 3.9982 1110.5 3.9982 934.57 3.9982 946.56 3.9982 - │ │ │ │ │ │ │ │ <---- 23 --------- 901 --------- 2150 -------- 3016 -------- 4093 -------- 5038 -------- 5962 -------- 6778 -------- 8056 -------- 9277 -------- 10530 -------- 11769 -------- 13020 -------- 14001 -------- 14982 -------- 16046 -------- 17072 -------- 18149 -------- 18935 -------- 19920 -------- 20876 -------- 21953 -------- 22859 -------- 23908 -------- 24923 -------- 25938 -------- 26865 -------- 27943 -------- 28938 -------- 29813 -------- 30844 -------- 31647 -------- 32585 -------- 33704 -------- 34617 -------- 35448 -------- 36338 ------- 37071 -------- 38029 -------- 39162 -------- 40163 -------- 41103 -------- 42008 -------- 42828 -------- 43789 -------- 44720 -------- 45920 -------- 47197 -------- 48149 -------- 49054 -------- 49906 -------- 51054 -------- 51940 -------- 53144 -------- 54301 -------- 55267 -------- 56318 -------- 57393 -------- 58223 -------- 59046 -------- 59995 -------- 61150 -------- 62024 -------- 62915 -------- 63943 -------- 65015 -------- 65840 -------- 66748 -------- 67584 -------- 68611 -------- 69729 -------- 70883 -------- 71725 -------- 72926 -------- 73924 -------- 74891 -------- 76176 -------- 77264 -------- 78405 -------- 79257 -------- 80310 -------- 81321 -------- 82270 -------- 83162 -------- 84049 -------- 85004 -------- 86255 -------- 87262 -------- 88259 -------- 89276 -------- 90374 -------- 91493 -------- 92454 -------- 93310 -------- 94246 -------- 95407 -------- 96295 -------- 97113 -------- 98069 -------- 98991 -------- 100116 ------- 100871 -------- 101805 -------- 102871 -------- 103776 ------- 104536 -------- 105497 -------- 106526 ------- 107293 -------- 108529 -------- 109518 -------- 110802 -------- 111761 -------- 113044 -------- 113923 -------- 115027 -------- 116119 ------- 116867 -------- 117681 -------- 118553 -------- 119501 -------- 120563 -------- 121563 -------- 122437 -------- 123400 -------- 124288 -------- 125209 -------- 126234 -------- 127465 -------- 128356 -------- 129458 -------- 130604 -------- 131668 ------- 132428 -------- 133365 -------- 134403 -------- 135446 ------- 136179 -------- 137262 -------- 138380 -------- 139242 -------- 140134 -------- 141190 -------- 142146 -------- 143244 -------- 144097 -------- 145011 -------- 145982 -------- 146981 -------- 148207 -------- 149115 -------- 150119 -------- 151183 -------- 152627 -------- 153735 -------- 154585 -------- 155535 -------- 156315 -------- 157258 -------- 158494 -------- 159570 -------- 160487 -------- 161464 -------- 162446 -------- 163673 -------- 164509 -------- 165550 -------- 166548 -------- 167495 -------- 168601 -------- 169889 -------- 170916 -------- 172026 -------- 173351 -------- 174278 -------- 175359 -------- 176720 -------- 177872 -------- 179135 -------- 180304 -------- 181217 -------- 182345 -------- 183194 -------- 184282 -------- 185142 -------- 186147 -------- 187099 -------- 188024 -------- 189029 -------- 189936 -------- 190977 -------- 192044 -------- 193012 -------- 193858 -------- 195011 -------- 195927 -------- 197043 -------- 198236 -------- 199104 -------- 199995 - │ │ │ │ │ │ │ │ histogram(3)= 0 40940 1.1794e+05 41120 - │ │ │ │ │ │ │ │ <--- 'Manufacturer#1' ------------ 'Manufacturer#5' - │ │ │ │ │ │ │ │ histogram(5)= 0 1360 1.9708e+05 1560 - │ │ │ │ │ │ │ │ <--- 'ECONOMY ANODIZED BRASS' ------------ 'STANDARD POLISHED TIN' - │ │ │ │ │ │ │ │ histogram(6)= 0 4240 1.9186e+05 3900 - │ │ │ │ │ │ │ │ <--- 1 ------------- 50 - │ │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ │ └── fd: (1)-->(3,5,6) - │ │ │ │ │ │ └── filters - │ │ │ │ │ │ ├── p_size:6 = 15 [type=bool, outer=(6), constraints=(/6: [/15 - /15]; tight), fd=()-->(6)] - │ │ │ │ │ │ └── p_type:5 LIKE '%BRASS' [type=bool, outer=(5), constraints=(/5: (/NULL - ])] - │ │ │ │ │ └── filters (true) - │ │ │ │ ├── inner-join (hash) - │ │ │ │ │ ├── save-table-name: q2_inner_join_11 - │ │ │ │ │ ├── columns: s_suppkey:12(int!null) s_name:13(char!null) s_address:14(varchar!null) s_nationkey:15(int!null) s_phone:16(char!null) s_acctbal:17(float!null) s_comment:18(varchar!null) n_nationkey:28(int!null) n_name:29(char!null) n_regionkey:30(int!null) r_regionkey:34(int!null) r_name:35(char!null) - │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ │ ├── stats: [rows=2000, distinct(12)=1844.81, null(12)=0, distinct(13)=1846.09, null(13)=0, distinct(14)=1846.27, null(14)=0, distinct(15)=5, null(15)=0, distinct(16)=1846.27, null(16)=0, distinct(17)=1845.67, null(17)=0, distinct(18)=1845.06, null(18)=0, distinct(28)=5, null(28)=0, distinct(29)=5, null(29)=0, distinct(30)=1, null(30)=0, distinct(34)=1, null(34)=0, distinct(35)=0.996222, null(35)=0] - │ │ │ │ │ ├── key: (12) - │ │ │ │ │ ├── fd: ()-->(35), (12)-->(13-18), (28)-->(29,30), (30)==(34), (34)==(30), (15)==(28), (28)==(15) - │ │ │ │ │ ├── scan supplier - │ │ │ │ │ │ ├── save-table-name: q2_scan_12 - │ │ │ │ │ │ ├── columns: s_suppkey:12(int!null) s_name:13(char!null) s_address:14(varchar!null) s_nationkey:15(int!null) s_phone:16(char!null) s_acctbal:17(float!null) s_comment:18(varchar!null) - │ │ │ │ │ │ ├── stats: [rows=10000, distinct(12)=9920, null(12)=0, distinct(13)=9990, null(13)=0, distinct(14)=10000, null(14)=0, distinct(15)=25, null(15)=0, distinct(16)=10000, null(16)=0, distinct(17)=9967, null(17)=0, distinct(18)=9934, null(18)=0] - │ │ │ │ │ │ │ histogram(12)= 0 0 0 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 0 0 - │ │ │ │ │ │ │ <--- -9223372036854775808 --- 1 ---- 51 ---- 101 ---- 151 ---- 201 ---- 251 ---- 301 ---- 351 ---- 401 ---- 451 ---- 501 ---- 551 ---- 601 ---- 651 ---- 701 ---- 751 ---- 801 ---- 851 ---- 901 ---- 951 ---- 1001 ---- 1051 ---- 1101 ---- 1151 ---- 1201 ---- 1251 ---- 1301 ---- 1351 ---- 1401 ---- 1451 ---- 1501 ---- 1551 ---- 1601 ---- 1651 ---- 1701 ---- 1751 ---- 1801 ---- 1851 ---- 1901 ---- 1951 ---- 2001 ---- 2051 ---- 2101 ---- 2151 ---- 2201 ---- 2251 ---- 2301 ---- 2351 ---- 2401 ---- 2451 ---- 2501 ---- 2551 ---- 2601 ---- 2651 ---- 2701 ---- 2751 ---- 2801 ---- 2851 ---- 2901 ---- 2951 ---- 3001 ---- 3051 ---- 3101 ---- 3151 ---- 3201 ---- 3251 ---- 3301 ---- 3351 ---- 3401 ---- 3451 ---- 3501 ---- 3551 ---- 3601 ---- 3651 ---- 3701 ---- 3751 ---- 3801 ---- 3851 ---- 3901 ---- 3951 ---- 4001 ---- 4051 ---- 4101 ---- 4151 ---- 4201 ---- 4251 ---- 4301 ---- 4351 ---- 4401 ---- 4451 ---- 4501 ---- 4551 ---- 4601 ---- 4651 ---- 4701 ---- 4751 ---- 4801 ---- 4851 ---- 4901 ---- 4951 ---- 5001 ---- 5051 ---- 5101 ---- 5151 ---- 5201 ---- 5251 ---- 5301 ---- 5351 ---- 5401 ---- 5451 ---- 5501 ---- 5551 ---- 5601 ---- 5651 ---- 5701 ---- 5751 ---- 5801 ---- 5851 ---- 5901 ---- 5951 ---- 6001 ---- 6051 ---- 6101 ---- 6151 ---- 6201 ---- 6251 ---- 6301 ---- 6351 ---- 6401 ---- 6451 ---- 6501 ---- 6551 ---- 6601 ---- 6651 ---- 6701 ---- 6751 ---- 6801 ---- 6851 ---- 6901 ---- 6951 ---- 7001 ---- 7051 ---- 7101 ---- 7151 ---- 7201 ---- 7251 ---- 7301 ---- 7351 ---- 7401 ---- 7451 ---- 7501 ---- 7552 ---- 7603 ---- 7654 ---- 7705 ---- 7756 ---- 7807 ---- 7858 ---- 7909 ---- 7960 ---- 8011 ---- 8062 ---- 8113 ---- 8164 ---- 8215 ---- 8266 ---- 8317 ---- 8368 ---- 8419 ---- 8470 ---- 8521 ---- 8572 ---- 8623 ---- 8674 ---- 8725 ---- 8776 ---- 8827 ---- 8878 ---- 8929 ---- 8980 ---- 9031 ---- 9082 ---- 9133 ---- 9184 ---- 9235 ---- 9286 ---- 9337 ---- 9388 ---- 9439 ---- 9490 ---- 9541 ---- 9592 ---- 9643 ---- 9694 ---- 9745 ---- 9796 ---- 9847 ---- 9898 ---- 9949 ---- 10000 --- 9223372036854775807 - │ │ │ │ │ │ │ histogram(13)= 0 1 9998 1 - │ │ │ │ │ │ │ <--- 'Supplier#000000001' ------ 'Supplier#000010000' - │ │ │ │ │ │ │ histogram(14)= 0 1 9998 1 - │ │ │ │ │ │ │ <--- ' 9aW1wwnBJJPnCx,nox0MA48Y0zpI1IeVfYZ' ------ 'zzfDhdtZcvmVzA8rNFU,Yctj1zBN' - │ │ │ │ │ │ │ histogram(15)= 0 420 0 413 0 397 0 412 0 415 0 380 0 402 0 396 0 415 0 405 0 393 0 438 0 377 0 362 0 376 0 373 0 406 0 421 0 407 0 398 0 411 0 399 0 401 0 390 0 393 - │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 - │ │ │ │ │ │ │ histogram(16)= 0 1 9998 1 - │ │ │ │ │ │ │ <--- '10-102-116-6785' ------ '34-998-900-4911' - │ │ │ │ │ │ │ histogram(17)= 0 1 9998 1 - │ │ │ │ │ │ │ <--- -998.22 ------ 9999.72 - │ │ │ │ │ │ │ histogram(18)= 0 1 9998 1 - │ │ │ │ │ │ │ <--- ' about the blithely express foxes. bli' ------ 'zzle furiously. bold accounts haggle furiously ironic excuses. fur' - │ │ │ │ │ │ ├── key: (12) - │ │ │ │ │ │ └── fd: (12)-->(13-18) - │ │ │ │ │ ├── inner-join (hash) - │ │ │ │ │ │ ├── save-table-name: q2_inner_join_13 - │ │ │ │ │ │ ├── columns: n_nationkey:28(int!null) n_name:29(char!null) n_regionkey:30(int!null) r_regionkey:34(int!null) r_name:35(char!null) - │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ │ │ ├── stats: [rows=5, distinct(28)=5, null(28)=0, distinct(29)=5, null(29)=0, distinct(30)=1, null(30)=0, distinct(34)=1, null(34)=0, distinct(35)=0.996222, null(35)=0] - │ │ │ │ │ │ ├── key: (28) - │ │ │ │ │ │ ├── fd: ()-->(35), (28)-->(29,30), (30)==(34), (34)==(30) - │ │ │ │ │ │ ├── scan nation - │ │ │ │ │ │ │ ├── save-table-name: q2_scan_14 - │ │ │ │ │ │ │ ├── columns: n_nationkey:28(int!null) n_name:29(char!null) n_regionkey:30(int!null) - │ │ │ │ │ │ │ ├── stats: [rows=25, distinct(28)=25, null(28)=0, distinct(29)=25, null(29)=0, distinct(30)=5, null(30)=0] - │ │ │ │ │ │ │ │ histogram(28)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 - │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 - │ │ │ │ │ │ │ │ histogram(29)= 0 1 23 1 - │ │ │ │ │ │ │ │ <--- 'ALGERIA' ---- 'VIETNAM' - │ │ │ │ │ │ │ │ histogram(30)= 0 5 0 5 0 5 0 5 0 5 - │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 - │ │ │ │ │ │ │ ├── key: (28) - │ │ │ │ │ │ │ └── fd: (28)-->(29,30) - │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── save-table-name: q2_select_15 - │ │ │ │ │ │ │ ├── columns: r_regionkey:34(int!null) r_name:35(char!null) - │ │ │ │ │ │ │ ├── stats: [rows=1, distinct(34)=1, null(34)=0, distinct(35)=1, null(35)=0] - │ │ │ │ │ │ │ │ histogram(35)= 0 1 - │ │ │ │ │ │ │ │ <--- 'EUROPE' - │ │ │ │ │ │ │ ├── key: (34) - │ │ │ │ │ │ │ ├── fd: ()-->(35) - │ │ │ │ │ │ │ ├── scan region - │ │ │ │ │ │ │ │ ├── save-table-name: q2_scan_16 - │ │ │ │ │ │ │ │ ├── columns: r_regionkey:34(int!null) r_name:35(char!null) - │ │ │ │ │ │ │ │ ├── stats: [rows=5, distinct(34)=5, null(34)=0, distinct(35)=5, null(35)=0] - │ │ │ │ │ │ │ │ │ histogram(34)= 0 1 0 1 0 1 0 1 0 1 - │ │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 - │ │ │ │ │ │ │ │ │ histogram(35)= 0 1 3 1 - │ │ │ │ │ │ │ │ │ <--- 'AFRICA' --- 'MIDDLE EAST' - │ │ │ │ │ │ │ │ ├── key: (34) - │ │ │ │ │ │ │ │ └── fd: (34)-->(35) - │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── r_name:35 = 'EUROPE' [type=bool, outer=(35), constraints=(/35: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(35)] - │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── n_regionkey:30 = r_regionkey:34 [type=bool, outer=(30,34), constraints=(/30: (/NULL - ]; /34: (/NULL - ]), fd=(30)==(34), (34)==(30)] - │ │ │ │ │ └── filters - │ │ │ │ │ └── s_nationkey:15 = n_nationkey:28 [type=bool, outer=(15,28), constraints=(/15: (/NULL - ]; /28: (/NULL - ]), fd=(15)==(28), (28)==(15)] - │ │ │ │ └── filters - │ │ │ │ └── s_suppkey:12 = ps_suppkey:22 [type=bool, outer=(12,22), constraints=(/12: (/NULL - ]; /22: (/NULL - ]), fd=(12)==(22), (22)==(12)] - │ │ │ └── filters (true) - │ │ ├── inner-join (lookup supplier@s_nk) - │ │ │ ├── save-table-name: q2_lookup_join_17 - │ │ │ ├── columns: s_suppkey:46(int!null) s_nationkey:49(int!null) n_nationkey:55(int!null) n_regionkey:57(int!null) r_regionkey:61(int!null) r_name:62(char!null) - │ │ │ ├── key columns: [55] = [49] - │ │ │ ├── stats: [rows=2000, distinct(46)=1844.81, null(46)=0, distinct(49)=5, null(49)=0, distinct(55)=5, null(55)=0, distinct(57)=1, null(57)=0, distinct(61)=1, null(61)=0, distinct(62)=0.996222, null(62)=0] - │ │ │ ├── key: (46) - │ │ │ ├── fd: ()-->(62), (46)-->(49), (55)-->(57), (57)==(61), (61)==(57), (49)==(55), (55)==(49) - │ │ │ ├── inner-join (lookup nation@n_rk) - │ │ │ │ ├── save-table-name: q2_lookup_join_18 - │ │ │ │ ├── columns: n_nationkey:55(int!null) n_regionkey:57(int!null) r_regionkey:61(int!null) r_name:62(char!null) - │ │ │ │ ├── key columns: [61] = [57] - │ │ │ │ ├── stats: [rows=5, distinct(55)=5, null(55)=0, distinct(57)=1, null(57)=0, distinct(61)=1, null(61)=0, distinct(62)=0.996222, null(62)=0] - │ │ │ │ ├── key: (55) - │ │ │ │ ├── fd: ()-->(62), (55)-->(57), (57)==(61), (61)==(57) - │ │ │ │ ├── select - │ │ │ │ │ ├── save-table-name: q2_select_19 - │ │ │ │ │ ├── columns: r_regionkey:61(int!null) r_name:62(char!null) - │ │ │ │ │ ├── stats: [rows=1, distinct(61)=1, null(61)=0, distinct(62)=1, null(62)=0] - │ │ │ │ │ │ histogram(62)= 0 1 - │ │ │ │ │ │ <--- 'EUROPE' - │ │ │ │ │ ├── key: (61) - │ │ │ │ │ ├── fd: ()-->(62) - │ │ │ │ │ ├── scan region - │ │ │ │ │ │ ├── save-table-name: q2_scan_20 - │ │ │ │ │ │ ├── columns: r_regionkey:61(int!null) r_name:62(char!null) - │ │ │ │ │ │ ├── stats: [rows=5, distinct(61)=5, null(61)=0, distinct(62)=5, null(62)=0] - │ │ │ │ │ │ │ histogram(61)= 0 1 0 1 0 1 0 1 0 1 - │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 - │ │ │ │ │ │ │ histogram(62)= 0 1 3 1 - │ │ │ │ │ │ │ <--- 'AFRICA' --- 'MIDDLE EAST' - │ │ │ │ │ │ ├── key: (61) - │ │ │ │ │ │ └── fd: (61)-->(62) - │ │ │ │ │ └── filters - │ │ │ │ │ └── r_name:62 = 'EUROPE' [type=bool, outer=(62), constraints=(/62: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(62)] - │ │ │ │ └── filters (true) - │ │ │ └── filters (true) - │ │ └── filters - │ │ └── s_suppkey:46 = ps_suppkey:40 [type=bool, outer=(40,46), constraints=(/40: (/NULL - ]; /46: (/NULL - ]), fd=(40)==(46), (46)==(40)] - │ └── aggregations - │ ├── min [as=min:66, type=float, outer=(42)] - │ │ └── ps_supplycost:42 [type=float] - │ ├── const-agg [as=s_name:13, type=char, outer=(13)] - │ │ └── s_name:13 [type=char] - │ ├── const-agg [as=s_address:14, type=varchar, outer=(14)] - │ │ └── s_address:14 [type=varchar] - │ ├── const-agg [as=s_phone:16, type=char, outer=(16)] - │ │ └── s_phone:16 [type=char] - │ ├── const-agg [as=s_acctbal:17, type=float, outer=(17)] - │ │ └── s_acctbal:17 [type=float] - │ ├── const-agg [as=s_comment:18, type=varchar, outer=(18)] - │ │ └── s_comment:18 [type=varchar] - │ ├── const-agg [as=ps_supplycost:24, type=float, outer=(24)] - │ │ └── ps_supplycost:24 [type=float] - │ ├── const-agg [as=n_name:29, type=char, outer=(29)] - │ │ └── n_name:29 [type=char] - │ ├── const-agg [as=p_mfgr:3, type=char, outer=(3)] - │ │ └── p_mfgr:3 [type=char] - │ └── const-agg [as=p_partkey:1, type=int, outer=(1)] - │ └── p_partkey:1 [type=int] - └── filters - └── ps_supplycost:24 = min:66 [type=bool, outer=(24,66), constraints=(/24: (/NULL - ]; /66: (/NULL - ]), fd=(24)==(66), (66)==(24)] + ├── fd: ()-->(6), (1)-->(3,5), (21,22)-->(13,14,16-18,24,29,66), (22)-->(13,14,16-18,29), (24)==(66), (66)==(24), (1)==(21), (21)==(1) + ├── ordering: -17,+29,+13,+(1|21) opt(6) [actual: -17,+29,+13,+21] + ├── inner-join (lookup part) + │ ├── save-table-name: q2_lookup_join_3 + │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) p_type:5(varchar!null) p_size:6(int!null) s_name:13(char!null) s_address:14(varchar!null) s_phone:16(char!null) s_acctbal:17(float!null) s_comment:18(varchar!null) ps_partkey:21(int!null) ps_suppkey:22(int!null) ps_supplycost:24(float!null) n_name:29(char!null) min:66(float!null) + │ ├── key columns: [21] = [1] + │ ├── lookup columns are key + │ ├── stats: [rows=1.000013, distinct(1)=1, null(1)=0, distinct(3)=0.906664, null(3)=0, distinct(5)=0.99706, null(5)=0, distinct(6)=0.632263, null(6)=0, distinct(13)=0.632263, null(13)=0, distinct(14)=0.632263, null(14)=0, distinct(16)=0.632263, null(16)=0, distinct(17)=0.632263, null(17)=0, distinct(18)=0.632263, null(18)=0, distinct(21)=1, null(21)=0, distinct(22)=0.632251, null(22)=0, distinct(24)=0.632263, null(24)=0, distinct(29)=0.632263, null(29)=0, distinct(66)=0.632263, null(66)=0] + │ ├── key: (21,22) + │ ├── fd: ()-->(6), (1)-->(3,5), (21,22)-->(13,14,16-18,24,29,66), (22)-->(13,14,16-18,29), (24)==(66), (66)==(24), (1)==(21), (21)==(1) + │ ├── ordering: -17,+29,+13,+(1|21) opt(6) [actual: -17,+29,+13,+21] + │ ├── limit hint: 100.00 + │ ├── sort + │ │ ├── save-table-name: q2_sort_4 + │ │ ├── columns: s_name:13(char!null) s_address:14(varchar!null) s_phone:16(char!null) s_acctbal:17(float!null) s_comment:18(varchar!null) ps_partkey:21(int!null) ps_suppkey:22(int!null) ps_supplycost:24(float!null) n_name:29(char!null) min:66(float!null) + │ │ ├── stats: [rows=1, distinct(13)=1, null(13)=0, distinct(14)=1, null(14)=0, distinct(16)=1, null(16)=0, distinct(17)=1, null(17)=0, distinct(18)=1, null(18)=0, distinct(21)=1, null(21)=0, distinct(22)=0.999954, null(22)=0, distinct(24)=1, null(24)=0, distinct(29)=1, null(29)=0, distinct(66)=1, null(66)=0] + │ │ ├── key: (21,22) + │ │ ├── fd: (21,22)-->(13,14,16-18,24,29,66), (22)-->(13,14,16-18,29), (24)==(66), (66)==(24) + │ │ ├── ordering: -17,+29,+13,+21 + │ │ └── select + │ │ ├── save-table-name: q2_select_5 + │ │ ├── columns: s_name:13(char!null) s_address:14(varchar!null) s_phone:16(char!null) s_acctbal:17(float!null) s_comment:18(varchar!null) ps_partkey:21(int!null) ps_suppkey:22(int!null) ps_supplycost:24(float!null) n_name:29(char!null) min:66(float!null) + │ │ ├── stats: [rows=1, distinct(13)=1, null(13)=0, distinct(14)=1, null(14)=0, distinct(16)=1, null(16)=0, distinct(17)=1, null(17)=0, distinct(18)=1, null(18)=0, distinct(21)=1, null(21)=0, distinct(22)=0.999954, null(22)=0, distinct(24)=1, null(24)=0, distinct(29)=1, null(29)=0, distinct(66)=1, null(66)=0] + │ │ ├── key: (21,22) + │ │ ├── fd: (21,22)-->(13,14,16-18,24,29,66), (22)-->(13,14,16-18,29), (24)==(66), (66)==(24) + │ │ ├── group-by (hash) + │ │ │ ├── save-table-name: q2_group_by_6 + │ │ │ ├── columns: s_name:13(char!null) s_address:14(varchar!null) s_phone:16(char!null) s_acctbal:17(float!null) s_comment:18(varchar!null) ps_partkey:21(int!null) ps_suppkey:22(int!null) ps_supplycost:24(float!null) n_name:29(char!null) min:66(float!null) + │ │ │ ├── grouping columns: ps_partkey:21(int!null) ps_suppkey:22(int!null) + │ │ │ ├── stats: [rows=113913.7, distinct(13)=113914, null(13)=0, distinct(14)=113914, null(14)=0, distinct(16)=113914, null(16)=0, distinct(17)=113914, null(17)=0, distinct(18)=113914, null(18)=0, distinct(21)=110568, null(21)=0, distinct(22)=9920, null(22)=0, distinct(24)=113914, null(24)=0, distinct(29)=113914, null(29)=0, distinct(66)=113914, null(66)=0, distinct(21,22)=113914, null(21,22)=0] + │ │ │ ├── key: (21,22) + │ │ │ ├── fd: (21,22)-->(13,14,16-18,24,29,66), (22)-->(13,14,16-18,29) + │ │ │ ├── inner-join (hash) + │ │ │ │ ├── save-table-name: q2_inner_join_7 + │ │ │ │ ├── columns: s_suppkey:12(int!null) s_name:13(char!null) s_address:14(varchar!null) s_nationkey:15(int!null) s_phone:16(char!null) s_acctbal:17(float!null) s_comment:18(varchar!null) ps_partkey:21(int!null) ps_suppkey:22(int!null) ps_supplycost:24(float!null) n_nationkey:28(int!null) n_name:29(char!null) n_regionkey:30(int!null) r_regionkey:34(int!null) r_name:35(char!null) ps_partkey:39(int!null) ps_suppkey:40(int!null) ps_supplycost:42(float!null) s_suppkey:46(int!null) s_nationkey:49(int!null) n_nationkey:55(int!null) n_regionkey:57(int!null) r_regionkey:61(int!null) r_name:62(char!null) + │ │ │ │ ├── stats: [rows=233690.5, distinct(12)=9920, null(12)=0, distinct(13)=9990, null(13)=0, distinct(14)=10000, null(14)=0, distinct(15)=5, null(15)=0, distinct(16)=10000, null(16)=0, distinct(17)=9967, null(17)=0, distinct(18)=9934, null(18)=0, distinct(21)=110568, null(21)=0, distinct(22)=9920, null(22)=0, distinct(24)=76388.7, null(24)=0, distinct(28)=5, null(28)=0, distinct(29)=5, null(29)=0, distinct(30)=1, null(30)=0, distinct(34)=1, null(34)=0, distinct(35)=0.996222, null(35)=0, distinct(39)=110568, null(39)=0, distinct(40)=1844.81, null(40)=0, distinct(42)=75888.9, null(42)=0, distinct(46)=1844.81, null(46)=0, distinct(49)=5, null(49)=0, distinct(55)=5, null(55)=0, distinct(57)=1, null(57)=0, distinct(61)=1, null(61)=0, distinct(62)=0.996222, null(62)=0, distinct(21,22)=113914, null(21,22)=0] + │ │ │ │ ├── key: (22,39,46) + │ │ │ │ ├── fd: ()-->(35,62), (12)-->(13-18), (21,22)-->(24), (12)==(22), (22)==(12), (28)-->(29,30), (30)==(34), (34)==(30), (15)==(28), (28)==(15), (39,40)-->(42), (46)-->(49), (55)-->(57), (57)==(61), (61)==(57), (49)==(55), (55)==(49), (40)==(46), (46)==(40), (21)==(39), (39)==(21) + │ │ │ │ ├── inner-join (hash) + │ │ │ │ │ ├── save-table-name: q2_inner_join_8 + │ │ │ │ │ ├── columns: s_suppkey:12(int!null) s_name:13(char!null) s_address:14(varchar!null) s_nationkey:15(int!null) s_phone:16(char!null) s_acctbal:17(float!null) s_comment:18(varchar!null) ps_partkey:21(int!null) ps_suppkey:22(int!null) ps_supplycost:24(float!null) n_nationkey:28(int!null) n_name:29(char!null) n_regionkey:30(int!null) r_regionkey:34(int!null) r_name:35(char!null) + │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + │ │ │ │ │ ├── stats: [rows=161290.3, distinct(12)=9920, null(12)=0, distinct(13)=9990, null(13)=0, distinct(14)=10000, null(14)=0, distinct(15)=5, null(15)=0, distinct(16)=10000, null(16)=0, distinct(17)=9967, null(17)=0, distinct(18)=9934, null(18)=0, distinct(21)=111321, null(21)=0, distinct(22)=9920, null(22)=0, distinct(24)=80888.4, null(24)=0, distinct(28)=5, null(28)=0, distinct(29)=5, null(29)=0, distinct(30)=1, null(30)=0, distinct(34)=1, null(34)=0, distinct(35)=0.996222, null(35)=0, distinct(21,22)=140582, null(21,22)=0] + │ │ │ │ │ ├── key: (21,22) + │ │ │ │ │ ├── fd: ()-->(35), (12)-->(13-18), (21,22)-->(24), (12)==(22), (22)==(12), (28)-->(29,30), (30)==(34), (34)==(30), (15)==(28), (28)==(15) + │ │ │ │ │ ├── scan partsupp + │ │ │ │ │ │ ├── save-table-name: q2_scan_9 + │ │ │ │ │ │ ├── columns: ps_partkey:21(int!null) ps_suppkey:22(int!null) ps_supplycost:24(float!null) + │ │ │ │ │ │ ├── stats: [rows=800000, distinct(21)=199241, null(21)=0, distinct(22)=9920, null(22)=0, distinct(24)=100379, null(24)=0, distinct(21,22)=798302, null(21,22)=0] + │ │ │ │ │ │ │ histogram(21)= 0 79.993 3912.7 79.993 3933.7 79.993 3920.7 79.993 3917.7 79.993 3929.7 79.993 3912.7 79.993 3932.7 79.993 3918.7 158.99 3914.7 79.993 3928.7 79.993 3910.7 79.993 3904.7 79.993 3924.7 79.993 3914.7 79.993 3909.7 79.993 3917.7 79.993 3926.7 79.993 3913.7 79.993 3905.7 79.993 3912.7 79.993 3931.7 79.993 3926.7 79.993 3926.7 79.993 3906.7 79.993 3923.7 79.993 3904.7 79.993 3904.7 79.993 3907.7 158.99 3979.6 79.993 3906.7 79.993 3914.7 79.993 3918.7 79.993 3917.7 79.993 3826.7 158.99 3936.7 79.993 3908.7 79.993 3926.7 79.993 3930.7 79.993 3967.6 79.993 3910.7 79.993 3922.7 79.993 3914.7 79.993 3913.7 79.993 3915.7 79.993 3919.7 79.993 3916.7 79.993 3920.7 79.993 3926.7 79.993 3908.7 79.993 3904.7 158.99 3926.7 79.993 3922.7 79.993 3918.7 79.993 3908.7 79.993 3919.7 79.993 3908.7 79.993 3907.7 79.993 3916.7 79.993 3917.7 79.993 3905.7 79.993 3918.7 79.993 3940.7 79.993 3916.7 79.993 3923.7 79.993 3909.7 79.993 3915.7 79.993 3911.7 79.993 3915.7 79.993 3914.7 79.993 3948.6 79.993 3924.7 79.993 3916.7 79.993 3943.7 79.993 3933.7 79.993 3915.7 79.993 3916.7 79.993 3914.7 79.993 3919.7 79.993 3916.7 79.993 3912.7 79.993 3904.7 79.993 3913.7 79.993 3909.7 79.993 3914.7 79.993 3910.7 79.993 3923.7 79.993 3913.7 79.993 3914.7 79.993 3921.7 79.993 3927.7 79.993 3921.7 79.993 3924.7 158.99 3910.7 79.993 3916.7 79.993 3949.6 79.993 3922.7 79.993 3915.7 79.993 3942.7 79.993 3915.7 79.993 3917.7 79.993 3842.7 158.99 3911.7 79.993 3923.7 79.993 3923.7 79.993 3906.7 79.993 3925.7 79.993 3951.6 79.993 3933.7 79.993 3916.7 79.993 3903.7 79.993 3923.7 79.993 3932.7 79.993 3928.7 79.993 3905.7 79.993 3921.7 79.993 3920.7 79.993 3910.7 79.993 3912.7 79.993 3916.7 79.993 3922.7 79.993 3911.7 79.993 3906.7 79.993 3921.7 79.993 3911.7 79.993 3911.7 79.993 3926.7 79.993 3912.7 79.993 3945.6 79.993 3910.7 79.993 3922.7 79.993 3918.7 79.993 3911.7 79.993 3917.7 79.993 3945.6 79.993 3926.7 79.993 3926.7 79.993 3917.7 79.993 3904.7 79.993 3925.7 79.993 3912.7 79.993 3912.7 79.993 3954.6 79.993 3915.7 79.993 3912.7 79.993 3910.7 79.993 3909.7 79.993 3911.7 79.993 3903.7 79.993 3915.7 79.993 3949.6 79.993 3923.7 79.993 3921.7 79.993 3909.7 79.993 3905.7 79.993 3988.6 79.993 3988.6 79.993 3999.6 79.993 4003.6 79.993 3998.6 79.993 4021.6 79.993 4027.6 79.993 4005.6 79.993 3999.6 79.993 3997.6 79.993 3988.6 79.993 3989.6 79.993 4004.6 79.993 3984.6 79.993 3999.6 79.993 3999.6 79.993 4019.6 79.993 4011.6 79.993 4020.6 79.993 4012.6 79.993 3996.6 79.993 4029.6 79.993 4004.6 158.99 3912.7 79.993 3995.6 79.993 3989.6 79.993 3991.6 79.993 3986.6 79.993 3986.6 79.993 4006.6 79.993 3988.6 79.993 3989.6 79.993 3989.6 79.993 3998.6 79.993 4012.6 79.993 4017.6 79.993 4017.6 79.993 3996.6 79.993 3994.6 79.993 4009.6 79.993 3995.6 79.993 3996.6 79.993 3991.6 79.993 4006.6 79.993 4020.6 79.993 + │ │ │ │ │ │ │ <---- 13 --------- 942 --------- 2097 -------- 3127 -------- 4125 -------- 5247 -------- 6181 -------- 7326 -------- 8333 -------- 9292 -------- 10410 -------- 11308 -------- 12057 -------- 13131 -------- 14088 -------- 14972 -------- 15975 -------- 17072 -------- 18019 -------- 18798 -------- 19734 -------- 20877 -------- 21973 -------- 23067 -------- 23887 -------- 24957 -------- 25716 -------- 26450 -------- 27291 -------- 28733 -------- 29539 -------- 30499 -------- 31512 -------- 32509 -------- 33286 -------- 34464 -------- 35311 -------- 36406 -------- 37541 -------- 38918 -------- 39818 -------- 40879 -------- 41843 -------- 42789 -------- 43757 -------- 44778 -------- 45769 -------- 46806 -------- 47899 -------- 48763 -------- 49507 -------- 50607 -------- 51663 -------- 52669 -------- 53525 -------- 54549 -------- 55415 -------- 56261 -------- 57242 -------- 58242 -------- 59036 -------- 60050 -------- 61259 -------- 62240 -------- 63307 -------- 64178 -------- 65152 -------- 66063 -------- 67040 -------- 68005 -------- 69273 -------- 70354 -------- 71339 -------- 72569 -------- 73724 -------- 74695 -------- 75684 -------- 76646 -------- 77670 -------- 78657 -------- 79587 -------- 80331 -------- 81281 -------- 82150 -------- 83115 -------- 84014 -------- 85082 -------- 86031 -------- 86990 -------- 88034 -------- 89138 -------- 90187 -------- 91260 -------- 92150 -------- 93140 -------- 94413 -------- 95469 -------- 96443 -------- 97666 -------- 98637 -------- 99633 -------- 100664 -------- 101572 -------- 102643 -------- 103706 -------- 104522 -------- 105605 -------- 106892 -------- 108047 -------- 109036 -------- 109721 -------- 110790 -------- 111938 -------- 113052 -------- 113830 -------- 114873 -------- 115912 -------- 116814 -------- 117737 -------- 118721 -------- 119776 -------- 120692 -------- 121500 -------- 122545 -------- 123457 -------- 124366 -------- 125466 -------- 126391 -------- 127638 -------- 128533 -------- 129586 -------- 130602 -------- 131508 -------- 132509 -------- 133756 -------- 134848 -------- 135944 -------- 136945 -------- 137706 -------- 138791 -------- 139720 -------- 140657 -------- 141959 -------- 142929 -------- 143854 -------- 144743 -------- 145629 -------- 146548 -------- 147238 -------- 148209 -------- 149481 -------- 150548 -------- 151598 -------- 152481 -------- 153250 -------- 154137 -------- 155017 -------- 156060 -------- 157143 -------- 158169 -------- 159406 -------- 160686 -------- 161794 -------- 162837 -------- 163860 -------- 164730 -------- 165623 -------- 166716 -------- 167485 -------- 168526 -------- 169568 -------- 170793 -------- 171958 -------- 173192 -------- 174365 -------- 175367 -------- 176660 -------- 177754 -------- 178681 -------- 179672 -------- 180568 -------- 181502 -------- 182344 -------- 183171 -------- 184286 -------- 185174 -------- 186068 -------- 186966 -------- 187997 -------- 189168 -------- 190375 -------- 191583 -------- 192588 -------- 193575 -------- 194722 -------- 195713 -------- 196725 -------- 197653 -------- 198767 -------- 199999 + │ │ │ │ │ │ │ histogram(22)= 0 160 3920 160 3920 80 3920 160 3920 160 3920 240 3760 240 3920 80 3840 240 3920 240 3840 320 3760 240 3920 80 3840 160 3920 240 3920 320 3920 80 3920 80 3920 80 3840 160 3920 240 3760 240 3920 80 3840 160 3920 80 3920 160 3920 80 3920 160 3920 80 3920 160 3920 80 3760 240 3840 240 3920 80 3920 80 3840 240 3760 240 3920 80 3840 160 3840 160 3920 80 3920 80 3920 160 3760 240 3920 240 3920 80 3920 160 3920 80 3840 160 3920 160 3920 80 3840 160 3840 240 3920 160 3840 160 3920 160 3920 80 3840 160 3920 160 3840 160 3840 160 3920 80 3920 160 3920 160 3920 80 3920 80 3840 160 3840 160 3840 160 3920 80 3920 80 3840 240 3840 160 3920 320 3840 160 3840 240 3920 80 3920 80 3760 240 3840 160 3920 160 3920 80 3840 240 3920 80 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3840 160 3920 80 3920 160 3760 320 3920 80 3920 80 3840 160 3920 240 3920 80 3920 80 3920 80 3920 160 3840 160 3760 400 3760 240 3680 320 3840 240 3840 80 3840 160 3840 160 3920 80 3920 80 3920 80 3840 160 3920 80 3760 240 3920 80 3840 240 3840 80 3840 160 3920 240 3840 80 3840 80 3840 160 3920 80 3760 240 3920 80 3920 160 3840 160 3760 240 3760 240 3840 80 3920 160 3840 80 3920 80 3920 80 3840 400 3760 160 3840 80 3840 160 3760 160 3840 240 3840 160 3680 320 3760 160 3920 80 3920 80 3920 80 3920 80 3920 80 3840 160 3760 240 3840 160 3920 80 3840 160 3920 240 3840 160 3840 80 3840 160 3840 80 3920 80 3920 80 3920 160 3840 160 3840 160 3840 160 3760 160 3920 80 3920 80 3920 80 3920 80 3760 240 3920 80 3920 320 3760 160 3840 80 3840 80 3920 160 3840 80 3920 160 3760 160 3920 80 3920 80 3920 160 3840 160 3840 80 3840 160 3920 80 3920 80 3920 80 3840 160 3840 240 3840 160 3840 80 3920 80 3840 240 3840 80 3920 80 3920 80 3840 160 + │ │ │ │ │ │ │ <--- 2 ------ 50 ------ 104 ------ 153 ------ 213 ------ 281 ------ 320 ------ 366 ------ 411 ------ 462 ------ 515 ------ 548 ------ 600 ------ 649 ------ 697 ------ 743 ------ 793 ------ 845 ------ 893 ------ 953 ------ 1006 ------ 1052 ------ 1103 ------ 1158 ------ 1199 ------ 1246 ------ 1302 ------ 1375 ------ 1418 ------ 1475 ------ 1524 ------ 1563 ------ 1628 ------ 1689 ------ 1740 ------ 1799 ------ 1850 ------ 1901 ------ 1948 ------ 2017 ------ 2055 ------ 2099 ------ 2157 ------ 2214 ------ 2267 ------ 2319 ------ 2373 ------ 2428 ------ 2478 ------ 2546 ------ 2602 ------ 2657 ------ 2707 ------ 2760 ------ 2808 ------ 2852 ------ 2913 ------ 2968 ------ 3030 ------ 3069 ------ 3115 ------ 3165 ------ 3210 ------ 3256 ------ 3306 ------ 3365 ------ 3419 ------ 3469 ------ 3523 ------ 3576 ------ 3641 ------ 3694 ------ 3738 ------ 3806 ------ 3851 ------ 3900 ------ 3957 ------ 4004 ------ 4050 ------ 4095 ------ 4145 ------ 4201 ------ 4251 ------ 4293 ------ 4335 ------ 4380 ------ 4432 ------ 4484 ------ 4541 ------ 4593 ------ 4650 ------ 4706 ------ 4744 ------ 4804 ------ 4845 ------ 4897 ------ 4945 ------ 4992 ------ 5044 ------ 5108 ------ 5160 ------ 5207 ------ 5261 ------ 5319 ------ 5358 ------ 5404 ------ 5450 ------ 5490 ------ 5538 ------ 5590 ------ 5639 ------ 5686 ------ 5742 ------ 5788 ------ 5837 ------ 5884 ------ 5940 ------ 5985 ------ 6037 ------ 6090 ------ 6135 ------ 6185 ------ 6228 ------ 6271 ------ 6323 ------ 6376 ------ 6434 ------ 6474 ------ 6527 ------ 6586 ------ 6633 ------ 6674 ------ 6711 ------ 6751 ------ 6797 ------ 6835 ------ 6880 ------ 6918 ------ 6982 ------ 7026 ------ 7069 ------ 7123 ------ 7179 ------ 7238 ------ 7287 ------ 7336 ------ 7388 ------ 7438 ------ 7480 ------ 7528 ------ 7574 ------ 7620 ------ 7664 ------ 7706 ------ 7755 ------ 7805 ------ 7847 ------ 7896 ------ 7954 ------ 8014 ------ 8064 ------ 8108 ------ 8159 ------ 8207 ------ 8250 ------ 8304 ------ 8361 ------ 8410 ------ 8462 ------ 8513 ------ 8562 ------ 8608 ------ 8644 ------ 8706 ------ 8752 ------ 8799 ------ 8840 ------ 8902 ------ 8954 ------ 8995 ------ 9063 ------ 9106 ------ 9152 ------ 9202 ------ 9256 ------ 9310 ------ 9362 ------ 9409 ------ 9462 ------ 9504 ------ 9551 ------ 9598 ------ 9644 ------ 9689 ------ 9741 ------ 9800 ------ 9855 ------ 9896 ------ 9945 ------ 10000 + │ │ │ │ │ │ │ histogram(24)= 0 80 7.9984e+05 80 + │ │ │ │ │ │ │ <--- 1.14 ------------ 999.93 + │ │ │ │ │ │ ├── key: (21,22) + │ │ │ │ │ │ └── fd: (21,22)-->(24) + │ │ │ │ │ ├── inner-join (hash) + │ │ │ │ │ │ ├── save-table-name: q2_inner_join_10 + │ │ │ │ │ │ ├── columns: s_suppkey:12(int!null) s_name:13(char!null) s_address:14(varchar!null) s_nationkey:15(int!null) s_phone:16(char!null) s_acctbal:17(float!null) s_comment:18(varchar!null) n_nationkey:28(int!null) n_name:29(char!null) n_regionkey:30(int!null) r_regionkey:34(int!null) r_name:35(char!null) + │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + │ │ │ │ │ │ ├── stats: [rows=2000, distinct(12)=1844.81, null(12)=0, distinct(13)=1846.09, null(13)=0, distinct(14)=1846.27, null(14)=0, distinct(15)=5, null(15)=0, distinct(16)=1846.27, null(16)=0, distinct(17)=1845.67, null(17)=0, distinct(18)=1845.06, null(18)=0, distinct(28)=5, null(28)=0, distinct(29)=5, null(29)=0, distinct(30)=1, null(30)=0, distinct(34)=1, null(34)=0, distinct(35)=0.996222, null(35)=0] + │ │ │ │ │ │ ├── key: (12) + │ │ │ │ │ │ ├── fd: ()-->(35), (12)-->(13-18), (28)-->(29,30), (30)==(34), (34)==(30), (15)==(28), (28)==(15) + │ │ │ │ │ │ ├── scan supplier + │ │ │ │ │ │ │ ├── save-table-name: q2_scan_11 + │ │ │ │ │ │ │ ├── columns: s_suppkey:12(int!null) s_name:13(char!null) s_address:14(varchar!null) s_nationkey:15(int!null) s_phone:16(char!null) s_acctbal:17(float!null) s_comment:18(varchar!null) + │ │ │ │ │ │ │ ├── stats: [rows=10000, distinct(12)=9920, null(12)=0, distinct(13)=9990, null(13)=0, distinct(14)=10000, null(14)=0, distinct(15)=25, null(15)=0, distinct(16)=10000, null(16)=0, distinct(17)=9967, null(17)=0, distinct(18)=9934, null(18)=0] + │ │ │ │ │ │ │ │ histogram(12)= 0 0 0 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 0 0 + │ │ │ │ │ │ │ │ <--- -9223372036854775808 --- 1 ---- 51 ---- 101 ---- 151 ---- 201 ---- 251 ---- 301 ---- 351 ---- 401 ---- 451 ---- 501 ---- 551 ---- 601 ---- 651 ---- 701 ---- 751 ---- 801 ---- 851 ---- 901 ---- 951 ---- 1001 ---- 1051 ---- 1101 ---- 1151 ---- 1201 ---- 1251 ---- 1301 ---- 1351 ---- 1401 ---- 1451 ---- 1501 ---- 1551 ---- 1601 ---- 1651 ---- 1701 ---- 1751 ---- 1801 ---- 1851 ---- 1901 ---- 1951 ---- 2001 ---- 2051 ---- 2101 ---- 2151 ---- 2201 ---- 2251 ---- 2301 ---- 2351 ---- 2401 ---- 2451 ---- 2501 ---- 2551 ---- 2601 ---- 2651 ---- 2701 ---- 2751 ---- 2801 ---- 2851 ---- 2901 ---- 2951 ---- 3001 ---- 3051 ---- 3101 ---- 3151 ---- 3201 ---- 3251 ---- 3301 ---- 3351 ---- 3401 ---- 3451 ---- 3501 ---- 3551 ---- 3601 ---- 3651 ---- 3701 ---- 3751 ---- 3801 ---- 3851 ---- 3901 ---- 3951 ---- 4001 ---- 4051 ---- 4101 ---- 4151 ---- 4201 ---- 4251 ---- 4301 ---- 4351 ---- 4401 ---- 4451 ---- 4501 ---- 4551 ---- 4601 ---- 4651 ---- 4701 ---- 4751 ---- 4801 ---- 4851 ---- 4901 ---- 4951 ---- 5001 ---- 5051 ---- 5101 ---- 5151 ---- 5201 ---- 5251 ---- 5301 ---- 5351 ---- 5401 ---- 5451 ---- 5501 ---- 5551 ---- 5601 ---- 5651 ---- 5701 ---- 5751 ---- 5801 ---- 5851 ---- 5901 ---- 5951 ---- 6001 ---- 6051 ---- 6101 ---- 6151 ---- 6201 ---- 6251 ---- 6301 ---- 6351 ---- 6401 ---- 6451 ---- 6501 ---- 6551 ---- 6601 ---- 6651 ---- 6701 ---- 6751 ---- 6801 ---- 6851 ---- 6901 ---- 6951 ---- 7001 ---- 7051 ---- 7101 ---- 7151 ---- 7201 ---- 7251 ---- 7301 ---- 7351 ---- 7401 ---- 7451 ---- 7501 ---- 7552 ---- 7603 ---- 7654 ---- 7705 ---- 7756 ---- 7807 ---- 7858 ---- 7909 ---- 7960 ---- 8011 ---- 8062 ---- 8113 ---- 8164 ---- 8215 ---- 8266 ---- 8317 ---- 8368 ---- 8419 ---- 8470 ---- 8521 ---- 8572 ---- 8623 ---- 8674 ---- 8725 ---- 8776 ---- 8827 ---- 8878 ---- 8929 ---- 8980 ---- 9031 ---- 9082 ---- 9133 ---- 9184 ---- 9235 ---- 9286 ---- 9337 ---- 9388 ---- 9439 ---- 9490 ---- 9541 ---- 9592 ---- 9643 ---- 9694 ---- 9745 ---- 9796 ---- 9847 ---- 9898 ---- 9949 ---- 10000 --- 9223372036854775807 + │ │ │ │ │ │ │ │ histogram(13)= 0 1 9998 1 + │ │ │ │ │ │ │ │ <--- 'Supplier#000000001' ------ 'Supplier#000010000' + │ │ │ │ │ │ │ │ histogram(14)= 0 1 9998 1 + │ │ │ │ │ │ │ │ <--- ' 9aW1wwnBJJPnCx,nox0MA48Y0zpI1IeVfYZ' ------ 'zzfDhdtZcvmVzA8rNFU,Yctj1zBN' + │ │ │ │ │ │ │ │ histogram(15)= 0 420 0 413 0 397 0 412 0 415 0 380 0 402 0 396 0 415 0 405 0 393 0 438 0 377 0 362 0 376 0 373 0 406 0 421 0 407 0 398 0 411 0 399 0 401 0 390 0 393 + │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 + │ │ │ │ │ │ │ │ histogram(16)= 0 1 9998 1 + │ │ │ │ │ │ │ │ <--- '10-102-116-6785' ------ '34-998-900-4911' + │ │ │ │ │ │ │ │ histogram(17)= 0 1 9998 1 + │ │ │ │ │ │ │ │ <--- -998.22 ------ 9999.72 + │ │ │ │ │ │ │ │ histogram(18)= 0 1 9998 1 + │ │ │ │ │ │ │ │ <--- ' about the blithely express foxes. bli' ------ 'zzle furiously. bold accounts haggle furiously ironic excuses. fur' + │ │ │ │ │ │ │ ├── key: (12) + │ │ │ │ │ │ │ └── fd: (12)-->(13-18) + │ │ │ │ │ │ ├── inner-join (hash) + │ │ │ │ │ │ │ ├── save-table-name: q2_inner_join_12 + │ │ │ │ │ │ │ ├── columns: n_nationkey:28(int!null) n_name:29(char!null) n_regionkey:30(int!null) r_regionkey:34(int!null) r_name:35(char!null) + │ │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + │ │ │ │ │ │ │ ├── stats: [rows=5, distinct(28)=5, null(28)=0, distinct(29)=5, null(29)=0, distinct(30)=1, null(30)=0, distinct(34)=1, null(34)=0, distinct(35)=0.996222, null(35)=0] + │ │ │ │ │ │ │ ├── key: (28) + │ │ │ │ │ │ │ ├── fd: ()-->(35), (28)-->(29,30), (30)==(34), (34)==(30) + │ │ │ │ │ │ │ ├── scan nation + │ │ │ │ │ │ │ │ ├── save-table-name: q2_scan_13 + │ │ │ │ │ │ │ │ ├── columns: n_nationkey:28(int!null) n_name:29(char!null) n_regionkey:30(int!null) + │ │ │ │ │ │ │ │ ├── stats: [rows=25, distinct(28)=25, null(28)=0, distinct(29)=25, null(29)=0, distinct(30)=5, null(30)=0] + │ │ │ │ │ │ │ │ │ histogram(28)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 + │ │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 + │ │ │ │ │ │ │ │ │ histogram(29)= 0 1 23 1 + │ │ │ │ │ │ │ │ │ <--- 'ALGERIA' ---- 'VIETNAM' + │ │ │ │ │ │ │ │ │ histogram(30)= 0 5 0 5 0 5 0 5 0 5 + │ │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 + │ │ │ │ │ │ │ │ ├── key: (28) + │ │ │ │ │ │ │ │ └── fd: (28)-->(29,30) + │ │ │ │ │ │ │ ├── select + │ │ │ │ │ │ │ │ ├── save-table-name: q2_select_14 + │ │ │ │ │ │ │ │ ├── columns: r_regionkey:34(int!null) r_name:35(char!null) + │ │ │ │ │ │ │ │ ├── stats: [rows=1, distinct(34)=1, null(34)=0, distinct(35)=1, null(35)=0] + │ │ │ │ │ │ │ │ │ histogram(35)= 0 1 + │ │ │ │ │ │ │ │ │ <--- 'EUROPE' + │ │ │ │ │ │ │ │ ├── key: (34) + │ │ │ │ │ │ │ │ ├── fd: ()-->(35) + │ │ │ │ │ │ │ │ ├── scan region + │ │ │ │ │ │ │ │ │ ├── save-table-name: q2_scan_15 + │ │ │ │ │ │ │ │ │ ├── columns: r_regionkey:34(int!null) r_name:35(char!null) + │ │ │ │ │ │ │ │ │ ├── stats: [rows=5, distinct(34)=5, null(34)=0, distinct(35)=5, null(35)=0] + │ │ │ │ │ │ │ │ │ │ histogram(34)= 0 1 0 1 0 1 0 1 0 1 + │ │ │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 + │ │ │ │ │ │ │ │ │ │ histogram(35)= 0 1 3 1 + │ │ │ │ │ │ │ │ │ │ <--- 'AFRICA' --- 'MIDDLE EAST' + │ │ │ │ │ │ │ │ │ ├── key: (34) + │ │ │ │ │ │ │ │ │ └── fd: (34)-->(35) + │ │ │ │ │ │ │ │ └── filters + │ │ │ │ │ │ │ │ └── r_name:35 = 'EUROPE' [type=bool, outer=(35), constraints=(/35: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(35)] + │ │ │ │ │ │ │ └── filters + │ │ │ │ │ │ │ └── n_regionkey:30 = r_regionkey:34 [type=bool, outer=(30,34), constraints=(/30: (/NULL - ]; /34: (/NULL - ]), fd=(30)==(34), (34)==(30)] + │ │ │ │ │ │ └── filters + │ │ │ │ │ │ └── s_nationkey:15 = n_nationkey:28 [type=bool, outer=(15,28), constraints=(/15: (/NULL - ]; /28: (/NULL - ]), fd=(15)==(28), (28)==(15)] + │ │ │ │ │ └── filters + │ │ │ │ │ └── s_suppkey:12 = ps_suppkey:22 [type=bool, outer=(12,22), constraints=(/12: (/NULL - ]; /22: (/NULL - ]), fd=(12)==(22), (22)==(12)] + │ │ │ │ ├── inner-join (hash) + │ │ │ │ │ ├── save-table-name: q2_inner_join_16 + │ │ │ │ │ ├── columns: ps_partkey:39(int!null) ps_suppkey:40(int!null) ps_supplycost:42(float!null) s_suppkey:46(int!null) s_nationkey:49(int!null) n_nationkey:55(int!null) n_regionkey:57(int!null) r_regionkey:61(int!null) r_name:62(char!null) + │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + │ │ │ │ │ ├── stats: [rows=161290.3, distinct(39)=110568, null(39)=0, distinct(40)=1844.81, null(40)=0, distinct(42)=80252.1, null(42)=0, distinct(46)=1844.81, null(46)=0, distinct(49)=5, null(49)=0, distinct(55)=5, null(55)=0, distinct(57)=1, null(57)=0, distinct(61)=1, null(61)=0, distinct(62)=0.996222, null(62)=0] + │ │ │ │ │ ├── key: (39,46) + │ │ │ │ │ ├── fd: ()-->(62), (39,40)-->(42), (46)-->(49), (55)-->(57), (57)==(61), (61)==(57), (49)==(55), (55)==(49), (40)==(46), (46)==(40) + │ │ │ │ │ ├── scan partsupp + │ │ │ │ │ │ ├── save-table-name: q2_scan_17 + │ │ │ │ │ │ ├── columns: ps_partkey:39(int!null) ps_suppkey:40(int!null) ps_supplycost:42(float!null) + │ │ │ │ │ │ ├── stats: [rows=800000, distinct(39)=199241, null(39)=0, distinct(40)=9920, null(40)=0, distinct(42)=100379, null(42)=0] + │ │ │ │ │ │ │ histogram(39)= 0 79.993 3912.7 79.993 3933.7 79.993 3920.7 79.993 3917.7 79.993 3929.7 79.993 3912.7 79.993 3932.7 79.993 3918.7 158.99 3914.7 79.993 3928.7 79.993 3910.7 79.993 3904.7 79.993 3924.7 79.993 3914.7 79.993 3909.7 79.993 3917.7 79.993 3926.7 79.993 3913.7 79.993 3905.7 79.993 3912.7 79.993 3931.7 79.993 3926.7 79.993 3926.7 79.993 3906.7 79.993 3923.7 79.993 3904.7 79.993 3904.7 79.993 3907.7 158.99 3979.6 79.993 3906.7 79.993 3914.7 79.993 3918.7 79.993 3917.7 79.993 3826.7 158.99 3936.7 79.993 3908.7 79.993 3926.7 79.993 3930.7 79.993 3967.6 79.993 3910.7 79.993 3922.7 79.993 3914.7 79.993 3913.7 79.993 3915.7 79.993 3919.7 79.993 3916.7 79.993 3920.7 79.993 3926.7 79.993 3908.7 79.993 3904.7 158.99 3926.7 79.993 3922.7 79.993 3918.7 79.993 3908.7 79.993 3919.7 79.993 3908.7 79.993 3907.7 79.993 3916.7 79.993 3917.7 79.993 3905.7 79.993 3918.7 79.993 3940.7 79.993 3916.7 79.993 3923.7 79.993 3909.7 79.993 3915.7 79.993 3911.7 79.993 3915.7 79.993 3914.7 79.993 3948.6 79.993 3924.7 79.993 3916.7 79.993 3943.7 79.993 3933.7 79.993 3915.7 79.993 3916.7 79.993 3914.7 79.993 3919.7 79.993 3916.7 79.993 3912.7 79.993 3904.7 79.993 3913.7 79.993 3909.7 79.993 3914.7 79.993 3910.7 79.993 3923.7 79.993 3913.7 79.993 3914.7 79.993 3921.7 79.993 3927.7 79.993 3921.7 79.993 3924.7 158.99 3910.7 79.993 3916.7 79.993 3949.6 79.993 3922.7 79.993 3915.7 79.993 3942.7 79.993 3915.7 79.993 3917.7 79.993 3842.7 158.99 3911.7 79.993 3923.7 79.993 3923.7 79.993 3906.7 79.993 3925.7 79.993 3951.6 79.993 3933.7 79.993 3916.7 79.993 3903.7 79.993 3923.7 79.993 3932.7 79.993 3928.7 79.993 3905.7 79.993 3921.7 79.993 3920.7 79.993 3910.7 79.993 3912.7 79.993 3916.7 79.993 3922.7 79.993 3911.7 79.993 3906.7 79.993 3921.7 79.993 3911.7 79.993 3911.7 79.993 3926.7 79.993 3912.7 79.993 3945.6 79.993 3910.7 79.993 3922.7 79.993 3918.7 79.993 3911.7 79.993 3917.7 79.993 3945.6 79.993 3926.7 79.993 3926.7 79.993 3917.7 79.993 3904.7 79.993 3925.7 79.993 3912.7 79.993 3912.7 79.993 3954.6 79.993 3915.7 79.993 3912.7 79.993 3910.7 79.993 3909.7 79.993 3911.7 79.993 3903.7 79.993 3915.7 79.993 3949.6 79.993 3923.7 79.993 3921.7 79.993 3909.7 79.993 3905.7 79.993 3988.6 79.993 3988.6 79.993 3999.6 79.993 4003.6 79.993 3998.6 79.993 4021.6 79.993 4027.6 79.993 4005.6 79.993 3999.6 79.993 3997.6 79.993 3988.6 79.993 3989.6 79.993 4004.6 79.993 3984.6 79.993 3999.6 79.993 3999.6 79.993 4019.6 79.993 4011.6 79.993 4020.6 79.993 4012.6 79.993 3996.6 79.993 4029.6 79.993 4004.6 158.99 3912.7 79.993 3995.6 79.993 3989.6 79.993 3991.6 79.993 3986.6 79.993 3986.6 79.993 4006.6 79.993 3988.6 79.993 3989.6 79.993 3989.6 79.993 3998.6 79.993 4012.6 79.993 4017.6 79.993 4017.6 79.993 3996.6 79.993 3994.6 79.993 4009.6 79.993 3995.6 79.993 3996.6 79.993 3991.6 79.993 4006.6 79.993 4020.6 79.993 + │ │ │ │ │ │ │ <---- 13 --------- 942 --------- 2097 -------- 3127 -------- 4125 -------- 5247 -------- 6181 -------- 7326 -------- 8333 -------- 9292 -------- 10410 -------- 11308 -------- 12057 -------- 13131 -------- 14088 -------- 14972 -------- 15975 -------- 17072 -------- 18019 -------- 18798 -------- 19734 -------- 20877 -------- 21973 -------- 23067 -------- 23887 -------- 24957 -------- 25716 -------- 26450 -------- 27291 -------- 28733 -------- 29539 -------- 30499 -------- 31512 -------- 32509 -------- 33286 -------- 34464 -------- 35311 -------- 36406 -------- 37541 -------- 38918 -------- 39818 -------- 40879 -------- 41843 -------- 42789 -------- 43757 -------- 44778 -------- 45769 -------- 46806 -------- 47899 -------- 48763 -------- 49507 -------- 50607 -------- 51663 -------- 52669 -------- 53525 -------- 54549 -------- 55415 -------- 56261 -------- 57242 -------- 58242 -------- 59036 -------- 60050 -------- 61259 -------- 62240 -------- 63307 -------- 64178 -------- 65152 -------- 66063 -------- 67040 -------- 68005 -------- 69273 -------- 70354 -------- 71339 -------- 72569 -------- 73724 -------- 74695 -------- 75684 -------- 76646 -------- 77670 -------- 78657 -------- 79587 -------- 80331 -------- 81281 -------- 82150 -------- 83115 -------- 84014 -------- 85082 -------- 86031 -------- 86990 -------- 88034 -------- 89138 -------- 90187 -------- 91260 -------- 92150 -------- 93140 -------- 94413 -------- 95469 -------- 96443 -------- 97666 -------- 98637 -------- 99633 -------- 100664 -------- 101572 -------- 102643 -------- 103706 -------- 104522 -------- 105605 -------- 106892 -------- 108047 -------- 109036 -------- 109721 -------- 110790 -------- 111938 -------- 113052 -------- 113830 -------- 114873 -------- 115912 -------- 116814 -------- 117737 -------- 118721 -------- 119776 -------- 120692 -------- 121500 -------- 122545 -------- 123457 -------- 124366 -------- 125466 -------- 126391 -------- 127638 -------- 128533 -------- 129586 -------- 130602 -------- 131508 -------- 132509 -------- 133756 -------- 134848 -------- 135944 -------- 136945 -------- 137706 -------- 138791 -------- 139720 -------- 140657 -------- 141959 -------- 142929 -------- 143854 -------- 144743 -------- 145629 -------- 146548 -------- 147238 -------- 148209 -------- 149481 -------- 150548 -------- 151598 -------- 152481 -------- 153250 -------- 154137 -------- 155017 -------- 156060 -------- 157143 -------- 158169 -------- 159406 -------- 160686 -------- 161794 -------- 162837 -------- 163860 -------- 164730 -------- 165623 -------- 166716 -------- 167485 -------- 168526 -------- 169568 -------- 170793 -------- 171958 -------- 173192 -------- 174365 -------- 175367 -------- 176660 -------- 177754 -------- 178681 -------- 179672 -------- 180568 -------- 181502 -------- 182344 -------- 183171 -------- 184286 -------- 185174 -------- 186068 -------- 186966 -------- 187997 -------- 189168 -------- 190375 -------- 191583 -------- 192588 -------- 193575 -------- 194722 -------- 195713 -------- 196725 -------- 197653 -------- 198767 -------- 199999 + │ │ │ │ │ │ │ histogram(40)= 0 160 3920 160 3920 80 3920 160 3920 160 3920 240 3760 240 3920 80 3840 240 3920 240 3840 320 3760 240 3920 80 3840 160 3920 240 3920 320 3920 80 3920 80 3920 80 3840 160 3920 240 3760 240 3920 80 3840 160 3920 80 3920 160 3920 80 3920 160 3920 80 3920 160 3920 80 3760 240 3840 240 3920 80 3920 80 3840 240 3760 240 3920 80 3840 160 3840 160 3920 80 3920 80 3920 160 3760 240 3920 240 3920 80 3920 160 3920 80 3840 160 3920 160 3920 80 3840 160 3840 240 3920 160 3840 160 3920 160 3920 80 3840 160 3920 160 3840 160 3840 160 3920 80 3920 160 3920 160 3920 80 3920 80 3840 160 3840 160 3840 160 3920 80 3920 80 3840 240 3840 160 3920 320 3840 160 3840 240 3920 80 3920 80 3760 240 3840 160 3920 160 3920 80 3840 240 3920 80 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3840 160 3920 80 3920 160 3760 320 3920 80 3920 80 3840 160 3920 240 3920 80 3920 80 3920 80 3920 160 3840 160 3760 400 3760 240 3680 320 3840 240 3840 80 3840 160 3840 160 3920 80 3920 80 3920 80 3840 160 3920 80 3760 240 3920 80 3840 240 3840 80 3840 160 3920 240 3840 80 3840 80 3840 160 3920 80 3760 240 3920 80 3920 160 3840 160 3760 240 3760 240 3840 80 3920 160 3840 80 3920 80 3920 80 3840 400 3760 160 3840 80 3840 160 3760 160 3840 240 3840 160 3680 320 3760 160 3920 80 3920 80 3920 80 3920 80 3920 80 3840 160 3760 240 3840 160 3920 80 3840 160 3920 240 3840 160 3840 80 3840 160 3840 80 3920 80 3920 80 3920 160 3840 160 3840 160 3840 160 3760 160 3920 80 3920 80 3920 80 3920 80 3760 240 3920 80 3920 320 3760 160 3840 80 3840 80 3920 160 3840 80 3920 160 3760 160 3920 80 3920 80 3920 160 3840 160 3840 80 3840 160 3920 80 3920 80 3920 80 3840 160 3840 240 3840 160 3840 80 3920 80 3840 240 3840 80 3920 80 3920 80 3840 160 + │ │ │ │ │ │ │ <--- 2 ------ 50 ------ 104 ------ 153 ------ 213 ------ 281 ------ 320 ------ 366 ------ 411 ------ 462 ------ 515 ------ 548 ------ 600 ------ 649 ------ 697 ------ 743 ------ 793 ------ 845 ------ 893 ------ 953 ------ 1006 ------ 1052 ------ 1103 ------ 1158 ------ 1199 ------ 1246 ------ 1302 ------ 1375 ------ 1418 ------ 1475 ------ 1524 ------ 1563 ------ 1628 ------ 1689 ------ 1740 ------ 1799 ------ 1850 ------ 1901 ------ 1948 ------ 2017 ------ 2055 ------ 2099 ------ 2157 ------ 2214 ------ 2267 ------ 2319 ------ 2373 ------ 2428 ------ 2478 ------ 2546 ------ 2602 ------ 2657 ------ 2707 ------ 2760 ------ 2808 ------ 2852 ------ 2913 ------ 2968 ------ 3030 ------ 3069 ------ 3115 ------ 3165 ------ 3210 ------ 3256 ------ 3306 ------ 3365 ------ 3419 ------ 3469 ------ 3523 ------ 3576 ------ 3641 ------ 3694 ------ 3738 ------ 3806 ------ 3851 ------ 3900 ------ 3957 ------ 4004 ------ 4050 ------ 4095 ------ 4145 ------ 4201 ------ 4251 ------ 4293 ------ 4335 ------ 4380 ------ 4432 ------ 4484 ------ 4541 ------ 4593 ------ 4650 ------ 4706 ------ 4744 ------ 4804 ------ 4845 ------ 4897 ------ 4945 ------ 4992 ------ 5044 ------ 5108 ------ 5160 ------ 5207 ------ 5261 ------ 5319 ------ 5358 ------ 5404 ------ 5450 ------ 5490 ------ 5538 ------ 5590 ------ 5639 ------ 5686 ------ 5742 ------ 5788 ------ 5837 ------ 5884 ------ 5940 ------ 5985 ------ 6037 ------ 6090 ------ 6135 ------ 6185 ------ 6228 ------ 6271 ------ 6323 ------ 6376 ------ 6434 ------ 6474 ------ 6527 ------ 6586 ------ 6633 ------ 6674 ------ 6711 ------ 6751 ------ 6797 ------ 6835 ------ 6880 ------ 6918 ------ 6982 ------ 7026 ------ 7069 ------ 7123 ------ 7179 ------ 7238 ------ 7287 ------ 7336 ------ 7388 ------ 7438 ------ 7480 ------ 7528 ------ 7574 ------ 7620 ------ 7664 ------ 7706 ------ 7755 ------ 7805 ------ 7847 ------ 7896 ------ 7954 ------ 8014 ------ 8064 ------ 8108 ------ 8159 ------ 8207 ------ 8250 ------ 8304 ------ 8361 ------ 8410 ------ 8462 ------ 8513 ------ 8562 ------ 8608 ------ 8644 ------ 8706 ------ 8752 ------ 8799 ------ 8840 ------ 8902 ------ 8954 ------ 8995 ------ 9063 ------ 9106 ------ 9152 ------ 9202 ------ 9256 ------ 9310 ------ 9362 ------ 9409 ------ 9462 ------ 9504 ------ 9551 ------ 9598 ------ 9644 ------ 9689 ------ 9741 ------ 9800 ------ 9855 ------ 9896 ------ 9945 ------ 10000 + │ │ │ │ │ │ │ histogram(42)= 0 80 7.9984e+05 80 + │ │ │ │ │ │ │ <--- 1.14 ------------ 999.93 + │ │ │ │ │ │ ├── key: (39,40) + │ │ │ │ │ │ └── fd: (39,40)-->(42) + │ │ │ │ │ ├── inner-join (lookup supplier@s_nk) + │ │ │ │ │ │ ├── save-table-name: q2_lookup_join_18 + │ │ │ │ │ │ ├── columns: s_suppkey:46(int!null) s_nationkey:49(int!null) n_nationkey:55(int!null) n_regionkey:57(int!null) r_regionkey:61(int!null) r_name:62(char!null) + │ │ │ │ │ │ ├── key columns: [55] = [49] + │ │ │ │ │ │ ├── stats: [rows=2000, distinct(46)=1844.81, null(46)=0, distinct(49)=5, null(49)=0, distinct(55)=5, null(55)=0, distinct(57)=1, null(57)=0, distinct(61)=1, null(61)=0, distinct(62)=0.996222, null(62)=0] + │ │ │ │ │ │ ├── key: (46) + │ │ │ │ │ │ ├── fd: ()-->(62), (46)-->(49), (55)-->(57), (57)==(61), (61)==(57), (49)==(55), (55)==(49) + │ │ │ │ │ │ ├── inner-join (lookup nation@n_rk) + │ │ │ │ │ │ │ ├── save-table-name: q2_lookup_join_19 + │ │ │ │ │ │ │ ├── columns: n_nationkey:55(int!null) n_regionkey:57(int!null) r_regionkey:61(int!null) r_name:62(char!null) + │ │ │ │ │ │ │ ├── key columns: [61] = [57] + │ │ │ │ │ │ │ ├── stats: [rows=5, distinct(55)=5, null(55)=0, distinct(57)=1, null(57)=0, distinct(61)=1, null(61)=0, distinct(62)=0.996222, null(62)=0] + │ │ │ │ │ │ │ ├── key: (55) + │ │ │ │ │ │ │ ├── fd: ()-->(62), (55)-->(57), (57)==(61), (61)==(57) + │ │ │ │ │ │ │ ├── select + │ │ │ │ │ │ │ │ ├── save-table-name: q2_select_20 + │ │ │ │ │ │ │ │ ├── columns: r_regionkey:61(int!null) r_name:62(char!null) + │ │ │ │ │ │ │ │ ├── stats: [rows=1, distinct(61)=1, null(61)=0, distinct(62)=1, null(62)=0] + │ │ │ │ │ │ │ │ │ histogram(62)= 0 1 + │ │ │ │ │ │ │ │ │ <--- 'EUROPE' + │ │ │ │ │ │ │ │ ├── key: (61) + │ │ │ │ │ │ │ │ ├── fd: ()-->(62) + │ │ │ │ │ │ │ │ ├── scan region + │ │ │ │ │ │ │ │ │ ├── save-table-name: q2_scan_21 + │ │ │ │ │ │ │ │ │ ├── columns: r_regionkey:61(int!null) r_name:62(char!null) + │ │ │ │ │ │ │ │ │ ├── stats: [rows=5, distinct(61)=5, null(61)=0, distinct(62)=5, null(62)=0] + │ │ │ │ │ │ │ │ │ │ histogram(61)= 0 1 0 1 0 1 0 1 0 1 + │ │ │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 + │ │ │ │ │ │ │ │ │ │ histogram(62)= 0 1 3 1 + │ │ │ │ │ │ │ │ │ │ <--- 'AFRICA' --- 'MIDDLE EAST' + │ │ │ │ │ │ │ │ │ ├── key: (61) + │ │ │ │ │ │ │ │ │ └── fd: (61)-->(62) + │ │ │ │ │ │ │ │ └── filters + │ │ │ │ │ │ │ │ └── r_name:62 = 'EUROPE' [type=bool, outer=(62), constraints=(/62: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(62)] + │ │ │ │ │ │ │ └── filters (true) + │ │ │ │ │ │ └── filters (true) + │ │ │ │ │ └── filters + │ │ │ │ │ └── s_suppkey:46 = ps_suppkey:40 [type=bool, outer=(40,46), constraints=(/40: (/NULL - ]; /46: (/NULL - ]), fd=(40)==(46), (46)==(40)] + │ │ │ │ └── filters + │ │ │ │ └── ps_partkey:21 = ps_partkey:39 [type=bool, outer=(21,39), constraints=(/21: (/NULL - ]; /39: (/NULL - ]), fd=(21)==(39), (39)==(21)] + │ │ │ └── aggregations + │ │ │ ├── min [as=min:66, type=float, outer=(42)] + │ │ │ │ └── ps_supplycost:42 [type=float] + │ │ │ ├── const-agg [as=s_name:13, type=char, outer=(13)] + │ │ │ │ └── s_name:13 [type=char] + │ │ │ ├── const-agg [as=s_address:14, type=varchar, outer=(14)] + │ │ │ │ └── s_address:14 [type=varchar] + │ │ │ ├── const-agg [as=s_phone:16, type=char, outer=(16)] + │ │ │ │ └── s_phone:16 [type=char] + │ │ │ ├── const-agg [as=s_acctbal:17, type=float, outer=(17)] + │ │ │ │ └── s_acctbal:17 [type=float] + │ │ │ ├── const-agg [as=s_comment:18, type=varchar, outer=(18)] + │ │ │ │ └── s_comment:18 [type=varchar] + │ │ │ ├── const-agg [as=ps_supplycost:24, type=float, outer=(24)] + │ │ │ │ └── ps_supplycost:24 [type=float] + │ │ │ └── const-agg [as=n_name:29, type=char, outer=(29)] + │ │ │ └── n_name:29 [type=char] + │ │ └── filters + │ │ └── ps_supplycost:24 = min:66 [type=bool, outer=(24,66), constraints=(/24: (/NULL - ]; /66: (/NULL - ]), fd=(24)==(66), (66)==(24)] + │ └── filters + │ ├── p_size:6 = 15 [type=bool, outer=(6), constraints=(/6: [/15 - /15]; tight), fd=()-->(6)] + │ └── p_type:5 LIKE '%BRASS' [type=bool, outer=(5), constraints=(/5: (/NULL - ])] + └── 100 [type=int] ----Stats for q2_project_1---- column_names row_count distinct_count null_count {n_name} 100 5 0 {p_mfgr} 100 5 0 {p_partkey} 100 100 0 -{s_acctbal} 100 89 0 -{s_address} 100 89 0 -{s_comment} 100 89 0 -{s_name} 100 89 0 -{s_phone} 100 89 0 +{s_acctbal} 100 94 0 +{s_address} 100 94 0 +{s_comment} 100 94 0 +{s_name} 100 94 0 +{s_phone} 100 94 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err {n_name} 1.00 100.00 <== 1.00 5.00 <== 0.00 1.00 {p_mfgr} 1.00 100.00 <== 1.00 5.00 <== 0.00 1.00 {p_partkey} 1.00 100.00 <== 1.00 100.00 <== 0.00 1.00 -{s_acctbal} 1.00 100.00 <== 1.00 89.00 <== 0.00 1.00 -{s_address} 1.00 100.00 <== 1.00 89.00 <== 0.00 1.00 -{s_comment} 1.00 100.00 <== 1.00 89.00 <== 0.00 1.00 -{s_name} 1.00 100.00 <== 1.00 89.00 <== 0.00 1.00 -{s_phone} 1.00 100.00 <== 1.00 89.00 <== 0.00 1.00 +{s_acctbal} 1.00 100.00 <== 1.00 94.00 <== 0.00 1.00 +{s_address} 1.00 100.00 <== 1.00 94.00 <== 0.00 1.00 +{s_comment} 1.00 100.00 <== 1.00 94.00 <== 0.00 1.00 +{s_name} 1.00 100.00 <== 1.00 94.00 <== 0.00 1.00 +{s_phone} 1.00 100.00 <== 1.00 94.00 <== 0.00 1.00 -----Stats for q2_top_k_2---- +----Stats for q2_limit_2---- column_names row_count distinct_count null_count -{min} 100 100 0 +{min} 100 95 0 {n_name} 100 5 0 {p_mfgr} 100 5 0 {p_partkey} 100 100 0 +{p_size} 100 1 0 +{p_type} 100 26 0 {ps_partkey} 100 100 0 -{ps_suppkey} 100 89 0 -{ps_supplycost} 100 100 0 -{s_acctbal} 100 89 0 -{s_address} 100 89 0 -{s_comment} 100 89 0 -{s_name} 100 89 0 -{s_phone} 100 89 0 +{ps_suppkey} 100 94 0 +{ps_supplycost} 100 95 0 +{s_acctbal} 100 94 0 +{s_address} 100 94 0 +{s_comment} 100 94 0 +{s_name} 100 94 0 +{s_phone} 100 94 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{min} 1.00 100.00 <== 1.00 100.00 <== 0.00 1.00 +{min} 1.00 100.00 <== 1.00 95.00 <== 0.00 1.00 {n_name} 1.00 100.00 <== 1.00 5.00 <== 0.00 1.00 {p_mfgr} 1.00 100.00 <== 1.00 5.00 <== 0.00 1.00 {p_partkey} 1.00 100.00 <== 1.00 100.00 <== 0.00 1.00 +{p_size} 1.00 100.00 <== 1.00 1.00 0.00 1.00 +{p_type} 1.00 100.00 <== 1.00 26.00 <== 0.00 1.00 {ps_partkey} 1.00 100.00 <== 1.00 100.00 <== 0.00 1.00 -{ps_suppkey} 1.00 100.00 <== 1.00 89.00 <== 0.00 1.00 -{ps_supplycost} 1.00 100.00 <== 1.00 100.00 <== 0.00 1.00 -{s_acctbal} 1.00 100.00 <== 1.00 89.00 <== 0.00 1.00 -{s_address} 1.00 100.00 <== 1.00 89.00 <== 0.00 1.00 -{s_comment} 1.00 100.00 <== 1.00 89.00 <== 0.00 1.00 -{s_name} 1.00 100.00 <== 1.00 89.00 <== 0.00 1.00 -{s_phone} 1.00 100.00 <== 1.00 89.00 <== 0.00 1.00 +{ps_suppkey} 1.00 100.00 <== 1.00 94.00 <== 0.00 1.00 +{ps_supplycost} 1.00 100.00 <== 1.00 95.00 <== 0.00 1.00 +{s_acctbal} 1.00 100.00 <== 1.00 94.00 <== 0.00 1.00 +{s_address} 1.00 100.00 <== 1.00 94.00 <== 0.00 1.00 +{s_comment} 1.00 100.00 <== 1.00 94.00 <== 0.00 1.00 +{s_name} 1.00 100.00 <== 1.00 94.00 <== 0.00 1.00 +{s_phone} 1.00 100.00 <== 1.00 94.00 <== 0.00 1.00 -----Stats for q2_select_3---- +----Stats for q2_lookup_join_3---- column_names row_count distinct_count null_count -{min} 460 458 0 -{n_name} 460 5 0 -{p_mfgr} 460 5 0 -{p_partkey} 460 460 0 -{ps_partkey} 460 460 0 -{ps_suppkey} 460 406 0 -{ps_supplycost} 460 458 0 -{s_acctbal} 460 406 0 -{s_address} 460 406 0 -{s_comment} 460 406 0 -{s_name} 460 406 0 -{s_phone} 460 406 0 +{min} 100 95 0 +{n_name} 100 5 0 +{p_mfgr} 100 5 0 +{p_partkey} 100 100 0 +{p_size} 100 1 0 +{p_type} 100 26 0 +{ps_partkey} 100 100 0 +{ps_suppkey} 100 94 0 +{ps_supplycost} 100 95 0 +{s_acctbal} 100 94 0 +{s_address} 100 94 0 +{s_comment} 100 94 0 +{s_name} 100 94 0 +{s_phone} 100 94 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{min} 1.00 460.00 <== 1.00 458.00 <== 0.00 1.00 -{n_name} 1.00 460.00 <== 1.00 5.00 <== 0.00 1.00 -{p_mfgr} 1.00 460.00 <== 1.00 5.00 <== 0.00 1.00 -{p_partkey} 1.00 460.00 <== 1.00 460.00 <== 0.00 1.00 -{ps_partkey} 1.00 460.00 <== 1.00 460.00 <== 0.00 1.00 -{ps_suppkey} 1.00 460.00 <== 1.00 406.00 <== 0.00 1.00 -{ps_supplycost} 1.00 460.00 <== 1.00 458.00 <== 0.00 1.00 -{s_acctbal} 1.00 460.00 <== 1.00 406.00 <== 0.00 1.00 -{s_address} 1.00 460.00 <== 1.00 406.00 <== 0.00 1.00 -{s_comment} 1.00 460.00 <== 1.00 406.00 <== 0.00 1.00 -{s_name} 1.00 460.00 <== 1.00 406.00 <== 0.00 1.00 -{s_phone} 1.00 460.00 <== 1.00 406.00 <== 0.00 1.00 +{min} 1.00 100.00 <== 1.00 95.00 <== 0.00 1.00 +{n_name} 1.00 100.00 <== 1.00 5.00 <== 0.00 1.00 +{p_mfgr} 1.00 100.00 <== 1.00 5.00 <== 0.00 1.00 +{p_partkey} 1.00 100.00 <== 1.00 100.00 <== 0.00 1.00 +{p_size} 1.00 100.00 <== 1.00 1.00 0.00 1.00 +{p_type} 1.00 100.00 <== 1.00 26.00 <== 0.00 1.00 +{ps_partkey} 1.00 100.00 <== 1.00 100.00 <== 0.00 1.00 +{ps_suppkey} 1.00 100.00 <== 1.00 94.00 <== 0.00 1.00 +{ps_supplycost} 1.00 100.00 <== 1.00 95.00 <== 0.00 1.00 +{s_acctbal} 1.00 100.00 <== 1.00 94.00 <== 0.00 1.00 +{s_address} 1.00 100.00 <== 1.00 94.00 <== 0.00 1.00 +{s_comment} 1.00 100.00 <== 1.00 94.00 <== 0.00 1.00 +{s_name} 1.00 100.00 <== 1.00 94.00 <== 0.00 1.00 +{s_phone} 1.00 100.00 <== 1.00 94.00 <== 0.00 1.00 -----Stats for q2_group_by_4---- +----Stats for q2_sort_4---- column_names row_count distinct_count null_count -{min} 642 458 0 -{n_name} 642 5 0 -{p_mfgr} 642 5 0 -{p_partkey} 642 460 0 -{ps_partkey} 642 460 0 -{ps_suppkey} 642 548 0 -{ps_supplycost} 642 640 0 -{s_acctbal} 642 548 0 -{s_address} 642 548 0 -{s_comment} 642 548 0 -{s_name} 642 548 0 -{s_phone} 642 548 0 +{min} 32700 1000 0 +{n_name} 32700 5 0 +{ps_partkey} 32700 32444 0 +{ps_suppkey} 32700 553 0 +{ps_supplycost} 32700 1000 0 +{s_acctbal} 32700 553 0 +{s_address} 32700 553 0 +{s_comment} 32700 553 0 +{s_name} 32700 553 0 +{s_phone} 32700 553 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{min} 1476.00 2.30 <== 1476.00 3.22 <== 0.00 1.00 -{n_name} 1476.00 2.30 <== 1476.00 295.20 <== 0.00 1.00 -{p_mfgr} 1476.00 2.30 <== 1476.00 295.20 <== 0.00 1.00 -{p_partkey} 1476.00 2.30 <== 1476.00 3.21 <== 0.00 1.00 -{ps_partkey} 1476.00 2.30 <== 1171.00 2.55 <== 0.00 1.00 -{ps_suppkey} 1476.00 2.30 <== 1402.00 2.56 <== 0.00 1.00 -{ps_supplycost} 1476.00 2.30 <== 1476.00 2.31 <== 0.00 1.00 -{s_acctbal} 1476.00 2.30 <== 1476.00 2.69 <== 0.00 1.00 -{s_address} 1476.00 2.30 <== 1476.00 2.69 <== 0.00 1.00 -{s_comment} 1476.00 2.30 <== 1476.00 2.69 <== 0.00 1.00 -{s_name} 1476.00 2.30 <== 1476.00 2.69 <== 0.00 1.00 -{s_phone} 1476.00 2.30 <== 1476.00 2.69 <== 0.00 1.00 +{min} 1.00 32700.00 <== 1.00 1000.00 <== 0.00 1.00 +{n_name} 1.00 32700.00 <== 1.00 5.00 <== 0.00 1.00 +{ps_partkey} 1.00 32700.00 <== 1.00 32444.00 <== 0.00 1.00 +{ps_suppkey} 1.00 32700.00 <== 1.00 553.00 <== 0.00 1.00 +{ps_supplycost} 1.00 32700.00 <== 1.00 1000.00 <== 0.00 1.00 +{s_acctbal} 1.00 32700.00 <== 1.00 553.00 <== 0.00 1.00 +{s_address} 1.00 32700.00 <== 1.00 553.00 <== 0.00 1.00 +{s_comment} 1.00 32700.00 <== 1.00 553.00 <== 0.00 1.00 +{s_name} 1.00 32700.00 <== 1.00 553.00 <== 0.00 1.00 +{s_phone} 1.00 32700.00 <== 1.00 553.00 <== 0.00 1.00 -----Stats for q2_inner_join_5---- -column_names row_count distinct_count null_count -{n_name} 1070 5 0 -{n_nationkey_1} 1070 5 0 -{n_nationkey} 1070 5 0 -{n_regionkey_1} 1070 1 0 -{n_regionkey} 1070 1 0 -{p_mfgr} 1070 5 0 -{p_partkey} 1070 460 0 -{p_size} 1070 1 0 -{p_type} 1070 30 0 -{ps_partkey_1} 1070 460 0 -{ps_partkey} 1070 460 0 -{ps_suppkey_1} 1070 548 0 -{ps_suppkey} 1070 548 0 -{ps_supplycost_1} 1070 640 0 -{ps_supplycost} 1070 640 0 -{r_name_1} 1070 1 0 -{r_name} 1070 1 0 -{r_regionkey_1} 1070 1 0 -{r_regionkey} 1070 1 0 -{s_acctbal} 1070 548 0 -{s_address} 1070 548 0 -{s_comment} 1070 548 0 -{s_name} 1070 548 0 -{s_nationkey_1} 1070 5 0 -{s_nationkey} 1070 5 0 -{s_phone} 1070 548 0 -{s_suppkey_1} 1070 548 0 -{s_suppkey} 1070 548 0 +----Stats for q2_select_5---- +column_names row_count distinct_count null_count +{min} 117375 1000 0 +{n_name} 117375 5 0 +{ps_partkey} 117375 116575 0 +{ps_suppkey} 117375 1981 0 +{ps_supplycost} 117375 1000 0 +{s_acctbal} 117375 1979 0 +{s_address} 117375 1981 0 +{s_comment} 117375 1981 0 +{s_name} 117375 1981 0 +{s_phone} 117375 1981 0 ~~~~ -column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{n_name} 2816.00 2.63 <== 5.00 1.00 0.00 1.00 -{n_nationkey} 2816.00 2.63 <== 5.00 1.00 0.00 1.00 -{n_nationkey_1} 2816.00 2.63 <== 5.00 1.00 0.00 1.00 -{n_regionkey} 2816.00 2.63 <== 1.00 1.00 0.00 1.00 -{n_regionkey_1} 2816.00 2.63 <== 1.00 1.00 0.00 1.00 -{p_mfgr} 2816.00 2.63 <== 5.00 1.00 0.00 1.00 -{p_partkey} 2816.00 2.63 <== 1332.00 2.90 <== 0.00 1.00 -{p_size} 2816.00 2.63 <== 1.00 1.00 0.00 1.00 -{p_type} 2816.00 2.63 <== 150.00 5.00 <== 0.00 1.00 -{ps_partkey} 2816.00 2.63 <== 1171.00 2.55 <== 0.00 1.00 -{ps_partkey_1} 2816.00 2.63 <== 1332.00 2.90 <== 0.00 1.00 -{ps_suppkey} 2816.00 2.63 <== 1402.00 2.56 <== 0.00 1.00 -{ps_suppkey_1} 2816.00 2.63 <== 1444.00 2.64 <== 0.00 1.00 -{ps_supplycost} 2816.00 2.63 <== 1472.00 2.30 <== 0.00 1.00 -{ps_supplycost_1} 2816.00 2.63 <== 2767.00 4.32 <== 0.00 1.00 -{r_name} 2816.00 2.63 <== 1.00 1.00 0.00 1.00 -{r_name_1} 2816.00 2.63 <== 1.00 1.00 0.00 1.00 -{r_regionkey} 2816.00 2.63 <== 1.00 1.00 0.00 1.00 -{r_regionkey_1} 2816.00 2.63 <== 1.00 1.00 0.00 1.00 -{s_acctbal} 2816.00 2.63 <== 1402.00 2.56 <== 0.00 1.00 -{s_address} 2816.00 2.63 <== 1403.00 2.56 <== 0.00 1.00 -{s_comment} 2816.00 2.63 <== 1402.00 2.56 <== 0.00 1.00 -{s_name} 2816.00 2.63 <== 1402.00 2.56 <== 0.00 1.00 -{s_nationkey} 2816.00 2.63 <== 5.00 1.00 0.00 1.00 -{s_nationkey_1} 2816.00 2.63 <== 5.00 1.00 0.00 1.00 -{s_phone} 2816.00 2.63 <== 1403.00 2.56 <== 0.00 1.00 -{s_suppkey} 2816.00 2.63 <== 1402.00 2.56 <== 0.00 1.00 -{s_suppkey_1} 2816.00 2.63 <== 1444.00 2.64 <== 0.00 1.00 +column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err +{min} 1.00 117375.00 <== 1.00 1000.00 <== 0.00 1.00 +{n_name} 1.00 117375.00 <== 1.00 5.00 <== 0.00 1.00 +{ps_partkey} 1.00 117375.00 <== 1.00 116575.00 <== 0.00 1.00 +{ps_suppkey} 1.00 117375.00 <== 1.00 1981.00 <== 0.00 1.00 +{ps_supplycost} 1.00 117375.00 <== 1.00 1000.00 <== 0.00 1.00 +{s_acctbal} 1.00 117375.00 <== 1.00 1979.00 <== 0.00 1.00 +{s_address} 1.00 117375.00 <== 1.00 1981.00 <== 0.00 1.00 +{s_comment} 1.00 117375.00 <== 1.00 1981.00 <== 0.00 1.00 +{s_name} 1.00 117375.00 <== 1.00 1981.00 <== 0.00 1.00 +{s_phone} 1.00 117375.00 <== 1.00 1981.00 <== 0.00 1.00 -----Stats for q2_lookup_join_6---- +----Stats for q2_group_by_6---- +column_names row_count distinct_count null_count +{min} 158480 1000 0 +{n_name} 158480 5 0 +{ps_partkey} 158480 116575 0 +{ps_suppkey} 158480 1981 0 +{ps_supplycost} 158480 1000 0 +{s_acctbal} 158480 1979 0 +{s_address} 158480 1981 0 +{s_comment} 158480 1981 0 +{s_name} 158480 1981 0 +{s_phone} 158480 1981 0 +~~~~ +column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err +{min} 113914.00 1.39 113914.00 113.91 <== 0.00 1.00 +{n_name} 113914.00 1.39 113914.00 22782.80 <== 0.00 1.00 +{ps_partkey} 113914.00 1.39 110568.00 1.05 0.00 1.00 +{ps_suppkey} 113914.00 1.39 9920.00 5.01 <== 0.00 1.00 +{ps_supplycost} 113914.00 1.39 113914.00 113.91 <== 0.00 1.00 +{s_acctbal} 113914.00 1.39 113914.00 57.56 <== 0.00 1.00 +{s_address} 113914.00 1.39 113914.00 57.50 <== 0.00 1.00 +{s_comment} 113914.00 1.39 113914.00 57.50 <== 0.00 1.00 +{s_name} 113914.00 1.39 113914.00 57.50 <== 0.00 1.00 +{s_phone} 113914.00 1.39 113914.00 57.50 <== 0.00 1.00 + +----Stats for q2_inner_join_7---- column_names row_count distinct_count null_count -{n_name} 2568 5 0 -{n_nationkey} 2568 5 0 -{n_regionkey} 2568 1 0 -{p_mfgr} 2568 5 0 -{p_partkey} 2568 460 0 -{p_size} 2568 1 0 -{p_type} 2568 30 0 -{ps_partkey_1} 2568 460 0 -{ps_partkey} 2568 460 0 -{ps_suppkey_1} 2568 1661 0 -{ps_suppkey} 2568 548 0 -{ps_supplycost_1} 2568 1825 0 -{ps_supplycost} 2568 640 0 -{r_name} 2568 1 0 -{r_regionkey} 2568 1 0 -{s_acctbal} 2568 548 0 -{s_address} 2568 548 0 -{s_comment} 2568 548 0 -{s_name} 2568 548 0 -{s_nationkey} 2568 5 0 -{s_phone} 2568 548 0 -{s_suppkey} 2568 548 0 +{n_name} 252562 5 0 +{n_nationkey_1} 252562 5 0 +{n_nationkey} 252562 5 0 +{n_regionkey_1} 252562 1 0 +{n_regionkey} 252562 1 0 +{ps_partkey_1} 252562 116575 0 +{ps_partkey} 252562 116575 0 +{ps_suppkey_1} 252562 1981 0 +{ps_suppkey} 252562 1981 0 +{ps_supplycost_1} 252562 1000 0 +{ps_supplycost} 252562 1000 0 +{r_name_1} 252562 1 0 +{r_name} 252562 1 0 +{r_regionkey_1} 252562 1 0 +{r_regionkey} 252562 1 0 +{s_acctbal} 252562 1979 0 +{s_address} 252562 1981 0 +{s_comment} 252562 1981 0 +{s_name} 252562 1981 0 +{s_nationkey_1} 252562 5 0 +{s_nationkey} 252562 5 0 +{s_phone} 252562 1981 0 +{s_suppkey_1} 252562 1981 0 +{s_suppkey} 252562 1981 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{n_name} 4449.00 1.73 25.00 5.00 <== 0.00 1.00 -{n_nationkey} 4449.00 1.73 25.00 5.00 <== 0.00 1.00 -{n_regionkey} 4449.00 1.73 1.00 1.00 0.00 1.00 -{p_mfgr} 4449.00 1.73 5.00 1.00 0.00 1.00 -{p_partkey} 4449.00 1.73 1332.00 2.90 <== 0.00 1.00 -{p_size} 4449.00 1.73 1.00 1.00 0.00 1.00 -{p_type} 4449.00 1.73 150.00 5.00 <== 0.00 1.00 -{ps_partkey} 4449.00 1.73 1332.00 2.90 <== 0.00 1.00 -{ps_partkey_1} 4449.00 1.73 4398.00 9.56 <== 0.00 1.00 -{ps_suppkey} 4449.00 1.73 3585.00 6.54 <== 0.00 1.00 -{ps_suppkey_1} 4449.00 1.73 3585.00 2.16 <== 0.00 1.00 -{ps_supplycost} 4449.00 1.73 4352.00 6.80 <== 0.00 1.00 -{ps_supplycost_1} 4449.00 1.73 4352.00 2.38 <== 0.00 1.00 -{r_name} 4449.00 1.73 1.00 1.00 0.00 1.00 -{r_regionkey} 4449.00 1.73 1.00 1.00 0.00 1.00 -{s_acctbal} 4449.00 1.73 3589.00 6.55 <== 0.00 1.00 -{s_address} 4449.00 1.73 3591.00 6.55 <== 0.00 1.00 -{s_comment} 4449.00 1.73 3586.00 6.54 <== 0.00 1.00 -{s_name} 4449.00 1.73 3590.00 6.55 <== 0.00 1.00 -{s_nationkey} 4449.00 1.73 25.00 5.00 <== 0.00 1.00 -{s_phone} 4449.00 1.73 3591.00 6.55 <== 0.00 1.00 -{s_suppkey} 4449.00 1.73 3585.00 6.54 <== 0.00 1.00 +{n_name} 233690.00 1.08 5.00 1.00 0.00 1.00 +{n_nationkey} 233690.00 1.08 5.00 1.00 0.00 1.00 +{n_nationkey_1} 233690.00 1.08 5.00 1.00 0.00 1.00 +{n_regionkey} 233690.00 1.08 1.00 1.00 0.00 1.00 +{n_regionkey_1} 233690.00 1.08 1.00 1.00 0.00 1.00 +{ps_partkey} 233690.00 1.08 110568.00 1.05 0.00 1.00 +{ps_partkey_1} 233690.00 1.08 110568.00 1.05 0.00 1.00 +{ps_suppkey} 233690.00 1.08 9920.00 5.01 <== 0.00 1.00 +{ps_suppkey_1} 233690.00 1.08 1845.00 1.07 0.00 1.00 +{ps_supplycost} 233690.00 1.08 76389.00 76.39 <== 0.00 1.00 +{ps_supplycost_1} 233690.00 1.08 75889.00 75.89 <== 0.00 1.00 +{r_name} 233690.00 1.08 1.00 1.00 0.00 1.00 +{r_name_1} 233690.00 1.08 1.00 1.00 0.00 1.00 +{r_regionkey} 233690.00 1.08 1.00 1.00 0.00 1.00 +{r_regionkey_1} 233690.00 1.08 1.00 1.00 0.00 1.00 +{s_acctbal} 233690.00 1.08 9967.00 5.04 <== 0.00 1.00 +{s_address} 233690.00 1.08 10000.00 5.05 <== 0.00 1.00 +{s_comment} 233690.00 1.08 9934.00 5.01 <== 0.00 1.00 +{s_name} 233690.00 1.08 9990.00 5.04 <== 0.00 1.00 +{s_nationkey} 233690.00 1.08 5.00 1.00 0.00 1.00 +{s_nationkey_1} 233690.00 1.08 5.00 1.00 0.00 1.00 +{s_phone} 233690.00 1.08 10000.00 5.05 <== 0.00 1.00 +{s_suppkey} 233690.00 1.08 9920.00 5.01 <== 0.00 1.00 +{s_suppkey_1} 233690.00 1.08 1845.00 1.07 0.00 1.00 -----Stats for q2_inner_join_7---- +----Stats for q2_inner_join_8---- column_names row_count distinct_count null_count -{n_name} 642 5 0 -{n_nationkey} 642 5 0 -{n_regionkey} 642 1 0 -{p_mfgr} 642 5 0 -{p_partkey} 642 460 0 -{p_size} 642 1 0 -{p_type} 642 30 0 -{ps_partkey} 642 460 0 -{ps_suppkey} 642 548 0 -{ps_supplycost} 642 640 0 -{r_name} 642 1 0 -{r_regionkey} 642 1 0 -{s_acctbal} 642 548 0 -{s_address} 642 548 0 -{s_comment} 642 548 0 -{s_name} 642 548 0 -{s_nationkey} 642 5 0 -{s_phone} 642 548 0 -{s_suppkey} 642 548 0 +{n_name} 158480 5 0 +{n_nationkey} 158480 5 0 +{n_regionkey} 158480 1 0 +{ps_partkey} 158480 116575 0 +{ps_suppkey} 158480 1981 0 +{ps_supplycost} 158480 1000 0 +{r_name} 158480 1 0 +{r_regionkey} 158480 1 0 +{s_acctbal} 158480 1979 0 +{s_address} 158480 1981 0 +{s_comment} 158480 1981 0 +{s_name} 158480 1981 0 +{s_nationkey} 158480 5 0 +{s_phone} 158480 1981 0 +{s_suppkey} 158480 1981 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{n_name} 1930.00 3.01 <== 5.00 1.00 0.00 1.00 -{n_nationkey} 1930.00 3.01 <== 5.00 1.00 0.00 1.00 -{n_regionkey} 1930.00 3.01 <== 1.00 1.00 0.00 1.00 -{p_mfgr} 1930.00 3.01 <== 5.00 1.00 0.00 1.00 -{p_partkey} 1930.00 3.01 <== 1332.00 2.90 <== 0.00 1.00 -{p_size} 1930.00 3.01 <== 1.00 1.00 0.00 1.00 -{p_type} 1930.00 3.01 <== 150.00 5.00 <== 0.00 1.00 -{ps_partkey} 1930.00 3.01 <== 1332.00 2.90 <== 0.00 1.00 -{ps_suppkey} 1930.00 3.01 <== 1754.00 3.20 <== 0.00 1.00 -{ps_supplycost} 1930.00 3.01 <== 1908.00 2.98 <== 0.00 1.00 -{r_name} 1930.00 3.01 <== 1.00 1.00 0.00 1.00 -{r_regionkey} 1930.00 3.01 <== 1.00 1.00 0.00 1.00 -{s_acctbal} 1930.00 3.01 <== 1755.00 3.20 <== 0.00 1.00 -{s_address} 1930.00 3.01 <== 1756.00 3.20 <== 0.00 1.00 -{s_comment} 1930.00 3.01 <== 1754.00 3.20 <== 0.00 1.00 -{s_name} 1930.00 3.01 <== 1755.00 3.20 <== 0.00 1.00 -{s_nationkey} 1930.00 3.01 <== 5.00 1.00 0.00 1.00 -{s_phone} 1930.00 3.01 <== 1756.00 3.20 <== 0.00 1.00 -{s_suppkey} 1930.00 3.01 <== 1754.00 3.20 <== 0.00 1.00 +{n_name} 161290.00 1.02 5.00 1.00 0.00 1.00 +{n_nationkey} 161290.00 1.02 5.00 1.00 0.00 1.00 +{n_regionkey} 161290.00 1.02 1.00 1.00 0.00 1.00 +{ps_partkey} 161290.00 1.02 111321.00 1.05 0.00 1.00 +{ps_suppkey} 161290.00 1.02 9920.00 5.01 <== 0.00 1.00 +{ps_supplycost} 161290.00 1.02 80888.00 80.89 <== 0.00 1.00 +{r_name} 161290.00 1.02 1.00 1.00 0.00 1.00 +{r_regionkey} 161290.00 1.02 1.00 1.00 0.00 1.00 +{s_acctbal} 161290.00 1.02 9967.00 5.04 <== 0.00 1.00 +{s_address} 161290.00 1.02 10000.00 5.05 <== 0.00 1.00 +{s_comment} 161290.00 1.02 9934.00 5.01 <== 0.00 1.00 +{s_name} 161290.00 1.02 9990.00 5.04 <== 0.00 1.00 +{s_nationkey} 161290.00 1.02 5.00 1.00 0.00 1.00 +{s_phone} 161290.00 1.02 10000.00 5.05 <== 0.00 1.00 +{s_suppkey} 161290.00 1.02 9920.00 5.01 <== 0.00 1.00 -----Stats for q2_lookup_join_8---- +----Stats for q2_scan_9---- column_names row_count distinct_count null_count -{p_mfgr} 2988 5 0 -{p_partkey} 2988 747 0 -{p_size} 2988 1 0 -{p_type} 2988 30 0 -{ps_partkey} 2988 747 0 -{ps_suppkey} 2988 2570 0 -{ps_supplycost} 2988 2948 0 +{ps_partkey} 800000 199241 0 +{ps_suppkey} 800000 9920 0 +{ps_supplycost} 800000 1000 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{p_mfgr} 5350.00 1.79 5.00 1.00 0.00 1.00 -{p_partkey} 5350.00 1.79 1332.00 1.78 0.00 1.00 -{p_size} 5350.00 1.79 1.00 1.00 0.00 1.00 -{p_type} 5350.00 1.79 150.00 5.00 <== 0.00 1.00 -{ps_partkey} 5350.00 1.79 1332.00 1.78 0.00 1.00 -{ps_suppkey} 5350.00 1.79 4135.00 1.61 0.00 1.00 -{ps_supplycost} 5350.00 1.79 5210.00 1.77 0.00 1.00 +{ps_partkey} 800000.00 1.00 199241.00 1.00 0.00 1.00 +{ps_suppkey} 800000.00 1.00 9920.00 1.00 0.00 1.00 +{ps_supplycost} 800000.00 1.00 100379.00 100.38 <== 0.00 1.00 -----Stats for q2_select_9---- -column_names row_count distinct_count null_count -{p_mfgr} 747 5 0 -{p_partkey} 747 747 0 -{p_size} 747 1 0 -{p_type} 747 30 0 -~~~~ -column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{p_mfgr} 1332.00 1.78 5.00 1.00 0.00 1.00 -{p_partkey} 1332.00 1.78 1332.00 1.78 0.00 1.00 -{p_size} 1332.00 1.78 1.00 1.00 0.00 1.00 -{p_type} 1332.00 1.78 150.00 5.00 <== 0.00 1.00 - -----Stats for q2_scan_10---- -column_names row_count distinct_count null_count -{p_mfgr} 200000 5 0 -{p_partkey} 200000 199241 0 -{p_size} 200000 50 0 -{p_type} 200000 150 0 -~~~~ -column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{p_mfgr} 200000.00 1.00 5.00 1.00 0.00 1.00 -{p_partkey} 200000.00 1.00 199241.00 1.00 0.00 1.00 -{p_size} 200000.00 1.00 50.00 1.00 0.00 1.00 -{p_type} 200000.00 1.00 150.00 1.00 0.00 1.00 - -----Stats for q2_inner_join_11---- +----Stats for q2_inner_join_10---- column_names row_count distinct_count null_count -{n_name} 1987 5 0 -{n_nationkey} 1987 5 0 -{n_regionkey} 1987 1 0 -{r_name} 1987 1 0 -{r_regionkey} 1987 1 0 -{s_acctbal} 1987 1983 0 -{s_address} 1987 1986 0 -{s_comment} 1987 1987 0 -{s_name} 1987 1987 0 -{s_nationkey} 1987 5 0 -{s_phone} 1987 1987 0 -{s_suppkey} 1987 1987 0 +{n_name} 1981 5 0 +{n_nationkey} 1981 5 0 +{n_regionkey} 1981 1 0 +{r_name} 1981 1 0 +{r_regionkey} 1981 1 0 +{s_acctbal} 1981 1979 0 +{s_address} 1981 1981 0 +{s_comment} 1981 1981 0 +{s_name} 1981 1981 0 +{s_nationkey} 1981 5 0 +{s_phone} 1981 1981 0 +{s_suppkey} 1981 1981 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err {n_name} 2000.00 1.01 5.00 1.00 0.00 1.00 @@ -616,21 +578,21 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_ {r_name} 2000.00 1.01 1.00 1.00 0.00 1.00 {r_regionkey} 2000.00 1.01 1.00 1.00 0.00 1.00 {s_acctbal} 2000.00 1.01 1846.00 1.07 0.00 1.00 -{s_address} 2000.00 1.01 1846.00 1.08 0.00 1.00 -{s_comment} 2000.00 1.01 1845.00 1.08 0.00 1.00 -{s_name} 2000.00 1.01 1846.00 1.08 0.00 1.00 +{s_address} 2000.00 1.01 1846.00 1.07 0.00 1.00 +{s_comment} 2000.00 1.01 1845.00 1.07 0.00 1.00 +{s_name} 2000.00 1.01 1846.00 1.07 0.00 1.00 {s_nationkey} 2000.00 1.01 5.00 1.00 0.00 1.00 -{s_phone} 2000.00 1.01 1846.00 1.08 0.00 1.00 -{s_suppkey} 2000.00 1.01 1845.00 1.08 0.00 1.00 +{s_phone} 2000.00 1.01 1846.00 1.07 0.00 1.00 +{s_suppkey} 2000.00 1.01 1845.00 1.07 0.00 1.00 -----Stats for q2_scan_12---- +----Stats for q2_scan_11---- column_names row_count distinct_count null_count -{s_acctbal} 10000 9967 0 +{s_acctbal} 10000 10000 0 {s_address} 10000 10000 0 -{s_comment} 10000 9934 0 +{s_comment} 10000 9903 0 {s_name} 10000 9990 0 {s_nationkey} 10000 25 0 -{s_phone} 10000 10000 0 +{s_phone} 10000 9840 0 {s_suppkey} 10000 9920 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err @@ -639,10 +601,10 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_ {s_comment} 10000.00 1.00 9934.00 1.00 0.00 1.00 {s_name} 10000.00 1.00 9990.00 1.00 0.00 1.00 {s_nationkey} 10000.00 1.00 25.00 1.00 0.00 1.00 -{s_phone} 10000.00 1.00 10000.00 1.00 0.00 1.00 +{s_phone} 10000.00 1.00 10000.00 1.02 0.00 1.00 {s_suppkey} 10000.00 1.00 9920.00 1.00 0.00 1.00 -----Stats for q2_inner_join_13---- +----Stats for q2_inner_join_12---- column_names row_count distinct_count null_count {n_name} 5 5 0 {n_nationkey} 5 5 0 @@ -657,7 +619,7 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_ {r_name} 5.00 1.00 1.00 1.00 0.00 1.00 {r_regionkey} 5.00 1.00 1.00 1.00 0.00 1.00 -----Stats for q2_scan_14---- +----Stats for q2_scan_13---- column_names row_count distinct_count null_count {n_name} 25 25 0 {n_nationkey} 25 25 0 @@ -668,7 +630,7 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_ {n_nationkey} 25.00 1.00 25.00 1.00 0.00 1.00 {n_regionkey} 25.00 1.00 5.00 1.00 0.00 1.00 -----Stats for q2_select_15---- +----Stats for q2_select_14---- column_names row_count distinct_count null_count {r_name} 1 1 0 {r_regionkey} 1 1 0 @@ -677,7 +639,7 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_ {r_name} 1.00 1.00 1.00 1.00 0.00 1.00 {r_regionkey} 1.00 1.00 1.00 1.00 0.00 1.00 -----Stats for q2_scan_16---- +----Stats for q2_scan_15---- column_names row_count distinct_count null_count {r_name} 5 5 0 {r_regionkey} 5 5 0 @@ -686,14 +648,48 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_ {r_name} 5.00 1.00 5.00 1.00 0.00 1.00 {r_regionkey} 5.00 1.00 5.00 1.00 0.00 1.00 -----Stats for q2_lookup_join_17---- +----Stats for q2_inner_join_16---- +column_names row_count distinct_count null_count +{n_nationkey} 158480 5 0 +{n_regionkey} 158480 1 0 +{ps_partkey} 158480 116575 0 +{ps_suppkey} 158480 1981 0 +{ps_supplycost} 158480 1000 0 +{r_name} 158480 1 0 +{r_regionkey} 158480 1 0 +{s_nationkey} 158480 5 0 +{s_suppkey} 158480 1981 0 +~~~~ +column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err +{n_nationkey} 161290.00 1.02 5.00 1.00 0.00 1.00 +{n_regionkey} 161290.00 1.02 1.00 1.00 0.00 1.00 +{ps_partkey} 161290.00 1.02 110568.00 1.05 0.00 1.00 +{ps_suppkey} 161290.00 1.02 1845.00 1.07 0.00 1.00 +{ps_supplycost} 161290.00 1.02 80252.00 80.25 <== 0.00 1.00 +{r_name} 161290.00 1.02 1.00 1.00 0.00 1.00 +{r_regionkey} 161290.00 1.02 1.00 1.00 0.00 1.00 +{s_nationkey} 161290.00 1.02 5.00 1.00 0.00 1.00 +{s_suppkey} 161290.00 1.02 1845.00 1.07 0.00 1.00 + +----Stats for q2_scan_17---- +column_names row_count distinct_count null_count +{ps_partkey} 800000 199241 0 +{ps_suppkey} 800000 9920 0 +{ps_supplycost} 800000 1000 0 +~~~~ +column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err +{ps_partkey} 800000.00 1.00 199241.00 1.00 0.00 1.00 +{ps_suppkey} 800000.00 1.00 9920.00 1.00 0.00 1.00 +{ps_supplycost} 800000.00 1.00 100379.00 100.38 <== 0.00 1.00 + +----Stats for q2_lookup_join_18---- column_names row_count distinct_count null_count -{n_nationkey} 1987 5 0 -{n_regionkey} 1987 1 0 -{r_name} 1987 1 0 -{r_regionkey} 1987 1 0 -{s_nationkey} 1987 5 0 -{s_suppkey} 1987 1987 0 +{n_nationkey} 1981 5 0 +{n_regionkey} 1981 1 0 +{r_name} 1981 1 0 +{r_regionkey} 1981 1 0 +{s_nationkey} 1981 5 0 +{s_suppkey} 1981 1981 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err {n_nationkey} 2000.00 1.01 5.00 1.00 0.00 1.00 @@ -701,9 +697,9 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_ {r_name} 2000.00 1.01 1.00 1.00 0.00 1.00 {r_regionkey} 2000.00 1.01 1.00 1.00 0.00 1.00 {s_nationkey} 2000.00 1.01 5.00 1.00 0.00 1.00 -{s_suppkey} 2000.00 1.01 1845.00 1.08 0.00 1.00 +{s_suppkey} 2000.00 1.01 1845.00 1.07 0.00 1.00 -----Stats for q2_lookup_join_18---- +----Stats for q2_lookup_join_19---- column_names row_count distinct_count null_count {n_nationkey} 5 5 0 {n_regionkey} 5 1 0 @@ -716,7 +712,7 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_ {r_name} 5.00 1.00 1.00 1.00 0.00 1.00 {r_regionkey} 5.00 1.00 1.00 1.00 0.00 1.00 -----Stats for q2_select_19---- +----Stats for q2_select_20---- column_names row_count distinct_count null_count {r_name} 1 1 0 {r_regionkey} 1 1 0 @@ -725,7 +721,7 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_ {r_name} 1.00 1.00 1.00 1.00 0.00 1.00 {r_regionkey} 1.00 1.00 1.00 1.00 0.00 1.00 -----Stats for q2_scan_20---- +----Stats for q2_scan_21---- column_names row_count distinct_count null_count {r_name} 5 5 0 {r_regionkey} 5 5 0 diff --git a/pkg/sql/opt/norm/decorrelate_funcs.go b/pkg/sql/opt/norm/decorrelate_funcs.go index c6839e46f28d..b8d65a1ac140 100644 --- a/pkg/sql/opt/norm/decorrelate_funcs.go +++ b/pkg/sql/opt/norm/decorrelate_funcs.go @@ -1058,3 +1058,186 @@ func (r *subqueryHoister) constructGroupByAny( opt.ColSet{}, ) } + +// CanMaybeRemapOuterCols performs a best-effort check to minimize the cases +// where TryRemapOuterCols is called unnecessarily. +func (c *CustomFuncs) CanMaybeRemapOuterCols(input memo.RelExpr, filters memo.FiltersExpr) bool { + // The usages of ComputeEquivClosureNoCopy are ok because of the copy below. + outerCols := input.Relational().OuterCols.Copy() + equivGroup := input.Relational().FuncDeps.ComputeEquivClosureNoCopy(outerCols) + for i := range filters { + if equivGroup.Intersects(input.Relational().OutputCols) { + return true + } + equivGroup = filters[i].ScalarProps().FuncDeps.ComputeEquivClosureNoCopy(equivGroup) + } + return equivGroup.Intersects(input.Relational().OutputCols) +} + +// TryRemapOuterCols attempts to replace outer column references in the given +// expression with equivalent non-outer columns using equalities from the given +// filters. It accomplishes this by traversing the operator tree for each outer +// column with the set of equivalent non-outer columns, wherever it would be +// valid to push down a filter on those non-outer columns. If a reference to the +// outer column is discovered during this traversal, it is valid to replace it +// with one of the non-outer columns in the set. +func (c *CustomFuncs) TryRemapOuterCols( + expr memo.RelExpr, filters memo.FiltersExpr, +) (remapped memo.RelExpr, wasRemapped bool) { + outerCols := expr.Relational().OuterCols + remapped = expr + for col, ok := outerCols.Next(0); ok; col, ok = outerCols.Next(col + 1) { + // substituteCols is the set of input columns for which it may be possible to + // push a filter constraining the column to be equal to an outer column. + // Doing so would allow the column to be substituted for the outer column. + substituteCols := expr.Relational().FuncDeps.ComputeEquivGroup(col) + for i := range filters { + // ComputeEquivClosureNoCopy is ok here because ComputeEquivGroup builds + // a new ColSet. + substituteCols = filters[i].ScalarProps().FuncDeps.ComputeEquivClosureNoCopy(substituteCols) + } + substituteCols.DifferenceWith(outerCols) + remapped = c.tryRemapOuterCols(remapped, col, substituteCols).(memo.RelExpr) + } + wasRemapped = remapped != expr + return remapped, wasRemapped +} + +// tryRemapOuterCols handles the traversal and outer-column replacement for +// TryRemapOuterCols. It returns the replacement expression and whether an +// outer-column reference was successfully remapped. +func (c *CustomFuncs) tryRemapOuterCols( + expr opt.Expr, outerCol opt.ColumnID, substituteCols opt.ColSet, +) opt.Expr { + if substituteCols.Empty() { + // It is not possible to remap any references to the current outer + // column within this expression. + return expr + } + switch t := expr.(type) { + case *memo.VariableExpr: + if t.Col == outerCol { + if replaceCol, ok := substituteCols.Next(0); ok { + // This outer-column reference can be remapped. + return c.f.ConstructVariable(replaceCol) + } + } + case memo.RelExpr: + if !t.Relational().OuterCols.Contains(outerCol) { + // This expression does not reference the current outer column. + return t + } + // Modifications to substituteCols may be necessary in order to allow + // outer-column remapping within (the children of) a RelExpr. Note that + // getSubstituteColsRelExpr copies substituteCols before modifying it, so + // different branches of the traversal don't interact with the same ColSet. + substituteCols = c.getSubstituteColsRelExpr(t, substituteCols) + case opt.ScalarExpr: + // Any substitute columns that reach a ScalarExpr are valid candidates + // for outer-column replacement. No changes to substituteCols required. + } + replaceFn := func(e opt.Expr) opt.Expr { + return c.tryRemapOuterCols(e, outerCol, substituteCols) + } + return c.f.Replace(expr, replaceFn) +} + +// getSubstituteColsRelExpr modifies the given set of substitute columns to +// reflect the set of columns for which an equality with an outer column could +// be pushed through the given expression. The logic of getSubstituteColsRelExpr +// mirrors that of the filter push-down rules in select.opt and join.opt. +// TODO(drewk): null-rejection has to push down a 'col IS NOT NULL' filter - +// we should be able to share logic. Doing so would remove the issue of rule +// cycles. Any other rules that reuse this logic should reconsider the +// simplification made in getSubstituteColsSetOp. +// +// getSubstituteColsRelExpr copies substituteCols before performing any +// modifications, so the original ColSet is not mutated. +func (c *CustomFuncs) getSubstituteColsRelExpr( + expr memo.RelExpr, substituteCols opt.ColSet, +) opt.ColSet { + // Remove any columns that are not in the output of this expression. + // Non-output columns can be in substituteCols after a recursive call + // into the input of an expression that either has multiple relational + // inputs (e.g. Joins) or can synthesize columns (e.g. Projects). + // + // Note that substituteCols is copied here, so subsequent mutations can be + // performed in place. + substituteCols = substituteCols.Intersection(expr.Relational().OutputCols) + + // Depending on the expression, further modifications to substituteCols + // may be necessary. + switch t := expr.(type) { + case *memo.SelectExpr: + // [MergeSelects] + // No restrictions on push-down for the cols in substituteCols. + case *memo.ProjectExpr, *memo.ProjectSetExpr: + // [PushSelectIntoProject] + // [PushSelectIntoProjectSet] + // Filter push-down candidates can only reference input columns. + inputCols := t.Child(0).(memo.RelExpr).Relational().OutputCols + substituteCols.IntersectionWith(inputCols) + case *memo.InnerJoinExpr, *memo.InnerJoinApplyExpr: + // [MergeSelectInnerJoin] + // [PushFilterIntoJoinLeft] + // [PushFilterIntoJoinRight] + // No restrictions on push-down for the cols in substituteCols. + case *memo.LeftJoinExpr, *memo.LeftJoinApplyExpr, *memo.SemiJoinExpr, + *memo.SemiJoinApplyExpr, *memo.AntiJoinExpr, *memo.AntiJoinApplyExpr: + // [PushSelectIntoJoinLeft] + // [PushSelectCondLeftIntoJoinLeftAndRight] + substituteCols = getSubstituteColsLeftSemiAntiJoin(t, substituteCols) + case *memo.GroupByExpr, *memo.DistinctOnExpr: + // [PushSelectIntoGroupBy] + // Filters must refer only to grouping and ConstAgg columns. + private := t.Private().(*memo.GroupingPrivate) + aggs := t.Child(1).(*memo.AggregationsExpr) + substituteCols.IntersectionWith(c.GroupingAndConstCols(private, *aggs)) + case *memo.UnionExpr, *memo.UnionAllExpr, *memo.IntersectExpr, + *memo.IntersectAllExpr, *memo.ExceptExpr, *memo.ExceptAllExpr: + // [PushFilterIntoSetOp] + substituteCols = getSubstituteColsSetOp(t, substituteCols) + default: + // Filter push-down through this expression is not supported. + substituteCols = opt.ColSet{} + } + return substituteCols +} + +func getSubstituteColsLeftSemiAntiJoin(join memo.RelExpr, substituteCols opt.ColSet) opt.ColSet { + // It is always valid to push an equality between an outer and non-outer + // left column into the left input of a LeftJoin, SemiJoin, or AntiJoin. If + // one of the join filters constrains that left column to be equal to a right + // column, it is also possible to remap and push the equality into the right + // input. See the PushSelectCondLeftIntoJoinLeftAndRight rule for more info. + // + // We can satisfy these requirements by first restricting substituteCols + // to left input columns, then extending it with right input columns + // that are held equivalent by the join filters. + left := join.Child(0).(memo.RelExpr) + on := join.Child(2).(*memo.FiltersExpr) + substituteCols.IntersectionWith(left.Relational().OutputCols) + for i := range *on { + // The usage of ComputeEquivClosureNoCopy is ok because + // getSubstituteColsRelExpr copies the set. + substituteCols = (*on)[i].ScalarProps().FuncDeps.ComputeEquivClosureNoCopy(substituteCols) + } + return substituteCols +} + +func getSubstituteColsSetOp(set memo.RelExpr, substituteCols opt.ColSet) opt.ColSet { + // Because TryRemapOuterCols is the equivalent of pushing down an + // equality filter between an input column and an outer column, we don't + // have to worry about composite sensitivity here (see CanMapOnSetOp). + // Map the output columns contained in substituteCols to the columns from + // both inputs. + var newSubstituteCols opt.ColSet + private := set.Private().(*memo.SetPrivate) + for i, outCol := range private.OutCols { + if substituteCols.Contains(outCol) { + newSubstituteCols.Add(private.LeftCols[i]) + newSubstituteCols.Add(private.RightCols[i]) + } + } + return newSubstituteCols +} diff --git a/pkg/sql/opt/norm/rules/decorrelate.opt b/pkg/sql/opt/norm/rules/decorrelate.opt index 71c24801b3ac..4b4f7fe95242 100644 --- a/pkg/sql/opt/norm/rules/decorrelate.opt +++ b/pkg/sql/opt/norm/rules/decorrelate.opt @@ -66,6 +66,67 @@ (EmptyJoinPrivate) ) +# TryRemapJoinOuterColsRight attempts to replace outer column references in the +# right input of a join with equivalent non-outer columns using the ON filters. +# It is valid to do this whenever it is possible to push the equality filter(s) +# down the tree until it holds true for the outer-column reference. Using the +# following query as an example: +# +# SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM ab WHERE a = x) ON b = x +# +# It is possible to push the 'b = x' filter down into the correlated right input +# of the join, which would allow replacing the 'a = x' filter with 'a = b', thus +# decorrelating the join. The match condition is similar to that for +# PushFilterIntoJoinRight because TryRemapJoinOuterColsRight simulates filter +# push-down when it makes the replacement. +# +# It is desirable to attempt to fire TryRemapJoinOuterColsRight before other +# decorrelation rules because it does not perform any transformations beyond the +# variable replacement. This prevents situations where decorrelation rules make +# the plan worse in their attempts to decorrelate the query. For example, +# decorrelation can pull filters up the operator tree or hoist subqueries into +# joins. This can cause plan changes which are difficult for the optimizer to +# reverse, and it won't even attempt to do so if the query isn't successfully +# decorrelated. +[TryRemapJoinOuterColsRight, Normalize] +(InnerJoin | InnerJoinApply | LeftJoin | LeftJoinApply | SemiJoin + | SemiJoinApply | AntiJoin | AntiJoinApply + $left:* + $right:* & (HasOuterCols $right) + $on:* & + (CanMaybeRemapOuterCols $right $on) & + (Let ($remapped $ok):(TryRemapOuterCols $right $on) $ok) + $private:* +) +=> +((OpName) $left $remapped $on $private) + +# TryRemapJoinOuterColsLeft is similar to TryRemapJoinOuterColsRight, but it +# applies to the left input of a join. +[TryRemapJoinOuterColsLeft, Normalize] +(InnerJoin | InnerJoinApply | SemiJoin | SemiJoinApply + $left:* & (HasOuterCols $left) + $right:* + $on:* & + (CanMaybeRemapOuterCols $left $on) & + (Let ($remapped $ok):(TryRemapOuterCols $left $on) $ok) + $private:* +) +=> +((OpName) $remapped $right $on $private) + +# TryRemapSelectOuterCols is similar to TryRemapJoinOuterColsRight, but it +# applies to the input of a Select. +[TryRemapSelectOuterCols, Normalize] +(Select + $input:* & (HasOuterCols $input) + $on:* & + (CanMaybeRemapOuterCols $input $on) & + (Let ($remapped $ok):(TryRemapOuterCols $input $on) $ok) +) +=> +(Select $remapped $on) + # TryDecorrelateSelect "pushes down" the join apply into the select operator, # in order to eliminate any correlation between the select filter list and the # left side of the join, and also to keep "digging" down to find and eliminate diff --git a/pkg/sql/opt/norm/testdata/rules/decorrelate b/pkg/sql/opt/norm/testdata/rules/decorrelate index e5dcd78f5178..f97ae5cc5332 100644 --- a/pkg/sql/opt/norm/testdata/rules/decorrelate +++ b/pkg/sql/opt/norm/testdata/rules/decorrelate @@ -1763,31 +1763,30 @@ group-by (hash) norm expect=TryDecorrelateGroupBy SELECT * FROM xy, uv -WHERE x=v AND u=(SELECT max(i) FROM a WHERE k=x) +WHERE x(2), (5)-->(1,2,6), (1)==(6), (6)==(1) + ├── key: (1,5) + ├── fd: (1)-->(2), (5)-->(6), (1,5)-->(2,6) └── select ├── columns: x:1!null y:2 u:5!null v:6!null max:16!null - ├── key: (5) - ├── fd: (1)-->(2), (5)-->(1,2,6), (1)==(6), (6)==(1), (5)==(16), (16)==(5) + ├── key: (1,5) + ├── fd: (1)-->(2), (5)-->(6), (1,5)-->(2,6,16), (5)==(16), (16)==(5) ├── group-by (hash) │ ├── columns: x:1!null y:2 u:5!null v:6!null max:16!null - │ ├── grouping columns: u:5!null - │ ├── key: (5) - │ ├── fd: (1)-->(2), (5)-->(1,2,6,16), (1)==(6), (6)==(1) + │ ├── grouping columns: x:1!null u:5!null + │ ├── key: (1,5) + │ ├── fd: (1)-->(2), (5)-->(6), (1,5)-->(2,6,16) │ ├── inner-join (hash) │ │ ├── columns: x:1!null y:2 u:5!null v:6!null k:9!null i:10!null │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ ├── key: (5) - │ │ ├── fd: (1)-->(2), (5)-->(6), (1)==(6,9), (6)==(1,9), (9)-->(10), (9)==(1,6) - │ │ ├── inner-join (hash) + │ │ ├── key: (5,9) + │ │ ├── fd: (1)-->(2), (5)-->(6), (9)-->(10), (1)==(9), (9)==(1) + │ │ ├── inner-join (cross) │ │ │ ├── columns: x:1!null y:2 u:5!null v:6!null - │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - │ │ │ ├── key: (5) - │ │ │ ├── fd: (1)-->(2), (5)-->(6), (1)==(6), (6)==(1) + │ │ │ ├── key: (1,5) + │ │ │ ├── fd: (1)-->(2), (5)-->(6) │ │ │ ├── scan xy │ │ │ │ ├── columns: x:1!null y:2 │ │ │ │ ├── key: (1) @@ -1797,7 +1796,7 @@ project │ │ │ │ ├── key: (5) │ │ │ │ └── fd: (5)-->(6) │ │ │ └── filters - │ │ │ └── x:1 = v:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ │ │ └── x:1 < v:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] │ │ ├── select │ │ │ ├── columns: k:9!null i:10!null │ │ │ ├── key: (9) @@ -1815,10 +1814,8 @@ project │ │ └── i:10 │ ├── const-agg [as=v:6, outer=(6)] │ │ └── v:6 - │ ├── const-agg [as=y:2, outer=(2)] - │ │ └── y:2 - │ └── const-agg [as=x:1, outer=(1)] - │ └── x:1 + │ └── const-agg [as=y:2, outer=(2)] + │ └── y:2 └── filters └── u:5 = max:16 [outer=(5,16), constraints=(/5: (/NULL - ]; /16: (/NULL - ]), fd=(5)==(16), (16)==(5)] @@ -1827,31 +1824,30 @@ project norm expect=TryDecorrelateGroupBy SELECT * FROM xy, uv -WHERE x=v AND (SELECT max(i) FROM a WHERE k=x) IS DISTINCT FROM u +WHERE x(2), (5)-->(1,2,6), (1)==(6), (6)==(1) + ├── key: (1,5) + ├── fd: (1)-->(2), (5)-->(6), (1,5)-->(2,6) └── select ├── columns: x:1!null y:2 u:5!null v:6!null max:16 - ├── key: (5) - ├── fd: (1)-->(2), (5)-->(1,2,6,16), (1)==(6), (6)==(1) + ├── key: (1,5) + ├── fd: (1)-->(2), (5)-->(6), (1,5)-->(2,6,16) ├── group-by (hash) │ ├── columns: x:1!null y:2 u:5!null v:6!null max:16 - │ ├── grouping columns: u:5!null - │ ├── key: (5) - │ ├── fd: (1)-->(2), (5)-->(1,2,6,16), (1)==(6), (6)==(1) + │ ├── grouping columns: x:1!null u:5!null + │ ├── key: (1,5) + │ ├── fd: (1)-->(2), (5)-->(6), (1,5)-->(2,6,16) │ ├── left-join (hash) │ │ ├── columns: x:1!null y:2 u:5!null v:6!null k:9 i:10 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ ├── key: (5) - │ │ ├── fd: (1)-->(2), (5)-->(6,9,10), (1)==(6), (6)==(1), (9)-->(10) - │ │ ├── inner-join (hash) + │ │ ├── key: (1,5) + │ │ ├── fd: (1)-->(2), (5)-->(6), (9)-->(10), (1,5)-->(9,10) + │ │ ├── inner-join (cross) │ │ │ ├── columns: x:1!null y:2 u:5!null v:6!null - │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - │ │ │ ├── key: (5) - │ │ │ ├── fd: (1)-->(2), (5)-->(6), (1)==(6), (6)==(1) + │ │ │ ├── key: (1,5) + │ │ │ ├── fd: (1)-->(2), (5)-->(6) │ │ │ ├── scan xy │ │ │ │ ├── columns: x:1!null y:2 │ │ │ │ ├── key: (1) @@ -1861,7 +1857,7 @@ project │ │ │ │ ├── key: (5) │ │ │ │ └── fd: (5)-->(6) │ │ │ └── filters - │ │ │ └── x:1 = v:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ │ │ └── x:1 < v:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] │ │ ├── scan a │ │ │ ├── columns: k:9!null i:10 │ │ │ ├── key: (9) @@ -1873,10 +1869,8 @@ project │ │ └── i:10 │ ├── const-agg [as=v:6, outer=(6)] │ │ └── v:6 - │ ├── const-agg [as=y:2, outer=(2)] - │ │ └── y:2 - │ └── const-agg [as=x:1, outer=(1)] - │ └── x:1 + │ └── const-agg [as=y:2, outer=(2)] + │ └── y:2 └── filters └── u:5 IS DISTINCT FROM max:16 [outer=(5,16)] @@ -6035,3 +6029,593 @@ project │ └── xy.x:8 └── projections └── xy.x:8 [as=x:12, outer=(8)] + +# -------------------------------------------------- +# TryRemapJoinOuterColsRight +# -------------------------------------------------- + +# Case with Select. Matched join is an InnerJoin. +norm expect=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM ab WHERE b = x) ON a = x +---- +inner-join (hash) + ├── scan xy + ├── select + │ ├── scan ab + │ └── filters + │ └── b = a + └── filters + └── a = x + +# Case with Select. Matched join is a LeftJoin. +norm expect=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy LEFT JOIN LATERAL (SELECT * FROM ab WHERE b = x) ON a = x +---- +left-join (hash) + ├── scan xy + ├── select + │ ├── scan ab + │ └── filters + │ └── b = a + └── filters + └── a = x + +# Case with Project. +norm expect=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT a, a+x FROM ab) ON a = x +---- +inner-join (hash) + ├── scan xy + ├── project + │ ├── scan ab + │ └── projections + │ └── a + a + └── filters + └── a = x + +# Case with ProjectSet. +norm expect=TryRemapJoinOuterColsRight disable=ConvertZipArraysToValues format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT a, unnest(ARRAY[x, b]) FROM ab) ON a = x +---- +project + └── inner-join (hash) + ├── scan xy + ├── project-set + │ ├── scan ab + │ └── zip + │ └── unnest(ARRAY[a, b]) + └── filters + └── a = x + +# Case with an InnerJoin. +norm expect=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM (SELECT a, b, b+x FROM ab) INNER JOIN uv ON b = u) ON a = x +---- +inner-join (hash) + ├── scan xy + ├── inner-join (hash) + │ ├── project + │ │ ├── scan ab + │ │ └── projections + │ │ └── b + a + │ ├── scan uv + │ └── filters + │ └── b = u + └── filters + └── a = x + +# Case with an InnerJoin. The outer column is in the join filters. +norm expect=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM ab INNER JOIN uv ON b = u OR b = x) ON a = x +---- +inner-join (hash) + ├── scan xy + ├── inner-join (cross) + │ ├── scan ab + │ ├── scan uv + │ └── filters + │ └── (b = u) OR (b = a) + └── filters + └── a = x + +# Case with a LeftJoin. +norm expect=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM (SELECT a, b, b+x FROM ab) LEFT JOIN uv ON b = u) ON a = x +---- +inner-join (hash) + ├── scan xy + ├── left-join (hash) + │ ├── project + │ │ ├── scan ab + │ │ └── projections + │ │ └── b + a + │ ├── scan uv + │ └── filters + │ └── b = u + └── filters + └── a = x + +# Case with a LeftJoin. The outer column is in the join filters. +norm expect=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM ab LEFT JOIN uv ON b = u OR b = x) ON a = x +---- +inner-join (hash) + ├── scan xy + ├── left-join (cross) + │ ├── scan ab + │ ├── scan uv + │ └── filters + │ └── (b = u) OR (b = a) + └── filters + └── a = x + +# Case with a LeftJoin. The filter can be mapped to the right input. +norm expect=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM ab LEFT JOIN (SELECT u, v + x FROM uv) ON a = u) ON a = x +---- +inner-join (hash) + ├── scan xy + ├── left-join (hash) + │ ├── scan ab + │ ├── project + │ │ ├── scan uv + │ │ └── projections + │ │ └── v + u + │ └── filters + │ └── a = u + └── filters + └── a = x + +# Case with a LeftJoin. Both inputs have an outer column reference that can be +# remapped. +norm expect=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL ( + SELECT * FROM (SELECT a, b, b+x FROM ab) LEFT JOIN (SELECT u, v + x FROM uv) ON a = u +) ON a = x +---- +inner-join (hash) + ├── scan xy + ├── left-join (hash) + │ ├── project + │ │ ├── scan ab + │ │ └── projections + │ │ └── b + a + │ ├── project + │ │ ├── scan uv + │ │ └── projections + │ │ └── v + u + │ └── filters + │ └── a = u + └── filters + └── a = x + +# Case with a SemiJoin. +norm expect=TryRemapJoinOuterColsRight disable=TryDecorrelateSelect format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM (SELECT a, b+x AS c FROM ab) WHERE EXISTS (SELECT * FROM uv WHERE c = u)) ON a = x +---- +inner-join (hash) + ├── scan xy + ├── semi-join-apply + │ ├── project + │ │ ├── scan ab + │ │ └── projections + │ │ └── b + a + │ ├── select + │ │ ├── scan uv + │ │ └── filters + │ │ └── c = u + │ └── filters (true) + └── filters + └── a = x + +# Case with a SemiJoin. The outer column is in the join filters. +norm expect=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM ab WHERE EXISTS (SELECT * FROM uv WHERE a = u OR v = x)) ON a = x +---- +inner-join (hash) + ├── scan xy + ├── semi-join (cross) + │ ├── scan ab + │ ├── scan uv + │ └── filters + │ └── (a = u) OR (v = a) + └── filters + └── a = x + +# Case with an AntiJoin. +norm expect=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM (SELECT a, b+x AS c FROM ab) WHERE NOT EXISTS (SELECT * FROM uv WHERE c = u)) ON a = x +---- +inner-join (hash) + ├── scan xy + ├── anti-join (hash) + │ ├── project + │ │ ├── scan ab + │ │ └── projections + │ │ └── b + a + │ ├── scan uv + │ └── filters + │ └── c = u + └── filters + └── a = x + +# Case with an AntiJoin. The outer column is in the join filters. +norm expect=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM ab WHERE NOT EXISTS (SELECT * FROM uv WHERE a = u OR v = x)) ON a = x +---- +inner-join (hash) + ├── scan xy + ├── anti-join (cross) + │ ├── scan ab + │ ├── scan uv + │ └── filters + │ └── (a = u) OR (v = a) + └── filters + └── a = x + +# Case with a GroupBy. Equality references the grouping column. +norm expect=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT a, corr(b, x) FROM ab GROUP BY a) ON a = x +---- +inner-join (hash) + ├── scan xy + ├── group-by (hash) + │ ├── project + │ │ ├── scan ab + │ │ └── projections + │ │ └── a + │ └── aggregations + │ └── corr + │ ├── b + │ └── x + └── filters + └── a = xy.x + +# Case with a GroupBy. Equality references a ConstAgg column. +norm expect=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT v, corr(u, x) FROM uv GROUP BY u, v) ON v = x +---- +project + └── inner-join (hash) + ├── scan xy + ├── group-by (hash) + │ ├── project + │ │ ├── scan uv + │ │ └── projections + │ │ └── v + │ └── aggregations + │ ├── corr + │ │ ├── u + │ │ └── x + │ └── const-agg + │ └── v + └── filters + └── v = xy.x + +# Case with a DistinctOn. +norm expect=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT DISTINCT ON (a) * FROM (SELECT *, b+x FROM ab)) ON a = x +---- +inner-join (hash) + ├── scan xy + ├── distinct-on + │ ├── project + │ │ ├── scan ab + │ │ └── projections + │ │ └── b + a + │ └── aggregations + │ ├── first-agg + │ │ └── b + │ └── first-agg + │ └── "?column?" + └── filters + └── a = x + +# Case with a Union. The alias here is not necessary, but makes it more explicit +# that the "a = x" filter refers to the first output column of the Union, rather +# than just the left input. +norm expect=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM ab UNION (SELECT u, v+x FROM uv)) foo(a, b) ON foo.a = x +---- +inner-join (hash) + ├── scan xy + ├── union + │ ├── scan ab + │ └── project + │ ├── scan uv + │ └── projections + │ └── v + u + └── filters + └── a = x + +# Case with a Union. Both inputs should be remapped. +norm expect=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM (SELECT a, b+x FROM ab) UNION (SELECT u, v+x FROM uv)) foo(a, b) ON foo.a = x +---- +inner-join (hash) + ├── scan xy + ├── union + │ ├── project + │ │ ├── scan ab + │ │ └── projections + │ │ └── b + ab.a + │ └── project + │ ├── scan uv + │ └── projections + │ └── v + u + └── filters + └── a = x + +# Case with an Intersect. +norm expect=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM ab INTERSECT (SELECT u, v+x FROM uv)) foo(a, b) ON foo.a = x +---- +inner-join (hash) + ├── scan xy + ├── intersect-all + │ ├── scan ab + │ └── project + │ ├── scan uv + │ └── projections + │ └── v + u + └── filters + └── a = x + +# Case with an Intersect. Both inputs should be remapped. +norm format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM (SELECT a, b+x FROM ab) INTERSECT (SELECT u, v+x FROM uv)) foo(a, b) ON foo.a = x +---- +inner-join (hash) + ├── scan xy + ├── intersect-all + │ ├── project + │ │ ├── scan ab + │ │ └── projections + │ │ └── b + a + │ └── project + │ ├── scan uv + │ └── projections + │ └── v + u + └── filters + └── a = x + +# Case with an Except. +norm expect=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM ab EXCEPT (SELECT u, v+x FROM uv)) foo(a, b) ON foo.a = x +---- +inner-join (hash) + ├── scan xy + ├── except + │ ├── scan ab + │ └── project + │ ├── scan uv + │ └── projections + │ └── v + u + └── filters + └── a = x + +# Case with an Except. Both inputs should be remapped. +norm format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM (SELECT a, b+x FROM ab) EXCEPT (SELECT u, v+x FROM uv)) foo(a, b) ON foo.a = x +---- +inner-join (hash) + ├── scan xy + ├── except + │ ├── project + │ │ ├── scan ab + │ │ └── projections + │ │ └── b + a + │ └── project + │ ├── scan uv + │ └── projections + │ └── v + u + └── filters + └── a = x + +# No-op case with a LeftJoin. The filter cannot be mapped to the right input, +# because there is no suitable equality between left and right columns. +norm expect-not=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM ab LEFT JOIN (SELECT u, v + x FROM uv) ON a <> u) ON a = x +---- +left-join-apply + ├── inner-join (hash) + │ ├── scan xy + │ ├── scan ab + │ └── filters + │ └── a = x + ├── project + │ ├── scan uv + │ └── projections + │ └── v + x + └── filters + └── a != u + +# No-op case with FullJoin. +norm expect-not=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM (SELECT a, b, b+x FROM ab) FULL JOIN uv ON a = x) ON True +---- +inner-join-apply + ├── scan xy + ├── full-join (cross) + │ ├── project + │ │ ├── scan ab + │ │ └── projections + │ │ └── b + x + │ ├── scan uv + │ └── filters + │ └── a = x + └── filters (true) + +# No-op case because there is no outer-non-outer column equality. +norm expect-not=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM ab WHERE b = x) ON a <> x +---- +inner-join (hash) + ├── scan xy + ├── select + │ ├── scan ab + │ └── filters + │ └── a != b + └── filters + └── b = x + +# No-op case because the equality is a disjunct. +norm expect-not=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM ab WHERE b = x) ON a = x OR x IS NULL +---- +inner-join (hash) + ├── scan xy + ├── select + │ ├── scan ab + │ └── filters + │ └── (a = b) OR (b IS NULL) + └── filters + └── b = x + +# No-op case because the outer column is equal to a synthesized column, so push +# down is not possible below the Project. +norm expect-not=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT *, a+b AS c FROM ab WHERE b = x) ON c = x +---- +project + ├── inner-join (hash) + │ ├── scan xy + │ ├── select + │ │ ├── scan ab + │ │ └── filters + │ │ └── b = (a + b) + │ └── filters + │ └── b = x + └── projections + └── a + b + +# No-op case because the outer column is equal to a synthesized column, so push +# down is not possible below the GroupBy. +norm expect-not=TryRemapJoinOuterColsRight format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT v, corr(u, v) AS w FROM (SELECT *, u + x FROM uv) GROUP BY u, v) ON w = x +---- +project + └── inner-join (cross) + ├── scan xy + ├── group-by (hash) + │ ├── scan uv + │ └── aggregations + │ ├── corr + │ │ ├── u + │ │ └── v + │ └── const-agg + │ └── v + └── filters + └── corr = x + +# -------------------------------------------------- +# TryRemapJoinOuterColsLeft +# -------------------------------------------------- + +# Case with InnerJoin. +norm expect=TryRemapJoinOuterColsLeft format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM (SELECT * FROM ab WHERE b = x) INNER JOIN uv ON a = x) ON True +---- +inner-join (cross) + ├── inner-join (hash) + │ ├── scan xy + │ ├── select + │ │ ├── scan ab + │ │ └── filters + │ │ └── b = a + │ └── filters + │ └── a = x + ├── scan uv + └── filters (true) + +# Case with SemiJoin. +norm expect=TryRemapJoinOuterColsLeft format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM (SELECT *, b+x FROM ab) WHERE EXISTS (SELECT * FROM uv WHERE a = x)) ON True +---- +inner-join-apply + ├── scan xy + ├── semi-join (cross) + │ ├── project + │ │ ├── scan ab + │ │ └── projections + │ │ └── b + a + │ ├── scan uv + │ └── filters + │ └── a = x + └── filters (true) + +# No-op case with LeftJoin. +norm expect-not=TryRemapJoinOuterColsLeft format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM (SELECT * FROM ab WHERE b = x) LEFT JOIN uv ON a = x) ON True +---- +left-join (cross) + ├── inner-join (hash) + │ ├── scan xy + │ ├── scan ab + │ └── filters + │ └── b = x + ├── scan uv + └── filters + └── a = x + +# No-op case with AntiJoin. +norm expect-not=TryRemapJoinOuterColsLeft format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM (SELECT *, b+x FROM ab) WHERE NOT EXISTS (SELECT * FROM uv WHERE a = x)) ON True +---- +inner-join-apply + ├── scan xy + ├── anti-join (cross) + │ ├── project + │ │ ├── scan ab + │ │ └── projections + │ │ └── b + x + │ ├── scan uv + │ └── filters + │ └── a = x + └── filters (true) + +# No-op case with FullJoin. +norm expect-not=TryRemapJoinOuterColsLeft format=hide-all +SELECT * FROM xy INNER JOIN LATERAL (SELECT * FROM (SELECT * FROM ab WHERE b = x) FULL JOIN uv ON a = x) ON True +---- +inner-join-apply + ├── scan xy + ├── full-join (cross) + │ ├── select + │ │ ├── scan ab + │ │ └── filters + │ │ └── b = x + │ ├── scan uv + │ └── filters + │ └── a = x + └── filters (true) + +# -------------------------------------------------- +# TryRemapSelectOuterCols +# -------------------------------------------------- + +norm expect=TryRemapSelectOuterCols format=hide-all +SELECT * FROM xy LEFT JOIN LATERAL (SELECT * FROM (SELECT *, b+x FROM ab) WHERE a = x) ON True +---- +left-join (hash) + ├── scan xy + ├── project + │ ├── scan ab + │ └── projections + │ └── b + a + └── filters + └── a = x + +# Case where the replacement leads to an "a = a" filter, which in turn gets +# simplified into "a IS DISTINCT FROM NULL". +norm expect=TryRemapSelectOuterCols format=hide-all +SELECT * FROM xy WHERE x IN (SELECT a FROM ab WHERE a = x); +---- +semi-join (hash) + ├── scan xy + ├── select + │ ├── scan ab + │ └── filters + │ └── a IS DISTINCT FROM CAST(NULL AS INT8) + └── filters + └── x = a diff --git a/pkg/sql/opt/norm/testdata/rules/prune_cols b/pkg/sql/opt/norm/testdata/rules/prune_cols index d8391d3bbeb9..41a7f655458d 100644 --- a/pkg/sql/opt/norm/testdata/rules/prune_cols +++ b/pkg/sql/opt/norm/testdata/rules/prune_cols @@ -1008,7 +1008,7 @@ project └── i:2 < y:12 [outer=(2,12), constraints=(/2: (/NULL - ]; /12: (/NULL - ])] # ApplyJoin operator. -norm expect=PruneJoinLeftCols +norm expect=PruneJoinLeftCols disable=TryRemapJoinOuterColsRight SELECT k, i FROM a WHERE (SELECT k+1 AS r FROM xy WHERE y=k) = 1 @@ -4899,7 +4899,7 @@ anti-join (hash) └── abcde.a:1 = "family".a:8 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] # Test using multi-level nesting so we don't decorrelate the semi-join. -norm expect=PruneSemiAntiJoinRightCols +norm expect=PruneSemiAntiJoinRightCols disable=(TryRemapJoinOuterColsRight,TryRemapSelectOuterCols) SELECT a, b, c FROM @@ -4938,7 +4938,7 @@ semi-join-apply └── abcde.a:1 = "family".a:8 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] # Test using multi-level nesting so we don't decorrelate the anti-join. -norm expect=PruneSemiAntiJoinRightCols +norm expect=PruneSemiAntiJoinRightCols disable=(TryRemapJoinOuterColsRight,TryRemapSelectOuterCols) SELECT a, b, c FROM diff --git a/pkg/sql/opt/xform/testdata/external/hibernate b/pkg/sql/opt/xform/testdata/external/hibernate index 29da1eb9ff0c..133ed327f9e8 100644 --- a/pkg/sql/opt/xform/testdata/external/hibernate +++ b/pkg/sql/opt/xform/testdata/external/hibernate @@ -1211,30 +1211,30 @@ where project ├── columns: id1_2_:1!null address2_2_:2 createdo3_2_:3 name4_2_:4 nickname5_2_:5 version6_2_:6!null ├── fd: (1)-->(2-6) - └── select - ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null phones2_.id:9!null phones2_.order_id:13!null max:23!null + └── inner-join (lookup person [as=person0_]) + ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null phones2_.id:9!null phones2_.person_id:12!null phones2_.order_id:13!null max:23!null + ├── key columns: [12] = [1] + ├── lookup columns are key ├── key: (9) - ├── fd: (1)-->(2-6), (9)-->(1-6,13,23), (13)==(23), (23)==(13) - ├── group-by (hash) - │ ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null phones2_.id:9!null phones2_.order_id:13 max:23!null - │ ├── grouping columns: phones2_.id:9!null + ├── fd: (1)-->(2-6), (9)-->(12,13,23), (13)==(23), (23)==(13), (1)==(12), (12)==(1) + ├── select + │ ├── columns: phones2_.id:9!null phones2_.person_id:12!null phones2_.order_id:13!null max:23!null │ ├── key: (9) - │ ├── fd: (1)-->(2-6), (9)-->(1-6,13,23) - │ ├── inner-join (hash) - │ │ ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null phones2_.id:9!null phones2_.phone_type:11!null phones2_.person_id:12!null phones2_.order_id:13 phones1_.person_id:19!null phones1_.order_id:20!null - │ │ ├── fd: ()-->(11), (1)-->(2-6), (9)-->(12,13), (1)==(12,19), (12)==(1,19), (19)==(1,12) - │ │ ├── select - │ │ │ ├── columns: phones1_.person_id:19 phones1_.order_id:20!null - │ │ │ ├── scan phone [as=phones1_] - │ │ │ │ └── columns: phones1_.person_id:19 phones1_.order_id:20 - │ │ │ └── filters - │ │ │ └── phones1_.order_id:20 IS NOT NULL [outer=(20), constraints=(/20: (/NULL - ]; tight)] - │ │ ├── inner-join (lookup person [as=person0_]) - │ │ │ ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null phones2_.id:9!null phones2_.phone_type:11!null phones2_.person_id:12!null phones2_.order_id:13 - │ │ │ ├── key columns: [12] = [1] - │ │ │ ├── lookup columns are key - │ │ │ ├── key: (9) - │ │ │ ├── fd: ()-->(11), (1)-->(2-6), (9)-->(12,13), (1)==(12), (12)==(1) + │ ├── fd: (9)-->(12,13,23), (13)==(23), (23)==(13) + │ ├── group-by (hash) + │ │ ├── columns: phones2_.id:9!null phones2_.person_id:12!null phones2_.order_id:13 max:23!null + │ │ ├── grouping columns: phones2_.id:9!null + │ │ ├── key: (9) + │ │ ├── fd: (9)-->(12,13,23) + │ │ ├── inner-join (hash) + │ │ │ ├── columns: phones2_.id:9!null phones2_.phone_type:11!null phones2_.person_id:12!null phones2_.order_id:13 phones1_.person_id:19!null phones1_.order_id:20!null + │ │ │ ├── fd: ()-->(11), (9)-->(12,13), (12)==(19), (19)==(12) + │ │ │ ├── select + │ │ │ │ ├── columns: phones1_.person_id:19 phones1_.order_id:20!null + │ │ │ │ ├── scan phone [as=phones1_] + │ │ │ │ │ └── columns: phones1_.person_id:19 phones1_.order_id:20 + │ │ │ │ └── filters + │ │ │ │ └── phones1_.order_id:20 IS NOT NULL [outer=(20), constraints=(/20: (/NULL - ]; tight)] │ │ │ ├── select │ │ │ │ ├── columns: phones2_.id:9!null phones2_.phone_type:11!null phones2_.person_id:12 phones2_.order_id:13 │ │ │ │ ├── key: (9) @@ -1245,28 +1245,18 @@ project │ │ │ │ │ └── fd: (9)-->(11-13) │ │ │ │ └── filters │ │ │ │ └── phones2_.phone_type:11 = 'LAND_LINE' [outer=(11), constraints=(/11: [/'LAND_LINE' - /'LAND_LINE']; tight), fd=()-->(11)] - │ │ │ └── filters (true) - │ │ └── filters - │ │ └── person0_.id:1 = phones1_.person_id:19 [outer=(1,19), constraints=(/1: (/NULL - ]; /19: (/NULL - ]), fd=(1)==(19), (19)==(1)] - │ └── aggregations - │ ├── max [as=max:23, outer=(20)] - │ │ └── phones1_.order_id:20 - │ ├── const-agg [as=phones2_.order_id:13, outer=(13)] - │ │ └── phones2_.order_id:13 - │ ├── const-agg [as=address:2, outer=(2)] - │ │ └── address:2 - │ ├── const-agg [as=createdon:3, outer=(3)] - │ │ └── createdon:3 - │ ├── const-agg [as=name:4, outer=(4)] - │ │ └── name:4 - │ ├── const-agg [as=nickname:5, outer=(5)] - │ │ └── nickname:5 - │ ├── const-agg [as=version:6, outer=(6)] - │ │ └── version:6 - │ └── const-agg [as=person0_.id:1, outer=(1)] - │ └── person0_.id:1 - └── filters - └── phones2_.order_id:13 = max:23 [outer=(13,23), constraints=(/13: (/NULL - ]; /23: (/NULL - ]), fd=(13)==(23), (23)==(13)] + │ │ │ └── filters + │ │ │ └── phones2_.person_id:12 = phones1_.person_id:19 [outer=(12,19), constraints=(/12: (/NULL - ]; /19: (/NULL - ]), fd=(12)==(19), (19)==(12)] + │ │ └── aggregations + │ │ ├── max [as=max:23, outer=(20)] + │ │ │ └── phones1_.order_id:20 + │ │ ├── const-agg [as=phones2_.person_id:12, outer=(12)] + │ │ │ └── phones2_.person_id:12 + │ │ └── const-agg [as=phones2_.order_id:13, outer=(13)] + │ │ └── phones2_.order_id:13 + │ └── filters + │ └── phones2_.order_id:13 = max:23 [outer=(13,23), constraints=(/13: (/NULL - ]; /23: (/NULL - ]), fd=(13)==(23), (23)==(13)] + └── filters (true) opt select diff --git a/pkg/sql/opt/xform/testdata/external/tpch b/pkg/sql/opt/xform/testdata/external/tpch index 393c6d7a0ec9..08ae89801b81 100644 --- a/pkg/sql/opt/xform/testdata/external/tpch +++ b/pkg/sql/opt/xform/testdata/external/tpch @@ -147,137 +147,140 @@ project ├── cardinality: [0 - 100] ├── fd: (1)-->(3) ├── ordering: -17,+29,+13,+1 - └── top-k - ├── columns: p_partkey:1!null p_mfgr:3!null s_name:13!null s_address:14!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_name:29!null min:66!null - ├── internal-ordering: -17,+29,+13,+(1|21) - ├── k: 100 + └── limit + ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_name:13!null s_address:14!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_name:29!null min:66!null + ├── internal-ordering: -17,+29,+13,+(1|21) opt(6) ├── cardinality: [0 - 100] ├── key: (21,22) - ├── fd: (1)-->(3), (21,22)-->(1,3,13,14,16-18,24,29,66), (1)==(21), (21)==(1), (22)-->(13,14,16-18,29), (24)==(66), (66)==(24) - ├── ordering: -17,+29,+13,+(1|21) [actual: -17,+29,+13,+1] - └── select - ├── columns: p_partkey:1!null p_mfgr:3!null s_name:13!null s_address:14!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_name:29!null min:66!null - ├── key: (21,22) - ├── fd: (1)-->(3), (21,22)-->(1,3,13,14,16-18,24,29,66), (1)==(21), (21)==(1), (22)-->(13,14,16-18,29), (24)==(66), (66)==(24) - ├── group-by (hash) - │ ├── columns: p_partkey:1!null p_mfgr:3!null s_name:13!null s_address:14!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_name:29!null min:66!null - │ ├── grouping columns: ps_partkey:21!null ps_suppkey:22!null - │ ├── key: (21,22) - │ ├── fd: (1)-->(3), (21,22)-->(1,3,13,14,16-18,24,29,66), (1)==(21), (21)==(1), (22)-->(13,14,16-18,29) - │ ├── inner-join (hash) - │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:12!null s_name:13!null s_address:14!null s_nationkey:15!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_nationkey:28!null n_name:29!null n_regionkey:30!null r_regionkey:34!null r_name:35!null ps_partkey:39!null ps_suppkey:40!null ps_supplycost:42!null s_suppkey:46!null s_nationkey:49!null n_nationkey:55!null n_regionkey:57!null r_regionkey:61!null r_name:62!null - │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ ├── key: (22,39,46) - │ │ ├── fd: ()-->(6,35,62), (1)-->(3,5), (12)-->(13-18), (21,22)-->(24), (12)==(22), (22)==(12), (28)-->(29,30), (30)==(34), (34)==(30), (15)==(28), (28)==(15), (1)==(21,39), (21)==(1,39), (39,40)-->(42), (46)-->(49), (55)-->(57), (57)==(61), (61)==(57), (49)==(55), (55)==(49), (40)==(46), (46)==(40), (39)==(1,21) - │ │ ├── inner-join (lookup partsupp) - │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:12!null s_name:13!null s_address:14!null s_nationkey:15!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_nationkey:28!null n_name:29!null n_regionkey:30!null r_regionkey:34!null r_name:35!null ps_partkey:39!null ps_suppkey:40!null ps_supplycost:42!null - │ │ │ ├── key columns: [1] = [39] - │ │ │ ├── key: (22,39,40) - │ │ │ ├── fd: ()-->(6,35), (1)-->(3,5), (28)-->(29,30), (12)-->(13-18), (21,22)-->(24), (39,40)-->(42), (21)==(1,39), (39)==(1,21), (12)==(22), (22)==(12), (15)==(28), (28)==(15), (30)==(34), (34)==(30), (1)==(21,39) - │ │ │ ├── inner-join (hash) - │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:12!null s_name:13!null s_address:14!null s_nationkey:15!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_nationkey:28!null n_name:29!null n_regionkey:30!null r_regionkey:34!null r_name:35!null - │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ ├── key: (21,22) - │ │ │ │ ├── fd: ()-->(6,35), (1)-->(3,5), (12)-->(13-18), (21,22)-->(24), (12)==(22), (22)==(12), (28)-->(29,30), (30)==(34), (34)==(30), (15)==(28), (28)==(15), (1)==(21), (21)==(1) - │ │ │ │ ├── inner-join (lookup partsupp) - │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null - │ │ │ │ │ ├── key columns: [1] = [21] - │ │ │ │ │ ├── key: (21,22) - │ │ │ │ │ ├── fd: ()-->(6), (1)-->(3,5), (21,22)-->(24), (1)==(21), (21)==(1) - │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null - │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ ├── fd: ()-->(6), (1)-->(3,5) - │ │ │ │ │ │ ├── scan part - │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null - │ │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ │ └── fd: (1)-->(3,5,6) - │ │ │ │ │ │ └── filters - │ │ │ │ │ │ ├── p_size:6 = 15 [outer=(6), constraints=(/6: [/15 - /15]; tight), fd=()-->(6)] - │ │ │ │ │ │ └── p_type:5 LIKE '%BRASS' [outer=(5), constraints=(/5: (/NULL - ])] - │ │ │ │ │ └── filters (true) - │ │ │ │ ├── inner-join (hash) - │ │ │ │ │ ├── columns: s_suppkey:12!null s_name:13!null s_address:14!null s_nationkey:15!null s_phone:16!null s_acctbal:17!null s_comment:18!null n_nationkey:28!null n_name:29!null n_regionkey:30!null r_regionkey:34!null r_name:35!null - │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ │ ├── key: (12) - │ │ │ │ │ ├── fd: ()-->(35), (12)-->(13-18), (28)-->(29,30), (30)==(34), (34)==(30), (15)==(28), (28)==(15) - │ │ │ │ │ ├── scan supplier - │ │ │ │ │ │ ├── columns: s_suppkey:12!null s_name:13!null s_address:14!null s_nationkey:15!null s_phone:16!null s_acctbal:17!null s_comment:18!null - │ │ │ │ │ │ ├── key: (12) - │ │ │ │ │ │ └── fd: (12)-->(13-18) - │ │ │ │ │ ├── inner-join (hash) - │ │ │ │ │ │ ├── columns: n_nationkey:28!null n_name:29!null n_regionkey:30!null r_regionkey:34!null r_name:35!null - │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ │ │ ├── key: (28) - │ │ │ │ │ │ ├── fd: ()-->(35), (28)-->(29,30), (30)==(34), (34)==(30) - │ │ │ │ │ │ ├── scan nation - │ │ │ │ │ │ │ ├── columns: n_nationkey:28!null n_name:29!null n_regionkey:30!null - │ │ │ │ │ │ │ ├── key: (28) - │ │ │ │ │ │ │ └── fd: (28)-->(29,30) - │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: r_regionkey:34!null r_name:35!null - │ │ │ │ │ │ │ ├── key: (34) - │ │ │ │ │ │ │ ├── fd: ()-->(35) - │ │ │ │ │ │ │ ├── scan region - │ │ │ │ │ │ │ │ ├── columns: r_regionkey:34!null r_name:35!null - │ │ │ │ │ │ │ │ ├── key: (34) - │ │ │ │ │ │ │ │ └── fd: (34)-->(35) - │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── r_name:35 = 'EUROPE' [outer=(35), constraints=(/35: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(35)] - │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── n_regionkey:30 = r_regionkey:34 [outer=(30,34), constraints=(/30: (/NULL - ]; /34: (/NULL - ]), fd=(30)==(34), (34)==(30)] - │ │ │ │ │ └── filters - │ │ │ │ │ └── s_nationkey:15 = n_nationkey:28 [outer=(15,28), constraints=(/15: (/NULL - ]; /28: (/NULL - ]), fd=(15)==(28), (28)==(15)] - │ │ │ │ └── filters - │ │ │ │ └── s_suppkey:12 = ps_suppkey:22 [outer=(12,22), constraints=(/12: (/NULL - ]; /22: (/NULL - ]), fd=(12)==(22), (22)==(12)] - │ │ │ └── filters (true) - │ │ ├── inner-join (lookup supplier@s_nk) - │ │ │ ├── columns: s_suppkey:46!null s_nationkey:49!null n_nationkey:55!null n_regionkey:57!null r_regionkey:61!null r_name:62!null - │ │ │ ├── key columns: [55] = [49] - │ │ │ ├── key: (46) - │ │ │ ├── fd: ()-->(62), (46)-->(49), (55)-->(57), (57)==(61), (61)==(57), (49)==(55), (55)==(49) - │ │ │ ├── inner-join (lookup nation@n_rk) - │ │ │ │ ├── columns: n_nationkey:55!null n_regionkey:57!null r_regionkey:61!null r_name:62!null - │ │ │ │ ├── key columns: [61] = [57] - │ │ │ │ ├── key: (55) - │ │ │ │ ├── fd: ()-->(62), (55)-->(57), (57)==(61), (61)==(57) - │ │ │ │ ├── select - │ │ │ │ │ ├── columns: r_regionkey:61!null r_name:62!null - │ │ │ │ │ ├── key: (61) - │ │ │ │ │ ├── fd: ()-->(62) - │ │ │ │ │ ├── scan region - │ │ │ │ │ │ ├── columns: r_regionkey:61!null r_name:62!null - │ │ │ │ │ │ ├── key: (61) - │ │ │ │ │ │ └── fd: (61)-->(62) - │ │ │ │ │ └── filters - │ │ │ │ │ └── r_name:62 = 'EUROPE' [outer=(62), constraints=(/62: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(62)] - │ │ │ │ └── filters (true) - │ │ │ └── filters (true) - │ │ └── filters - │ │ └── s_suppkey:46 = ps_suppkey:40 [outer=(40,46), constraints=(/40: (/NULL - ]; /46: (/NULL - ]), fd=(40)==(46), (46)==(40)] - │ └── aggregations - │ ├── min [as=min:66, outer=(42)] - │ │ └── ps_supplycost:42 - │ ├── const-agg [as=s_name:13, outer=(13)] - │ │ └── s_name:13 - │ ├── const-agg [as=s_address:14, outer=(14)] - │ │ └── s_address:14 - │ ├── const-agg [as=s_phone:16, outer=(16)] - │ │ └── s_phone:16 - │ ├── const-agg [as=s_acctbal:17, outer=(17)] - │ │ └── s_acctbal:17 - │ ├── const-agg [as=s_comment:18, outer=(18)] - │ │ └── s_comment:18 - │ ├── const-agg [as=ps_supplycost:24, outer=(24)] - │ │ └── ps_supplycost:24 - │ ├── const-agg [as=n_name:29, outer=(29)] - │ │ └── n_name:29 - │ ├── const-agg [as=p_mfgr:3, outer=(3)] - │ │ └── p_mfgr:3 - │ └── const-agg [as=p_partkey:1, outer=(1)] - │ └── p_partkey:1 - └── filters - └── ps_supplycost:24 = min:66 [outer=(24,66), constraints=(/24: (/NULL - ]; /66: (/NULL - ]), fd=(24)==(66), (66)==(24)] + ├── fd: ()-->(6), (1)-->(3,5), (21,22)-->(13,14,16-18,24,29,66), (22)-->(13,14,16-18,29), (24)==(66), (66)==(24), (1)==(21), (21)==(1) + ├── ordering: -17,+29,+13,+(1|21) opt(6) [actual: -17,+29,+13,+21] + ├── inner-join (lookup part) + │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_name:13!null s_address:14!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_name:29!null min:66!null + │ ├── key columns: [21] = [1] + │ ├── lookup columns are key + │ ├── key: (21,22) + │ ├── fd: ()-->(6), (1)-->(3,5), (21,22)-->(13,14,16-18,24,29,66), (22)-->(13,14,16-18,29), (24)==(66), (66)==(24), (1)==(21), (21)==(1) + │ ├── ordering: -17,+29,+13,+(1|21) opt(6) [actual: -17,+29,+13,+21] + │ ├── limit hint: 100.00 + │ ├── sort + │ │ ├── columns: s_name:13!null s_address:14!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_name:29!null min:66!null + │ │ ├── key: (21,22) + │ │ ├── fd: (21,22)-->(13,14,16-18,24,29,66), (22)-->(13,14,16-18,29), (24)==(66), (66)==(24) + │ │ ├── ordering: -17,+29,+13,+21 + │ │ └── select + │ │ ├── columns: s_name:13!null s_address:14!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_name:29!null min:66!null + │ │ ├── key: (21,22) + │ │ ├── fd: (21,22)-->(13,14,16-18,24,29,66), (22)-->(13,14,16-18,29), (24)==(66), (66)==(24) + │ │ ├── group-by (hash) + │ │ │ ├── columns: s_name:13!null s_address:14!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_name:29!null min:66!null + │ │ │ ├── grouping columns: ps_partkey:21!null ps_suppkey:22!null + │ │ │ ├── key: (21,22) + │ │ │ ├── fd: (21,22)-->(13,14,16-18,24,29,66), (22)-->(13,14,16-18,29) + │ │ │ ├── inner-join (hash) + │ │ │ │ ├── columns: s_suppkey:12!null s_name:13!null s_address:14!null s_nationkey:15!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_nationkey:28!null n_name:29!null n_regionkey:30!null r_regionkey:34!null r_name:35!null ps_partkey:39!null ps_suppkey:40!null ps_supplycost:42!null s_suppkey:46!null s_nationkey:49!null n_nationkey:55!null n_regionkey:57!null r_regionkey:61!null r_name:62!null + │ │ │ │ ├── key: (22,39,46) + │ │ │ │ ├── fd: ()-->(35,62), (12)-->(13-18), (21,22)-->(24), (12)==(22), (22)==(12), (28)-->(29,30), (30)==(34), (34)==(30), (15)==(28), (28)==(15), (39,40)-->(42), (46)-->(49), (55)-->(57), (57)==(61), (61)==(57), (49)==(55), (55)==(49), (40)==(46), (46)==(40), (21)==(39), (39)==(21) + │ │ │ │ ├── inner-join (hash) + │ │ │ │ │ ├── columns: s_suppkey:12!null s_name:13!null s_address:14!null s_nationkey:15!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_nationkey:28!null n_name:29!null n_regionkey:30!null r_regionkey:34!null r_name:35!null + │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + │ │ │ │ │ ├── key: (21,22) + │ │ │ │ │ ├── fd: ()-->(35), (12)-->(13-18), (21,22)-->(24), (12)==(22), (22)==(12), (28)-->(29,30), (30)==(34), (34)==(30), (15)==(28), (28)==(15) + │ │ │ │ │ ├── scan partsupp + │ │ │ │ │ │ ├── columns: ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null + │ │ │ │ │ │ ├── key: (21,22) + │ │ │ │ │ │ └── fd: (21,22)-->(24) + │ │ │ │ │ ├── inner-join (hash) + │ │ │ │ │ │ ├── columns: s_suppkey:12!null s_name:13!null s_address:14!null s_nationkey:15!null s_phone:16!null s_acctbal:17!null s_comment:18!null n_nationkey:28!null n_name:29!null n_regionkey:30!null r_regionkey:34!null r_name:35!null + │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + │ │ │ │ │ │ ├── key: (12) + │ │ │ │ │ │ ├── fd: ()-->(35), (12)-->(13-18), (28)-->(29,30), (30)==(34), (34)==(30), (15)==(28), (28)==(15) + │ │ │ │ │ │ ├── scan supplier + │ │ │ │ │ │ │ ├── columns: s_suppkey:12!null s_name:13!null s_address:14!null s_nationkey:15!null s_phone:16!null s_acctbal:17!null s_comment:18!null + │ │ │ │ │ │ │ ├── key: (12) + │ │ │ │ │ │ │ └── fd: (12)-->(13-18) + │ │ │ │ │ │ ├── inner-join (hash) + │ │ │ │ │ │ │ ├── columns: n_nationkey:28!null n_name:29!null n_regionkey:30!null r_regionkey:34!null r_name:35!null + │ │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + │ │ │ │ │ │ │ ├── key: (28) + │ │ │ │ │ │ │ ├── fd: ()-->(35), (28)-->(29,30), (30)==(34), (34)==(30) + │ │ │ │ │ │ │ ├── scan nation + │ │ │ │ │ │ │ │ ├── columns: n_nationkey:28!null n_name:29!null n_regionkey:30!null + │ │ │ │ │ │ │ │ ├── key: (28) + │ │ │ │ │ │ │ │ └── fd: (28)-->(29,30) + │ │ │ │ │ │ │ ├── select + │ │ │ │ │ │ │ │ ├── columns: r_regionkey:34!null r_name:35!null + │ │ │ │ │ │ │ │ ├── key: (34) + │ │ │ │ │ │ │ │ ├── fd: ()-->(35) + │ │ │ │ │ │ │ │ ├── scan region + │ │ │ │ │ │ │ │ │ ├── columns: r_regionkey:34!null r_name:35!null + │ │ │ │ │ │ │ │ │ ├── key: (34) + │ │ │ │ │ │ │ │ │ └── fd: (34)-->(35) + │ │ │ │ │ │ │ │ └── filters + │ │ │ │ │ │ │ │ └── r_name:35 = 'EUROPE' [outer=(35), constraints=(/35: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(35)] + │ │ │ │ │ │ │ └── filters + │ │ │ │ │ │ │ └── n_regionkey:30 = r_regionkey:34 [outer=(30,34), constraints=(/30: (/NULL - ]; /34: (/NULL - ]), fd=(30)==(34), (34)==(30)] + │ │ │ │ │ │ └── filters + │ │ │ │ │ │ └── s_nationkey:15 = n_nationkey:28 [outer=(15,28), constraints=(/15: (/NULL - ]; /28: (/NULL - ]), fd=(15)==(28), (28)==(15)] + │ │ │ │ │ └── filters + │ │ │ │ │ └── s_suppkey:12 = ps_suppkey:22 [outer=(12,22), constraints=(/12: (/NULL - ]; /22: (/NULL - ]), fd=(12)==(22), (22)==(12)] + │ │ │ │ ├── inner-join (hash) + │ │ │ │ │ ├── columns: ps_partkey:39!null ps_suppkey:40!null ps_supplycost:42!null s_suppkey:46!null s_nationkey:49!null n_nationkey:55!null n_regionkey:57!null r_regionkey:61!null r_name:62!null + │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + │ │ │ │ │ ├── key: (39,46) + │ │ │ │ │ ├── fd: ()-->(62), (39,40)-->(42), (46)-->(49), (55)-->(57), (57)==(61), (61)==(57), (49)==(55), (55)==(49), (40)==(46), (46)==(40) + │ │ │ │ │ ├── scan partsupp + │ │ │ │ │ │ ├── columns: ps_partkey:39!null ps_suppkey:40!null ps_supplycost:42!null + │ │ │ │ │ │ ├── key: (39,40) + │ │ │ │ │ │ └── fd: (39,40)-->(42) + │ │ │ │ │ ├── inner-join (lookup supplier@s_nk) + │ │ │ │ │ │ ├── columns: s_suppkey:46!null s_nationkey:49!null n_nationkey:55!null n_regionkey:57!null r_regionkey:61!null r_name:62!null + │ │ │ │ │ │ ├── key columns: [55] = [49] + │ │ │ │ │ │ ├── key: (46) + │ │ │ │ │ │ ├── fd: ()-->(62), (46)-->(49), (55)-->(57), (57)==(61), (61)==(57), (49)==(55), (55)==(49) + │ │ │ │ │ │ ├── inner-join (lookup nation@n_rk) + │ │ │ │ │ │ │ ├── columns: n_nationkey:55!null n_regionkey:57!null r_regionkey:61!null r_name:62!null + │ │ │ │ │ │ │ ├── key columns: [61] = [57] + │ │ │ │ │ │ │ ├── key: (55) + │ │ │ │ │ │ │ ├── fd: ()-->(62), (55)-->(57), (57)==(61), (61)==(57) + │ │ │ │ │ │ │ ├── select + │ │ │ │ │ │ │ │ ├── columns: r_regionkey:61!null r_name:62!null + │ │ │ │ │ │ │ │ ├── key: (61) + │ │ │ │ │ │ │ │ ├── fd: ()-->(62) + │ │ │ │ │ │ │ │ ├── scan region + │ │ │ │ │ │ │ │ │ ├── columns: r_regionkey:61!null r_name:62!null + │ │ │ │ │ │ │ │ │ ├── key: (61) + │ │ │ │ │ │ │ │ │ └── fd: (61)-->(62) + │ │ │ │ │ │ │ │ └── filters + │ │ │ │ │ │ │ │ └── r_name:62 = 'EUROPE' [outer=(62), constraints=(/62: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(62)] + │ │ │ │ │ │ │ └── filters (true) + │ │ │ │ │ │ └── filters (true) + │ │ │ │ │ └── filters + │ │ │ │ │ └── s_suppkey:46 = ps_suppkey:40 [outer=(40,46), constraints=(/40: (/NULL - ]; /46: (/NULL - ]), fd=(40)==(46), (46)==(40)] + │ │ │ │ └── filters + │ │ │ │ └── ps_partkey:21 = ps_partkey:39 [outer=(21,39), constraints=(/21: (/NULL - ]; /39: (/NULL - ]), fd=(21)==(39), (39)==(21)] + │ │ │ └── aggregations + │ │ │ ├── min [as=min:66, outer=(42)] + │ │ │ │ └── ps_supplycost:42 + │ │ │ ├── const-agg [as=s_name:13, outer=(13)] + │ │ │ │ └── s_name:13 + │ │ │ ├── const-agg [as=s_address:14, outer=(14)] + │ │ │ │ └── s_address:14 + │ │ │ ├── const-agg [as=s_phone:16, outer=(16)] + │ │ │ │ └── s_phone:16 + │ │ │ ├── const-agg [as=s_acctbal:17, outer=(17)] + │ │ │ │ └── s_acctbal:17 + │ │ │ ├── const-agg [as=s_comment:18, outer=(18)] + │ │ │ │ └── s_comment:18 + │ │ │ ├── const-agg [as=ps_supplycost:24, outer=(24)] + │ │ │ │ └── ps_supplycost:24 + │ │ │ └── const-agg [as=n_name:29, outer=(29)] + │ │ │ └── n_name:29 + │ │ └── filters + │ │ └── ps_supplycost:24 = min:66 [outer=(24,66), constraints=(/24: (/NULL - ]; /66: (/NULL - ]), fd=(24)==(66), (66)==(24)] + │ └── filters + │ ├── p_size:6 = 15 [outer=(6), constraints=(/6: [/15 - /15]; tight), fd=()-->(6)] + │ └── p_type:5 LIKE '%BRASS' [outer=(5), constraints=(/5: (/NULL - ])] + └── 100 # -------------------------------------------------- # Q3 diff --git a/pkg/sql/opt/xform/testdata/external/tpch-no-stats b/pkg/sql/opt/xform/testdata/external/tpch-no-stats index 57ccf606336b..d591e9c16817 100644 --- a/pkg/sql/opt/xform/testdata/external/tpch-no-stats +++ b/pkg/sql/opt/xform/testdata/external/tpch-no-stats @@ -148,124 +148,148 @@ project ├── cardinality: [0 - 100] ├── fd: (1)-->(3) ├── ordering: -17,+29,+13,+1 - └── top-k - ├── columns: p_partkey:1!null p_mfgr:3!null s_name:13!null s_address:14!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_name:29!null min:66!null - ├── internal-ordering: -17,+29,+13,+(1|21) - ├── k: 100 + └── limit + ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_name:13!null s_address:14!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_name:29!null min:66!null + ├── internal-ordering: -17,+29,+13,+(1|21) opt(6) ├── cardinality: [0 - 100] ├── key: (21,22) - ├── fd: (1)-->(3), (21,22)-->(1,3,13,14,16-18,24,29,66), (1)==(21), (21)==(1), (22)-->(13,14,16-18,29), (24)==(66), (66)==(24) - ├── ordering: -17,+29,+13,+(1|21) [actual: -17,+29,+13,+1] - └── select - ├── columns: p_partkey:1!null p_mfgr:3!null s_name:13!null s_address:14!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_name:29!null min:66!null - ├── key: (21,22) - ├── fd: (1)-->(3), (21,22)-->(1,3,13,14,16-18,24,29,66), (1)==(21), (21)==(1), (22)-->(13,14,16-18,29), (24)==(66), (66)==(24) - ├── group-by (streaming) - │ ├── columns: p_partkey:1!null p_mfgr:3!null s_name:13!null s_address:14!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_name:29!null min:66!null - │ ├── grouping columns: ps_partkey:21!null ps_suppkey:22!null - │ ├── internal-ordering: +(1|21|39),+(12|22) opt(6,35,62) - │ ├── key: (21,22) - │ ├── fd: (1)-->(3), (21,22)-->(1,3,13,14,16-18,24,29,66), (1)==(21), (21)==(1), (22)-->(13,14,16-18,29) - │ ├── inner-join (lookup region) - │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:12!null s_name:13!null s_address:14!null s_nationkey:15!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_nationkey:28!null n_name:29!null n_regionkey:30!null r_regionkey:34!null r_name:35!null ps_partkey:39!null ps_suppkey:40!null ps_supplycost:42!null s_suppkey:46!null s_nationkey:49!null n_nationkey:55!null n_regionkey:57!null r_regionkey:61!null r_name:62!null - │ │ ├── key columns: [57] = [61] - │ │ ├── lookup columns are key - │ │ ├── key: (22,39,46) - │ │ ├── fd: ()-->(6,35,62), (1)-->(3,5), (12)-->(13-18), (21,22)-->(24), (12)==(22), (22)==(12), (28)-->(29,30), (30)==(34), (34)==(30), (15)==(28), (28)==(15), (1)==(21,39), (21)==(1,39), (39,40)-->(42), (46)-->(49), (55)-->(57), (57)==(61), (61)==(57), (49)==(55), (55)==(49), (40)==(46), (46)==(40), (39)==(1,21) - │ │ ├── ordering: +(1|21|39),+(12|22) opt(6,35,62) [actual: +1,+22] - │ │ ├── inner-join (lookup nation) - │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:12!null s_name:13!null s_address:14!null s_nationkey:15!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_nationkey:28!null n_name:29!null n_regionkey:30!null r_regionkey:34!null r_name:35!null ps_partkey:39!null ps_suppkey:40!null ps_supplycost:42!null s_suppkey:46!null s_nationkey:49!null n_nationkey:55!null n_regionkey:57!null - │ │ │ ├── key columns: [49] = [55] - │ │ │ ├── lookup columns are key - │ │ │ ├── key: (22,39,46) - │ │ │ ├── fd: ()-->(6,35), (1)-->(3,5), (28)-->(29,30), (12)-->(13-18), (21,22)-->(24), (39,40)-->(42), (46)-->(49), (55)-->(57), (49)==(55), (55)==(49), (40)==(46), (46)==(40), (21)==(1,39), (39)==(1,21), (12)==(22), (22)==(12), (15)==(28), (28)==(15), (30)==(34), (34)==(30), (1)==(21,39) - │ │ │ ├── ordering: +(1|21|39),+(12|22) opt(6,35) [actual: +1,+22] - │ │ │ ├── inner-join (lookup supplier) - │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:12!null s_name:13!null s_address:14!null s_nationkey:15!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_nationkey:28!null n_name:29!null n_regionkey:30!null r_regionkey:34!null r_name:35!null ps_partkey:39!null ps_suppkey:40!null ps_supplycost:42!null s_suppkey:46!null s_nationkey:49!null - │ │ │ │ ├── key columns: [40] = [46] - │ │ │ │ ├── lookup columns are key - │ │ │ │ ├── key: (22,39,46) - │ │ │ │ ├── fd: ()-->(6,35), (1)-->(3,5), (28)-->(29,30), (12)-->(13-18), (21,22)-->(24), (39,40)-->(42), (46)-->(49), (40)==(46), (46)==(40), (21)==(1,39), (39)==(1,21), (12)==(22), (22)==(12), (15)==(28), (28)==(15), (30)==(34), (34)==(30), (1)==(21,39) - │ │ │ │ ├── ordering: +(1|21|39),+(12|22) opt(6,35) [actual: +1,+22] - │ │ │ │ ├── inner-join (lookup partsupp) - │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:12!null s_name:13!null s_address:14!null s_nationkey:15!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_nationkey:28!null n_name:29!null n_regionkey:30!null r_regionkey:34!null r_name:35!null ps_partkey:39!null ps_suppkey:40!null ps_supplycost:42!null - │ │ │ │ │ ├── key columns: [1] = [39] - │ │ │ │ │ ├── key: (22,39,40) - │ │ │ │ │ ├── fd: ()-->(6,35), (1)-->(3,5), (28)-->(29,30), (12)-->(13-18), (21,22)-->(24), (39,40)-->(42), (21)==(1,39), (39)==(1,21), (12)==(22), (22)==(12), (15)==(28), (28)==(15), (30)==(34), (34)==(30), (1)==(21,39) - │ │ │ │ │ ├── ordering: +(1|21|39),+(12|22) opt(6,35) [actual: +1,+22] - │ │ │ │ │ ├── inner-join (lookup region) - │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:12!null s_name:13!null s_address:14!null s_nationkey:15!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_nationkey:28!null n_name:29!null n_regionkey:30!null r_regionkey:34!null r_name:35!null - │ │ │ │ │ │ ├── key columns: [30] = [34] - │ │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ │ ├── key: (21,22) - │ │ │ │ │ │ ├── fd: ()-->(6,35), (1)-->(3,5), (12)-->(13-18), (21,22)-->(24), (12)==(22), (22)==(12), (28)-->(29,30), (30)==(34), (34)==(30), (15)==(28), (28)==(15), (1)==(21), (21)==(1) - │ │ │ │ │ │ ├── ordering: +(1|21),+(12|22) opt(6,35) [actual: +1,+22] - │ │ │ │ │ │ ├── inner-join (lookup nation) - │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:12!null s_name:13!null s_address:14!null s_nationkey:15!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_nationkey:28!null n_name:29!null n_regionkey:30!null - │ │ │ │ │ │ │ ├── key columns: [15] = [28] - │ │ │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ │ │ ├── key: (21,22) - │ │ │ │ │ │ │ ├── fd: ()-->(6), (1)-->(3,5), (21,22)-->(24), (12)-->(13-18), (28)-->(29,30), (15)==(28), (28)==(15), (12)==(22), (22)==(12), (1)==(21), (21)==(1) - │ │ │ │ │ │ │ ├── ordering: +(1|21),+(12|22) opt(6) [actual: +1,+22] - │ │ │ │ │ │ │ ├── inner-join (lookup supplier) - │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:12!null s_name:13!null s_address:14!null s_nationkey:15!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null - │ │ │ │ │ │ │ │ ├── key columns: [22] = [12] - │ │ │ │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ │ │ │ ├── key: (21,22) - │ │ │ │ │ │ │ │ ├── fd: ()-->(6), (1)-->(3,5), (12)-->(13-18), (21,22)-->(24), (12)==(22), (22)==(12), (1)==(21), (21)==(1) - │ │ │ │ │ │ │ │ ├── ordering: +(1|21),+(12|22) opt(6) [actual: +1,+22] - │ │ │ │ │ │ │ │ ├── inner-join (lookup partsupp) - │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null - │ │ │ │ │ │ │ │ │ ├── key columns: [1] = [21] - │ │ │ │ │ │ │ │ │ ├── key: (21,22) - │ │ │ │ │ │ │ │ │ ├── fd: ()-->(6), (1)-->(3,5), (21,22)-->(24), (1)==(21), (21)==(1) - │ │ │ │ │ │ │ │ │ ├── ordering: +(1|21),+22 opt(6) [actual: +1,+22] - │ │ │ │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null - │ │ │ │ │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ │ │ │ │ ├── fd: ()-->(6), (1)-->(3,5) - │ │ │ │ │ │ │ │ │ │ ├── ordering: +1 opt(6) [actual: +1] - │ │ │ │ │ │ │ │ │ │ ├── scan part - │ │ │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null - │ │ │ │ │ │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ │ │ │ │ │ ├── fd: (1)-->(3,5,6) - │ │ │ │ │ │ │ │ │ │ │ └── ordering: +1 - │ │ │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ │ │ ├── p_size:6 = 15 [outer=(6), constraints=(/6: [/15 - /15]; tight), fd=()-->(6)] - │ │ │ │ │ │ │ │ │ │ └── p_type:5 LIKE '%BRASS' [outer=(5), constraints=(/5: (/NULL - ])] - │ │ │ │ │ │ │ │ │ └── filters (true) - │ │ │ │ │ │ │ │ └── filters (true) - │ │ │ │ │ │ │ └── filters (true) - │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── r_name:35 = 'EUROPE' [outer=(35), constraints=(/35: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(35)] - │ │ │ │ │ └── filters (true) - │ │ │ │ └── filters (true) - │ │ │ └── filters (true) - │ │ └── filters - │ │ └── r_name:62 = 'EUROPE' [outer=(62), constraints=(/62: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(62)] - │ └── aggregations - │ ├── min [as=min:66, outer=(42)] - │ │ └── ps_supplycost:42 - │ ├── const-agg [as=s_name:13, outer=(13)] - │ │ └── s_name:13 - │ ├── const-agg [as=s_address:14, outer=(14)] - │ │ └── s_address:14 - │ ├── const-agg [as=s_phone:16, outer=(16)] - │ │ └── s_phone:16 - │ ├── const-agg [as=s_acctbal:17, outer=(17)] - │ │ └── s_acctbal:17 - │ ├── const-agg [as=s_comment:18, outer=(18)] - │ │ └── s_comment:18 - │ ├── const-agg [as=ps_supplycost:24, outer=(24)] - │ │ └── ps_supplycost:24 - │ ├── const-agg [as=n_name:29, outer=(29)] - │ │ └── n_name:29 - │ ├── const-agg [as=p_mfgr:3, outer=(3)] - │ │ └── p_mfgr:3 - │ └── const-agg [as=p_partkey:1, outer=(1)] - │ └── p_partkey:1 - └── filters - └── ps_supplycost:24 = min:66 [outer=(24,66), constraints=(/24: (/NULL - ]; /66: (/NULL - ]), fd=(24)==(66), (66)==(24)] + ├── fd: ()-->(6), (1)-->(3,5), (21,22)-->(13,14,16-18,24,29,66), (22)-->(13,14,16-18,29), (24)==(66), (66)==(24), (1)==(21), (21)==(1) + ├── ordering: -17,+29,+13,+(1|21) opt(6) [actual: -17,+29,+13,+21] + ├── inner-join (lookup part) + │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_name:13!null s_address:14!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_name:29!null min:66!null + │ ├── key columns: [21] = [1] + │ ├── lookup columns are key + │ ├── key: (21,22) + │ ├── fd: ()-->(6), (1)-->(3,5), (21,22)-->(13,14,16-18,24,29,66), (22)-->(13,14,16-18,29), (24)==(66), (66)==(24), (1)==(21), (21)==(1) + │ ├── ordering: -17,+29,+13,+(1|21) opt(6) [actual: -17,+29,+13,+21] + │ ├── limit hint: 100.00 + │ ├── sort + │ │ ├── columns: s_name:13!null s_address:14!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_name:29!null min:66!null + │ │ ├── key: (21,22) + │ │ ├── fd: (21,22)-->(13,14,16-18,24,29,66), (22)-->(13,14,16-18,29), (24)==(66), (66)==(24) + │ │ ├── ordering: -17,+29,+13,+21 + │ │ └── select + │ │ ├── columns: s_name:13!null s_address:14!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_name:29!null min:66!null + │ │ ├── key: (21,22) + │ │ ├── fd: (21,22)-->(13,14,16-18,24,29,66), (22)-->(13,14,16-18,29), (24)==(66), (66)==(24) + │ │ ├── group-by (hash) + │ │ │ ├── columns: s_name:13!null s_address:14!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_name:29!null min:66!null + │ │ │ ├── grouping columns: ps_partkey:21!null ps_suppkey:22!null + │ │ │ ├── key: (21,22) + │ │ │ ├── fd: (21,22)-->(13,14,16-18,24,29,66), (22)-->(13,14,16-18,29) + │ │ │ ├── inner-join (hash) + │ │ │ │ ├── columns: s_suppkey:12!null s_name:13!null s_address:14!null s_nationkey:15!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_nationkey:28!null n_name:29!null n_regionkey:30!null r_regionkey:34!null r_name:35!null ps_partkey:39!null ps_suppkey:40!null ps_supplycost:42!null s_suppkey:46!null s_nationkey:49!null n_nationkey:55!null n_regionkey:57!null r_regionkey:61!null r_name:62!null + │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + │ │ │ │ ├── key: (22,39,46) + │ │ │ │ ├── fd: ()-->(35,62), (12)-->(13-18), (21,22)-->(24), (12)==(22), (22)==(12), (28)-->(29,30), (30)==(34), (34)==(30), (15)==(28), (28)==(15), (39,40)-->(42), (46)-->(49), (55)-->(57), (57)==(61), (61)==(57), (49)==(55), (55)==(49), (40)==(46), (46)==(40), (21)==(39), (39)==(21) + │ │ │ │ ├── inner-join (hash) + │ │ │ │ │ ├── columns: s_suppkey:12!null s_name:13!null s_address:14!null s_nationkey:15!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_nationkey:28!null n_name:29!null n_regionkey:30!null r_regionkey:34!null r_name:35!null ps_partkey:39!null ps_suppkey:40!null ps_supplycost:42!null s_suppkey:46!null s_nationkey:49!null + │ │ │ │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-more) + │ │ │ │ │ ├── key: (22,39,46) + │ │ │ │ │ ├── fd: ()-->(35), (28)-->(29,30), (12)-->(13-18), (21,22)-->(24), (39,40)-->(42), (46)-->(49), (40)==(46), (46)==(40), (21)==(39), (39)==(21), (12)==(22), (22)==(12), (15)==(28), (28)==(15), (30)==(34), (34)==(30) + │ │ │ │ │ ├── inner-join (hash) + │ │ │ │ │ │ ├── columns: s_suppkey:12!null s_name:13!null s_address:14!null s_nationkey:15!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null n_nationkey:28!null n_name:29!null n_regionkey:30!null r_regionkey:34!null r_name:35!null + │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + │ │ │ │ │ │ ├── key: (21,22) + │ │ │ │ │ │ ├── fd: ()-->(35), (12)-->(13-18), (21,22)-->(24), (12)==(22), (22)==(12), (28)-->(29,30), (30)==(34), (34)==(30), (15)==(28), (28)==(15) + │ │ │ │ │ │ ├── inner-join (hash) + │ │ │ │ │ │ │ ├── columns: s_suppkey:12!null s_name:13!null s_address:14!null s_nationkey:15!null s_phone:16!null s_acctbal:17!null s_comment:18!null ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null + │ │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(exactly-one) + │ │ │ │ │ │ │ ├── key: (21,22) + │ │ │ │ │ │ │ ├── fd: (12)-->(13-18), (21,22)-->(24), (12)==(22), (22)==(12) + │ │ │ │ │ │ │ ├── scan supplier + │ │ │ │ │ │ │ │ ├── columns: s_suppkey:12!null s_name:13!null s_address:14!null s_nationkey:15!null s_phone:16!null s_acctbal:17!null s_comment:18!null + │ │ │ │ │ │ │ │ ├── key: (12) + │ │ │ │ │ │ │ │ └── fd: (12)-->(13-18) + │ │ │ │ │ │ │ ├── scan partsupp + │ │ │ │ │ │ │ │ ├── columns: ps_partkey:21!null ps_suppkey:22!null ps_supplycost:24!null + │ │ │ │ │ │ │ │ ├── key: (21,22) + │ │ │ │ │ │ │ │ └── fd: (21,22)-->(24) + │ │ │ │ │ │ │ └── filters + │ │ │ │ │ │ │ └── s_suppkey:12 = ps_suppkey:22 [outer=(12,22), constraints=(/12: (/NULL - ]; /22: (/NULL - ]), fd=(12)==(22), (22)==(12)] + │ │ │ │ │ │ ├── inner-join (lookup nation) + │ │ │ │ │ │ │ ├── columns: n_nationkey:28!null n_name:29!null n_regionkey:30!null r_regionkey:34!null r_name:35!null + │ │ │ │ │ │ │ ├── key columns: [28] = [28] + │ │ │ │ │ │ │ ├── lookup columns are key + │ │ │ │ │ │ │ ├── key: (28) + │ │ │ │ │ │ │ ├── fd: ()-->(35), (28)-->(29,30), (30)==(34), (34)==(30) + │ │ │ │ │ │ │ ├── inner-join (lookup nation@n_rk) + │ │ │ │ │ │ │ │ ├── columns: n_nationkey:28!null n_regionkey:30!null r_regionkey:34!null r_name:35!null + │ │ │ │ │ │ │ │ ├── key columns: [34] = [30] + │ │ │ │ │ │ │ │ ├── key: (28) + │ │ │ │ │ │ │ │ ├── fd: ()-->(35), (28)-->(30), (30)==(34), (34)==(30) + │ │ │ │ │ │ │ │ ├── select + │ │ │ │ │ │ │ │ │ ├── columns: r_regionkey:34!null r_name:35!null + │ │ │ │ │ │ │ │ │ ├── key: (34) + │ │ │ │ │ │ │ │ │ ├── fd: ()-->(35) + │ │ │ │ │ │ │ │ │ ├── scan region + │ │ │ │ │ │ │ │ │ │ ├── columns: r_regionkey:34!null r_name:35!null + │ │ │ │ │ │ │ │ │ │ ├── key: (34) + │ │ │ │ │ │ │ │ │ │ └── fd: (34)-->(35) + │ │ │ │ │ │ │ │ │ └── filters + │ │ │ │ │ │ │ │ │ └── r_name:35 = 'EUROPE' [outer=(35), constraints=(/35: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(35)] + │ │ │ │ │ │ │ │ └── filters (true) + │ │ │ │ │ │ │ └── filters (true) + │ │ │ │ │ │ └── filters + │ │ │ │ │ │ └── s_nationkey:15 = n_nationkey:28 [outer=(15,28), constraints=(/15: (/NULL - ]; /28: (/NULL - ]), fd=(15)==(28), (28)==(15)] + │ │ │ │ │ ├── inner-join (hash) + │ │ │ │ │ │ ├── columns: ps_partkey:39!null ps_suppkey:40!null ps_supplycost:42!null s_suppkey:46!null s_nationkey:49!null + │ │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) + │ │ │ │ │ │ ├── key: (39,46) + │ │ │ │ │ │ ├── fd: (39,40)-->(42), (46)-->(49), (40)==(46), (46)==(40) + │ │ │ │ │ │ ├── scan partsupp + │ │ │ │ │ │ │ ├── columns: ps_partkey:39!null ps_suppkey:40!null ps_supplycost:42!null + │ │ │ │ │ │ │ ├── key: (39,40) + │ │ │ │ │ │ │ └── fd: (39,40)-->(42) + │ │ │ │ │ │ ├── scan supplier@s_nk + │ │ │ │ │ │ │ ├── columns: s_suppkey:46!null s_nationkey:49!null + │ │ │ │ │ │ │ ├── key: (46) + │ │ │ │ │ │ │ └── fd: (46)-->(49) + │ │ │ │ │ │ └── filters + │ │ │ │ │ │ └── s_suppkey:46 = ps_suppkey:40 [outer=(40,46), constraints=(/40: (/NULL - ]; /46: (/NULL - ]), fd=(40)==(46), (46)==(40)] + │ │ │ │ │ └── filters + │ │ │ │ │ └── ps_partkey:21 = ps_partkey:39 [outer=(21,39), constraints=(/21: (/NULL - ]; /39: (/NULL - ]), fd=(21)==(39), (39)==(21)] + │ │ │ │ ├── inner-join (lookup nation@n_rk) + │ │ │ │ │ ├── columns: n_nationkey:55!null n_regionkey:57!null r_regionkey:61!null r_name:62!null + │ │ │ │ │ ├── key columns: [61] = [57] + │ │ │ │ │ ├── key: (55) + │ │ │ │ │ ├── fd: ()-->(62), (55)-->(57), (57)==(61), (61)==(57) + │ │ │ │ │ ├── select + │ │ │ │ │ │ ├── columns: r_regionkey:61!null r_name:62!null + │ │ │ │ │ │ ├── key: (61) + │ │ │ │ │ │ ├── fd: ()-->(62) + │ │ │ │ │ │ ├── scan region + │ │ │ │ │ │ │ ├── columns: r_regionkey:61!null r_name:62!null + │ │ │ │ │ │ │ ├── key: (61) + │ │ │ │ │ │ │ └── fd: (61)-->(62) + │ │ │ │ │ │ └── filters + │ │ │ │ │ │ └── r_name:62 = 'EUROPE' [outer=(62), constraints=(/62: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(62)] + │ │ │ │ │ └── filters (true) + │ │ │ │ └── filters + │ │ │ │ └── s_nationkey:49 = n_nationkey:55 [outer=(49,55), constraints=(/49: (/NULL - ]; /55: (/NULL - ]), fd=(49)==(55), (55)==(49)] + │ │ │ └── aggregations + │ │ │ ├── min [as=min:66, outer=(42)] + │ │ │ │ └── ps_supplycost:42 + │ │ │ ├── const-agg [as=s_name:13, outer=(13)] + │ │ │ │ └── s_name:13 + │ │ │ ├── const-agg [as=s_address:14, outer=(14)] + │ │ │ │ └── s_address:14 + │ │ │ ├── const-agg [as=s_phone:16, outer=(16)] + │ │ │ │ └── s_phone:16 + │ │ │ ├── const-agg [as=s_acctbal:17, outer=(17)] + │ │ │ │ └── s_acctbal:17 + │ │ │ ├── const-agg [as=s_comment:18, outer=(18)] + │ │ │ │ └── s_comment:18 + │ │ │ ├── const-agg [as=ps_supplycost:24, outer=(24)] + │ │ │ │ └── ps_supplycost:24 + │ │ │ └── const-agg [as=n_name:29, outer=(29)] + │ │ │ └── n_name:29 + │ │ └── filters + │ │ └── ps_supplycost:24 = min:66 [outer=(24,66), constraints=(/24: (/NULL - ]; /66: (/NULL - ]), fd=(24)==(66), (66)==(24)] + │ └── filters + │ ├── p_size:6 = 15 [outer=(6), constraints=(/6: [/15 - /15]; tight), fd=()-->(6)] + │ └── p_type:5 LIKE '%BRASS' [outer=(5), constraints=(/5: (/NULL - ])] + └── 100 # -------------------------------------------------- # Q3 diff --git a/pkg/sql/opt/xform/testdata/rules/disjunction_in_join b/pkg/sql/opt/xform/testdata/rules/disjunction_in_join index 39477bc3d3db..dabd51c08c85 100644 --- a/pkg/sql/opt/xform/testdata/rules/disjunction_in_join +++ b/pkg/sql/opt/xform/testdata/rules/disjunction_in_join @@ -2405,7 +2405,7 @@ project │ │ ├── (a1:1 = c1:7) OR (c1:7 = c2:8) [outer=(1,7,8), constraints=(/7: (/NULL - ])] │ │ └── a2:2 = c2:8 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] │ └── filters - │ └── (a1:1 = b1:14) OR (a1:1 = a2:2) [outer=(1,2,14), constraints=(/1: (/NULL - ])] + │ └── (a1:1 = b1:14) OR (a1:1 = c2:8) [outer=(1,8,14), constraints=(/1: (/NULL - ])] └── aggregations ├── const-agg [as=c1:7, outer=(7)] │ └── c1:7 diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index 5e1cb9ef5fde..ac8df5d9e392 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keyvisualizer" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/repstream" @@ -35,7 +36,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/evalcatalog" "github.com/cockroachdb/cockroach/pkg/sql/exprutil" "github.com/cockroachdb/cockroach/pkg/sql/idxusage" - "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/querycache" @@ -95,15 +95,8 @@ type extendedEvalContext struct { TxnModesSetter txnModesSetter - // Jobs refers to jobs in extraTxnState. Jobs is a pointer to a jobsCollection - // which is a slice because we need calls to resetExtraTxnState to reset the - // jobsCollection. - Jobs *jobsCollection - - // SchemaChangeJobRecords refers to schemaChangeJobsCache in extraTxnState of - // in sql.connExecutor. sql.connExecutor.createJobs() enqueues jobs with these - // records when transaction is committed. - SchemaChangeJobRecords map[descpb.ID]*jobs.Record + // jobs refers to jobs in extraTxnState. + jobs *txnJobsCollection statsProvider *persistedsqlstats.PersistedSQLStats @@ -145,21 +138,11 @@ func (evalCtx *extendedEvalContext) copy() *extendedEvalContext { // QueueJob creates a new job from record and queues it for execution after // the transaction commits. -func (evalCtx *extendedEvalContext) QueueJob( - ctx context.Context, txn isql.Txn, record jobs.Record, -) (*jobs.Job, error) { +func (evalCtx *extendedEvalContext) QueueJob(record *jobs.Record) jobspb.JobID { jobID := evalCtx.ExecCfg.JobRegistry.MakeJobID() - job, err := evalCtx.ExecCfg.JobRegistry.CreateJobWithTxn( - ctx, - record, - jobID, - txn, - ) - if err != nil { - return nil, err - } - evalCtx.Jobs.add(jobID) - return job, nil + record.JobID = jobID + evalCtx.jobs.addNonUniqueJobToCreate(record) + return jobID } // planner is the centerpiece of SQL statement execution combining session @@ -580,11 +563,10 @@ func (p *planner) InternalSQLTxn() descs.Txn { ie := MakeInternalExecutor(ief.server, ief.memMetrics, ief.monitor) ie.SetSessionData(p.SessionData()) ie.extraTxnState = &extraTxnState{ - txn: p.Txn(), - descCollection: p.Descriptors(), - jobs: p.extendedEvalCtx.Jobs, - schemaChangeJobRecords: p.extendedEvalCtx.SchemaChangeJobRecords, - schemaChangerState: p.extendedEvalCtx.SchemaChangerState, + txn: p.Txn(), + descCollection: p.Descriptors(), + jobs: p.extendedEvalCtx.jobs, + schemaChangerState: p.extendedEvalCtx.SchemaChangerState, } p.internalSQLTxn.init(p.txn, ie) } diff --git a/pkg/sql/schema.go b/pkg/sql/schema.go index 1bd80e16a2eb..e7524544f293 100644 --- a/pkg/sql/schema.go +++ b/pkg/sql/schema.go @@ -56,7 +56,7 @@ func (p *planner) writeSchemaDesc(ctx context.Context, desc *schemadesc.Mutable) func (p *planner) writeSchemaDescChange( ctx context.Context, desc *schemadesc.Mutable, jobDesc string, ) error { - record, recordExists := p.extendedEvalCtx.SchemaChangeJobRecords[desc.ID] + record, recordExists := p.extendedEvalCtx.jobs.uniqueToCreate[desc.ID] if recordExists { // Update it. record.AppendDescription(jobDesc) @@ -77,7 +77,7 @@ func (p *planner) writeSchemaDescChange( Progress: jobspb.SchemaChangeProgress{}, NonCancelable: true, } - p.extendedEvalCtx.SchemaChangeJobRecords[desc.ID] = &jobRecord + p.extendedEvalCtx.jobs.uniqueToCreate[desc.ID] = &jobRecord log.Infof(ctx, "queued new schema change job %d for schema %d", jobRecord.JobID, desc.ID) } diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/BUILD.bazel b/pkg/sql/schemachanger/scplan/internal/rules/current/BUILD.bazel index c3571fd2c62a..68142a001142 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/BUILD.bazel +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/BUILD.bazel @@ -17,6 +17,7 @@ go_library( "dep_garbage_collection.go", "dep_swap_index.go", "dep_two_version.go", + "helpers.go", "op_drop.go", "op_index_and_column.go", "registry.go", @@ -24,6 +25,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules/current", visibility = ["//pkg/sql/schemachanger/scplan:__subpackages__"], deps = [ + "//pkg/clusterversion", "//pkg/sql/schemachanger/rel", "//pkg/sql/schemachanger/scpb", "//pkg/sql/schemachanger/scplan/internal/opgen", @@ -31,6 +33,7 @@ go_library( "//pkg/sql/schemachanger/scplan/internal/scgraph", "//pkg/sql/schemachanger/screl", "//pkg/sql/sem/catid", + "@com_github_cockroachdb_errors//:errors", ], ) diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/assertions_test.go b/pkg/sql/schemachanger/scplan/internal/rules/current/assertions_test.go index 79f2ad030db9..6f5bfdcd7249 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/assertions_test.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/assertions_test.go @@ -62,7 +62,7 @@ func nonNilElement(element scpb.Element) scpb.Element { // One exception is foreign key constraint, which is not simple dependent nor data // element but it has a screl.ReferencedDescID attribute. func checkSimpleDependentsReferenceDescID(e scpb.Element) error { - if IsSimpleDependent(e) || IsData(e) { + if isSimpleDependent(e) || isData(e) { return nil } if _, ok := e.(*scpb.ForeignKeyConstraint); ok { @@ -85,15 +85,15 @@ func checkToAbsentCategories(e scpb.Element) error { s1 := opgen.NextStatus(e, scpb.Status_ABSENT, s0) switch s1 { case scpb.Status_DROPPED: - if IsDescriptor(e) || IsData(e) { + if isDescriptor(e) || isData(e) { return nil } case scpb.Status_VALIDATED, scpb.Status_WRITE_ONLY, scpb.Status_DELETE_ONLY: - if IsSubjectTo2VersionInvariant(e) { + if isSubjectTo2VersionInvariant(e) { return nil } case scpb.Status_ABSENT: - if IsSimpleDependent(e) { + if isSimpleDependent(e) { return nil } } @@ -103,7 +103,7 @@ func checkToAbsentCategories(e scpb.Element) error { // Assert that isWithTypeT covers all elements with embedded TypeTs. func checkIsWithTypeT(e scpb.Element) error { return screl.WalkTypes(e, func(t *types.T) error { - if IsWithTypeT(e) { + if isWithTypeT(e) { return nil } return errors.New("should verify isWithTypeT but doesn't") @@ -120,7 +120,7 @@ func checkIsWithExpression(e scpb.Element) error { case *scpb.RowLevelTTL: return nil } - if IsWithExpression(e) { + if isWithExpression(e) { return nil } return errors.New("should verify isWithExpression but doesn't") @@ -131,12 +131,12 @@ func checkIsWithExpression(e scpb.Element) error { // element. func checkIsColumnDependent(e scpb.Element) error { // Exclude columns themselves. - if IsColumn(e) { + if isColumn(e) { return nil } // A column dependent should have a ColumnID attribute. _, err := screl.Schema.GetAttribute(screl.ColumnID, e) - if IsColumnDependent(e) { + if isColumnDependent(e) { if err != nil { return errors.New("verifies isColumnDependent but doesn't have ColumnID attr") } @@ -150,12 +150,12 @@ func checkIsColumnDependent(e scpb.Element) error { // element. func checkIsIndexDependent(e scpb.Element) error { // Exclude indexes themselves and their data. - if IsIndex(e) || IsData(e) || IsSupportedNonIndexBackedConstraint(e) { + if isIndex(e) || isData(e) || isSupportedNonIndexBackedConstraint(e) { return nil } // An index dependent should have an IndexID attribute. _, err := screl.Schema.GetAttribute(screl.IndexID, e) - if IsIndexDependent(e) { + if isIndexDependent(e) { if err != nil { return errors.New("verifies isIndexDependent but doesn't have IndexID attr") } @@ -169,12 +169,12 @@ func checkIsIndexDependent(e scpb.Element) error { // element. func checkIsConstraintDependent(e scpb.Element) error { // Exclude constraints themselves. - if IsConstraint(e) { + if isConstraint(e) { return nil } // A constraint dependent should have a ConstraintID attribute. _, err := screl.Schema.GetAttribute(screl.ConstraintID, e) - if IsConstraintDependent(e) { + if isConstraintDependent(e) { if err != nil { return errors.New("verifies isConstraintDependent but doesn't have ConstraintID attr") } diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_column.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_column.go index 6d03da4fb5bd..3370d80e8bb3 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_column.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_column.go @@ -29,7 +29,7 @@ func init() { func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.Type((*scpb.Column)(nil)), - to.TypeFilter(IsColumnDependent), + to.TypeFilter(rulesVersionKey, isColumnDependent), JoinOnColumnID(from, to, "table-id", "col-id"), StatusesToPublicOrTransient(from, scpb.Status_DELETE_ONLY, to, scpb.Status_PUBLIC), } @@ -42,7 +42,7 @@ func init() { "dependent", "column", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsColumnDependent), + from.TypeFilter(rulesVersionKey, isColumnDependent), to.Type((*scpb.Column)(nil)), JoinOnColumnID(from, to, "table-id", "col-id"), StatusesToPublicOrTransient(from, scpb.Status_PUBLIC, to, scpb.Status_PUBLIC), diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_constraint.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_constraint.go index c1da8b356d10..81e89f3a8b79 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_constraint.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_constraint.go @@ -26,8 +26,8 @@ func init() { "dependent", "constraint", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsConstraintDependent), - to.TypeFilter(IsConstraint), + from.TypeFilter(rulesVersionKey, isConstraintDependent), + to.TypeFilter(rulesVersionKey, isConstraint), JoinOnConstraintID(from, to, "table-id", "constraint-id"), StatusesToPublicOrTransient(from, scpb.Status_PUBLIC, to, scpb.Status_PUBLIC), } diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_index.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_index.go index 4c0f616b0c55..b6cd3017ea60 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_index.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_index.go @@ -32,7 +32,7 @@ func init() { (*scpb.PrimaryIndex)(nil), (*scpb.SecondaryIndex)(nil), ), - to.TypeFilter(IsIndexDependent), + to.TypeFilter(rulesVersionKey, isIndexDependent), JoinOnIndexID(from, to, "table-id", "index-id"), StatusesToPublicOrTransient(from, scpb.Status_BACKFILL_ONLY, to, scpb.Status_PUBLIC), } @@ -46,7 +46,7 @@ func init() { func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.Type((*scpb.TemporaryIndex)(nil)), - to.TypeFilter(IsIndexDependent), + to.TypeFilter(rulesVersionKey, isIndexDependent), JoinOnIndexID(from, to, "table-id", "index-id"), StatusesToPublicOrTransient(from, scpb.Status_DELETE_ONLY, to, scpb.Status_PUBLIC), } @@ -59,8 +59,8 @@ func init() { "dependent", "index", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsIndexDependent), - to.TypeFilter(IsIndex), + from.TypeFilter(rulesVersionKey, isIndexDependent), + to.TypeFilter(rulesVersionKey, isIndex), JoinOnIndexID(from, to, "table-id", "index-id"), StatusesToPublicOrTransient(from, scpb.Status_PUBLIC, to, scpb.Status_PUBLIC), } diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_index_and_constraint.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_index_and_constraint.go index f47ddc272bef..7b34574a0b8b 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_index_and_constraint.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_add_index_and_constraint.go @@ -28,7 +28,7 @@ func init() { func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.Type((*scpb.PrimaryIndex)(nil)), - to.TypeFilter(IsSupportedNonIndexBackedConstraint), + to.TypeFilter(rulesVersionKey, isSupportedNonIndexBackedConstraint), JoinOnDescID(from, to, "table-id"), JoinOn( from, screl.IndexID, diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_column.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_column.go index 03927750fc4c..b1f179fc659c 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_column.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_column.go @@ -29,7 +29,7 @@ func init() { func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.Type((*scpb.Column)(nil)), - to.TypeFilter(IsColumnDependent), + to.TypeFilter(rulesVersionKey, isColumnDependent), JoinOnColumnID(from, to, "table-id", "col-id"), } }, @@ -42,7 +42,7 @@ func init() { scpb.Status_ABSENT, scpb.Status_ABSENT, func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsColumnDependent), + from.TypeFilter(rulesVersionKey, isColumnDependent), to.Type((*scpb.Column)(nil)), JoinOnColumnID(from, to, "table-id", "col-id"), } @@ -59,7 +59,7 @@ func init() { "dependent", "column-type", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsColumnTypeDependent), + from.TypeFilter(rulesVersionKey, isColumnTypeDependent), to.Type((*scpb.ColumnType)(nil)), JoinOnColumnID(from, to, "table-id", "col-id"), StatusesToAbsent(from, scpb.Status_ABSENT, to, scpb.Status_ABSENT), @@ -89,7 +89,7 @@ func init() { func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.Type((*scpb.ColumnType)(nil)), - from.DescriptorIsNotBeingDropped(), + descriptorIsNotBeingDropped(from.El), to.Type((*scpb.Column)(nil)), JoinOnColumnID(from, to, "table-id", "col-id"), StatusesToAbsent(from, scpb.Status_ABSENT, to, scpb.Status_ABSENT), diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_constraint.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_constraint.go index d62364898936..3a8984b67508 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_constraint.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_constraint.go @@ -31,8 +31,8 @@ func init() { scpb.Status_VALIDATED, scpb.Status_ABSENT, func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsSupportedNonIndexBackedConstraint), - to.TypeFilter(IsConstraintDependent), + from.TypeFilter(rulesVersionKey, isSupportedNonIndexBackedConstraint), + to.TypeFilter(rulesVersionKey, isConstraintDependent), JoinOnConstraintID(from, to, "table-id", "constraint-id"), } }, @@ -45,8 +45,8 @@ func init() { scpb.Status_ABSENT, scpb.Status_ABSENT, func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsConstraintDependent), - to.TypeFilter(IsSupportedNonIndexBackedConstraint), + from.TypeFilter(rulesVersionKey, isConstraintDependent), + to.TypeFilter(rulesVersionKey, isSupportedNonIndexBackedConstraint), JoinOnConstraintID(from, to, "table-id", "constraint-id"), } }, diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_index.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_index.go index beccaf2b4cea..baa9566e3d72 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_index.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_index.go @@ -28,8 +28,8 @@ func init() { scpb.Status_VALIDATED, scpb.Status_ABSENT, func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsIndex), - to.TypeFilter(IsIndexDependent), + from.TypeFilter(rulesVersionKey, isIndex), + to.TypeFilter(rulesVersionKey, isIndexDependent), JoinOnIndexID(from, to, "table-id", "index-id"), } }, @@ -41,8 +41,8 @@ func init() { scpb.Status_ABSENT, scpb.Status_ABSENT, func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsIndexDependent), - to.TypeFilter(IsIndex), + from.TypeFilter(rulesVersionKey, isIndexDependent), + to.TypeFilter(rulesVersionKey, isIndex), JoinOnIndexID(from, to, "table-id", "index-id"), } }, @@ -87,7 +87,7 @@ func init() { func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.Type((*scpb.IndexColumn)(nil)), - to.TypeFilter(IsIndex), + to.TypeFilter(rulesVersionKey, isIndex), JoinOnIndexID(from, to, "table-id", "index-id"), } }, @@ -113,7 +113,7 @@ func init() { func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.Type((*scpb.SecondaryIndexPartial)(nil)), - from.DescriptorIsNotBeingDropped(), + descriptorIsNotBeingDropped(from.El), to.Type((*scpb.SecondaryIndex)(nil)), JoinOnIndexID(from, to, "table-id", "index-id"), } diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_index_and_column.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_index_and_column.go index 10d05594e602..eaea144a4152 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_index_and_column.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_index_and_column.go @@ -36,7 +36,7 @@ func init() { to.Type((*scpb.Column)(nil)), ColumnInIndex(ic, from, relationID, columnID, "index-id"), JoinOnColumnID(ic, to, relationID, columnID), - ic.DescriptorIsNotBeingDropped(), + descriptorIsNotBeingDropped(ic.El), } }) @@ -52,7 +52,7 @@ func init() { ColumnInIndex(ic, from, relationID, columnID, "index-id"), JoinOnColumnID(ic, to, relationID, columnID), StatusesToAbsent(from, scpb.Status_VALIDATED, to, scpb.Status_WRITE_ONLY), - ic.DescriptorIsNotBeingDropped(), + descriptorIsNotBeingDropped(ic.El), rel.Filter("isIndexKeyColumnKey", ic.El)( func(ic *scpb.IndexColumn) bool { return ic.Kind == scpb.IndexColumn_KEY diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_object.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_object.go index 0dcfbe87c224..a38ade0ff73b 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_object.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_object.go @@ -34,7 +34,7 @@ func init() { "dropped", "absent", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsDescriptor), + from.TypeFilter(rulesVersionKey, isDescriptor), from.El.AttrEqVar(screl.DescID, "_"), from.El.AttrEqVar(rel.Self, to.El), StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), @@ -47,8 +47,8 @@ func init() { "descriptor", "dependent", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsDescriptor), - to.TypeFilter(IsSimpleDependent), + from.TypeFilter(rulesVersionKey, isDescriptor), + to.TypeFilter(rulesVersionKey, isSimpleDependent), JoinOnDescID(from, to, "desc-id"), StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), } @@ -61,7 +61,7 @@ func init() { func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.Type((*scpb.Table)(nil), (*scpb.View)(nil), (*scpb.Sequence)(nil)), - to.TypeFilter(IsColumn), + to.TypeFilter(rulesVersionKey, isColumn), JoinOnDescID(from, to, "desc-id"), StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_WRITE_ONLY), } @@ -74,7 +74,7 @@ func init() { func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.Type((*scpb.Table)(nil), (*scpb.View)(nil)), - to.TypeFilter(IsIndex), + to.TypeFilter(rulesVersionKey, isIndex), JoinOnDescID(from, to, "desc-id"), StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_VALIDATED), } @@ -88,7 +88,7 @@ func init() { func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.Type((*scpb.Table)(nil)), - to.TypeFilter(IsSupportedNonIndexBackedConstraint), + to.TypeFilter(rulesVersionKey, isSupportedNonIndexBackedConstraint), JoinOnDescID(from, to, "desc-id"), StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_WRITE_ONLY), } @@ -114,8 +114,8 @@ func init() { "referenced-descriptor", "referencing-via-attr", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsDescriptor), - to.TypeFilter(IsSimpleDependent), + from.TypeFilter(rulesVersionKey, isDescriptor), + to.TypeFilter(rulesVersionKey, isSimpleDependent), JoinReferencedDescID(to, from, "desc-id"), StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), } @@ -129,10 +129,10 @@ func init() { func(from, to NodeVars) rel.Clauses { fromDescID := rel.Var("fromDescID") return rel.Clauses{ - from.TypeFilter(IsTypeDescriptor), + from.TypeFilter(rulesVersionKey, isTypeDescriptor), from.DescIDEq(fromDescID), to.ReferencedTypeDescIDsContain(fromDescID), - to.TypeFilter(IsSimpleDependent, Or(IsWithTypeT, IsWithExpression)), + to.TypeFilter(rulesVersionKey, isSimpleDependent, Or(isWithTypeT, isWithExpression)), StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), } }, @@ -148,7 +148,7 @@ func init() { from.Type((*scpb.Sequence)(nil)), from.DescIDEq(seqID), to.ReferencedSequenceIDsContains(seqID), - to.TypeFilter(IsSimpleDependent, IsWithExpression), + to.TypeFilter(rulesVersionKey, isSimpleDependent, isWithExpression), StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), } }, diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_garbage_collection.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_garbage_collection.go index c447a070a1cf..300a83fb9084 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_garbage_collection.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_garbage_collection.go @@ -31,7 +31,7 @@ func init() { "table", "data", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsDescriptor), + from.TypeFilter(rulesVersionKey, isDescriptor), to.Type((*scpb.TableData)(nil)), JoinOnDescID(from, to, "table-id"), StatusesToAbsent(from, scpb.Status_ABSENT, to, scpb.Status_DROPPED), @@ -45,7 +45,7 @@ func init() { "database", "data", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsDescriptor), + from.TypeFilter(rulesVersionKey, isDescriptor), to.Type((*scpb.DatabaseData)(nil)), JoinOnDescID(from, to, "db-id"), StatusesToAbsent(from, scpb.Status_ABSENT, to, scpb.Status_DROPPED), @@ -60,7 +60,7 @@ func init() { scpb.Status_ABSENT, scpb.Status_DROPPED, func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsIndex), + from.TypeFilter(rulesVersionKey, isIndex), to.Type((*scpb.IndexData)(nil)), JoinOnIndexID(from, to, "table-id", "index-id"), } @@ -87,8 +87,8 @@ func init() { scpb.Status_DROPPED, scpb.Status_DROPPED, func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsData), - to.TypeFilter(IsData), + from.TypeFilter(rulesVersionKey, isData), + to.TypeFilter(rulesVersionKey, isData), JoinOnDescID(from, to, "desc-id"), FilterElements("SmallerIDsFirst", from, to, func(a, b scpb.Element) bool { aDescID, aIdxID := dataIDs(a) diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_two_version.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_two_version.go index 35700ed242d6..e2f77fa7a340 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_two_version.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_two_version.go @@ -69,7 +69,7 @@ func init() { from.Target.AttrEq(screl.TargetStatus, targetStatus.Status()), from.Node.AttrEq(screl.CurrentStatus, t.From()), to.Node.AttrEq(screl.CurrentStatus, t.To()), - from.DescriptorIsNotBeingDropped(), + descriptorIsNotBeingDropped(from.El), } if len(prePrevStatuses) > 0 { clauses = append(clauses, @@ -104,7 +104,7 @@ func init() { } } _ = ForEachElement(func(el scpb.Element) error { - if !IsSubjectTo2VersionInvariant(el) { + if !isSubjectTo2VersionInvariant(el) { return nil } if opgen.HasPublic(el) { diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/helpers.go b/pkg/sql/schemachanger/scplan/internal/rules/current/helpers.go new file mode 100644 index 000000000000..6700a6987fb8 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/helpers.go @@ -0,0 +1,224 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package current + +import ( + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl" + "github.com/cockroachdb/errors" +) + +const ( + // rulesVersion version of elements that can be appended to rel rule names. + rulesVersion = "-23.1" +) + +// rulesVersionKey version of elements used by this rule set. +var rulesVersionKey = clusterversion.V23_1 + +// descriptorIsNotBeingDropped creates a clause which leads to the outer clause +// failing to unify if the passed element is part of a descriptor and +// that descriptor is being dropped. +var descriptorIsNotBeingDropped = screl.Schema.DefNotJoin1( + "descriptorIsNotBeingDropped"+rulesVersion, "element", func( + element rel.Var, + ) rel.Clauses { + descriptor := rules.MkNodeVars("descriptor") + return rel.Clauses{ + descriptor.TypeFilter(rulesVersionKey, isDescriptor), + descriptor.JoinTarget(), + rules.JoinOnDescIDUntyped(descriptor.El, element, "id"), + descriptor.TargetStatus(scpb.ToAbsent), + } + }, +) + +// isDescriptor returns true for a descriptor-element, i.e. an element which +// owns its corresponding descriptor. +func isDescriptor(e scpb.Element) bool { + switch e.(type) { + case *scpb.Database, *scpb.Schema, *scpb.Table, *scpb.View, *scpb.Sequence, + *scpb.AliasType, *scpb.EnumType, *scpb.CompositeType: + return true + } + return false +} + +// IsDescriptor returns true for a descriptor-element, i.e. an element which +// owns its corresponding descriptor. This is only used for exports +func IsDescriptor(e scpb.Element) bool { + return isDescriptor(e) +} + +func isSubjectTo2VersionInvariant(e scpb.Element) bool { + // TODO(ajwerner): This should include constraints and enum values but it + // currently does not because we do not support dropping them unless we're + // dropping the descriptor and we do not support adding them. + return isIndex(e) || isColumn(e) || isSupportedNonIndexBackedConstraint(e) +} + +func isIndex(e scpb.Element) bool { + switch e.(type) { + case *scpb.PrimaryIndex, *scpb.SecondaryIndex, *scpb.TemporaryIndex: + return true + } + return false +} + +func isColumn(e scpb.Element) bool { + _, ok := e.(*scpb.Column) + return ok +} + +func isSimpleDependent(e scpb.Element) bool { + return !isDescriptor(e) && !isSubjectTo2VersionInvariant(e) && !isData(e) +} + +func getTypeT(element scpb.Element) (*scpb.TypeT, error) { + switch e := element.(type) { + case *scpb.ColumnType: + if e == nil { + return nil, nil + } + return &e.TypeT, nil + case *scpb.AliasType: + if e == nil { + return nil, nil + } + return &e.TypeT, nil + } + return nil, errors.AssertionFailedf("element %T does not have an embedded scpb.TypeT", element) +} + +func isWithTypeT(element scpb.Element) bool { + _, err := getTypeT(element) + return err == nil +} + +func getExpression(element scpb.Element) (*scpb.Expression, error) { + switch e := element.(type) { + case *scpb.ColumnType: + if e == nil { + return nil, nil + } + return e.ComputeExpr, nil + case *scpb.ColumnDefaultExpression: + if e == nil { + return nil, nil + } + return &e.Expression, nil + case *scpb.ColumnOnUpdateExpression: + if e == nil { + return nil, nil + } + return &e.Expression, nil + case *scpb.SecondaryIndexPartial: + if e == nil { + return nil, nil + } + return &e.Expression, nil + case *scpb.CheckConstraint: + if e == nil { + return nil, nil + } + return &e.Expression, nil + } + return nil, errors.AssertionFailedf("element %T does not have an embedded scpb.Expression", element) +} + +func isWithExpression(element scpb.Element) bool { + _, err := getExpression(element) + return err == nil +} + +func isTypeDescriptor(element scpb.Element) bool { + switch element.(type) { + case *scpb.EnumType, *scpb.AliasType, *scpb.CompositeType: + return true + default: + return false + } +} + +func isColumnDependent(e scpb.Element) bool { + switch e.(type) { + case *scpb.ColumnType: + return true + case *scpb.ColumnName, *scpb.ColumnComment, *scpb.IndexColumn: + return true + } + return isColumnTypeDependent(e) +} + +func isColumnTypeDependent(e scpb.Element) bool { + switch e.(type) { + case *scpb.SequenceOwner, *scpb.ColumnDefaultExpression, *scpb.ColumnOnUpdateExpression: + return true + } + return false +} + +func isIndexDependent(e scpb.Element) bool { + switch e.(type) { + case *scpb.IndexName, *scpb.IndexComment, *scpb.IndexColumn: + return true + case *scpb.IndexPartitioning, *scpb.SecondaryIndexPartial: + return true + } + return false +} + +// isSupportedNonIndexBackedConstraint a non-index-backed constraint is one of {Check, FK, UniqueWithoutIndex}. We only +// support Check for now. +// TODO (xiang): Expand this predicate to include other non-index-backed constraints +// when we properly support adding/dropping them in the new schema changer. +func isSupportedNonIndexBackedConstraint(e scpb.Element) bool { + switch e.(type) { + case *scpb.CheckConstraint, *scpb.ForeignKeyConstraint, *scpb.UniqueWithoutIndexConstraint: + return true + } + return false +} + +func isConstraint(e scpb.Element) bool { + switch e.(type) { + case *scpb.PrimaryIndex, *scpb.SecondaryIndex, *scpb.TemporaryIndex: + return true + case *scpb.CheckConstraint, *scpb.UniqueWithoutIndexConstraint, *scpb.ForeignKeyConstraint: + return true + } + return false +} + +func isConstraintDependent(e scpb.Element) bool { + switch e.(type) { + case *scpb.ConstraintWithoutIndexName: + return true + case *scpb.ConstraintComment: + return true + } + return false +} + +func isData(e scpb.Element) bool { + switch e.(type) { + case *scpb.DatabaseData: + return true + case *scpb.TableData: + return true + case *scpb.IndexData: + return true + } + return false +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/op_drop.go b/pkg/sql/schemachanger/scplan/internal/rules/current/op_drop.go index b8b727659244..b8a801347cd4 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/op_drop.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/op_drop.go @@ -130,7 +130,7 @@ func init() { (*scpb.Table)(nil), (*scpb.View)(nil), ), - index.TypeFilter(IsIndex), + index.TypeFilter(rulesVersionKey, isIndex), dep.Type( (*scpb.IndexName)(nil), (*scpb.IndexPartitioning)(nil), @@ -234,7 +234,7 @@ func init() { "skip element removal ops on descriptor drop", dep.Node, screl.MustQuery( - desc.TypeFilter(IsDescriptor), + desc.TypeFilter(rulesVersionKey, isDescriptor), dep.Type( (*scpb.ColumnFamily)(nil), (*scpb.Owner)(nil), diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/op_index_and_column.go b/pkg/sql/schemachanger/scplan/internal/rules/current/op_index_and_column.go index b9ffc1209abe..16d822599cb8 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/op_index_and_column.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/op_index_and_column.go @@ -29,7 +29,7 @@ func init() { ic.Node, screl.MustQuery( ic.Type((*scpb.IndexColumn)(nil)), - index.TypeFilter(IsIndex), + index.TypeFilter(rulesVersionKey, isIndex), JoinOnIndexID(ic, index, relationID, indexID), ic.JoinTargetNode(), ic.TargetStatus(scpb.ToAbsent, scpb.Transient), diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules index fe686ba878bb..b442398cb6cb 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules @@ -9,15 +9,15 @@ ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id): ColumnInSwappedInPrimaryIndex($index-column, $index, $table-id, $column-id, $index-id): - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) - sourceIndexIsSet($index) -DescriptorIsNotBeingDropped($element): +ToPublicOrTransient($target1, $target2): + - $target1[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] + - $target2[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] +descriptorIsNotBeingDropped-23.1($element): not-join: - $descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] - joinTarget($descriptor, $descriptor-Target) - joinOnDescID($descriptor, $element, $id) - $descriptor-Target[TargetStatus] = ABSENT -ToPublicOrTransient($target1, $target2): - - $target1[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] - - $target2[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] joinOnColumnID($a, $b, $desc-id, $col-id): - joinOnDescID($a, $b, $desc-id) - $a[ColumnID] = $col-id @@ -113,7 +113,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'CheckConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT' @@ -129,7 +129,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = ABSENT - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - nodeNotExistsWithStatusIn_WRITE_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -146,7 +146,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'CheckConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY' @@ -162,7 +162,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'CheckConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' @@ -178,7 +178,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'CheckConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' @@ -194,7 +194,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' @@ -210,7 +210,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = ABSENT - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY' @@ -226,7 +226,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY' @@ -242,7 +242,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to PUBLIC uphold 2-version invariant: ABSENT->DELETE_ONLY' @@ -258,7 +258,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY' @@ -274,7 +274,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC' @@ -290,7 +290,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = PUBLIC - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: DEFAULT or ON UPDATE existence precedes writes to column @@ -319,7 +319,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ForeignKeyConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT' @@ -335,7 +335,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = ABSENT - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - nodeNotExistsWithStatusIn_WRITE_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -352,7 +352,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY' @@ -368,7 +368,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' @@ -384,7 +384,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' @@ -400,7 +400,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY' @@ -416,7 +416,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY' @@ -432,7 +432,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = BACKFILL_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' @@ -448,7 +448,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = ABSENT - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - nodeNotExistsWithStatusIn_TRANSIENT_DELETE_ONLY_BACKFILLED_TRANSIENT_BACKFILLED_BACKFILL_ONLY_TRANSIENT_BACKFILL_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -465,7 +465,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY' @@ -481,7 +481,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = MERGE_ONLY - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' @@ -497,7 +497,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_ABSENT->ABSENT' @@ -513,7 +513,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_ABSENT - $next-Node[CurrentStatus] = ABSENT - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_BACKFILLED->DELETE_ONLY' @@ -529,7 +529,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_BACKFILL_ONLY->DELETE_ONLY' @@ -545,7 +545,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_BACKFILL_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->DELETE_ONLY' @@ -561,7 +561,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_MERGED->WRITE_ONLY' @@ -577,7 +577,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_MERGE_ONLY->WRITE_ONLY' @@ -593,7 +593,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_MERGE_ONLY - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_VALIDATED->VALIDATED' @@ -609,7 +609,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_VALIDATED - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_WRITE_ONLY->WRITE_ONLY' @@ -625,7 +625,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_WRITE_ONLY - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY' @@ -641,7 +641,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - nodeNotExistsWithStatusIn_TRANSIENT_VALIDATED($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -658,7 +658,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - nodeNotExistsWithStatusIn_VALIDATED_TRANSIENT_WRITE_ONLY_MERGE_ONLY_TRANSIENT_MERGE_ONLY_MERGED_TRANSIENT_MERGED($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -675,7 +675,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = BACKFILL_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY' @@ -691,7 +691,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED' @@ -707,7 +707,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = BACKFILL_ONLY - $next-Node[CurrentStatus] = BACKFILLED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY' @@ -723,7 +723,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = MERGE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY' @@ -739,7 +739,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED' @@ -755,7 +755,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = MERGE_ONLY - $next-Node[CurrentStatus] = MERGED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' @@ -771,7 +771,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' @@ -787,7 +787,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->BACKFILL_ONLY' @@ -803,7 +803,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = BACKFILL_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY' @@ -819,7 +819,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED' @@ -835,7 +835,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = BACKFILL_ONLY - $next-Node[CurrentStatus] = BACKFILLED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY' @@ -851,7 +851,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = MERGE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: MERGED->WRITE_ONLY' @@ -867,7 +867,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: MERGE_ONLY->MERGED' @@ -883,7 +883,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = MERGE_ONLY - $next-Node[CurrentStatus] = MERGED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: PUBLIC->TRANSIENT_VALIDATED' @@ -899,7 +899,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = TRANSIENT_VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_BACKFILLED->TRANSIENT_DELETE_ONLY' @@ -915,7 +915,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_BACKFILLED - $next-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_BACKFILL_ONLY->TRANSIENT_DELETE_ONLY' @@ -931,7 +931,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_BACKFILL_ONLY - $next-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT' @@ -947,7 +947,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY - $next-Node[CurrentStatus] = TRANSIENT_ABSENT - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - nodeNotExistsWithStatusIn_TRANSIENT_BACKFILLED_TRANSIENT_BACKFILL_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -964,7 +964,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_MERGED - $next-Node[CurrentStatus] = TRANSIENT_WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_MERGE_ONLY->TRANSIENT_WRITE_ONLY' @@ -980,7 +980,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_MERGE_ONLY - $next-Node[CurrentStatus] = TRANSIENT_WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_VALIDATED->TRANSIENT_WRITE_ONLY' @@ -996,7 +996,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_VALIDATED - $next-Node[CurrentStatus] = TRANSIENT_WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_WRITE_ONLY->TRANSIENT_DELETE_ONLY' @@ -1012,7 +1012,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_WRITE_ONLY - $next-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - nodeNotExistsWithStatusIn_TRANSIENT_VALIDATED_TRANSIENT_MERGE_ONLY_TRANSIENT_MERGED($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -1029,7 +1029,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED' @@ -1045,7 +1045,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY' @@ -1061,7 +1061,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY' @@ -1077,7 +1077,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = BACKFILL_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' @@ -1093,7 +1093,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = ABSENT - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - nodeNotExistsWithStatusIn_BACKFILLED_BACKFILL_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -1110,7 +1110,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY' @@ -1126,7 +1126,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = MERGE_ONLY - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' @@ -1142,7 +1142,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY' @@ -1158,7 +1158,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY' @@ -1174,7 +1174,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - nodeNotExistsWithStatusIn_VALIDATED_MERGE_ONLY_MERGED($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -1191,7 +1191,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = BACKFILL_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY' @@ -1207,7 +1207,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED' @@ -1223,7 +1223,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = BACKFILL_ONLY - $next-Node[CurrentStatus] = BACKFILLED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY' @@ -1239,7 +1239,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = MERGE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY' @@ -1255,7 +1255,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED' @@ -1271,7 +1271,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = MERGE_ONLY - $next-Node[CurrentStatus] = MERGED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' @@ -1287,7 +1287,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' @@ -1303,7 +1303,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' @@ -1319,7 +1319,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = ABSENT - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - nodeNotExistsWithStatusIn_TRANSIENT_DELETE_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -1336,7 +1336,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_ABSENT - $next-Node[CurrentStatus] = ABSENT - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->DELETE_ONLY' @@ -1352,7 +1352,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY' @@ -1368,7 +1368,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY' @@ -1384,7 +1384,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY' @@ -1400,7 +1400,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT' @@ -1416,7 +1416,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY - $next-Node[CurrentStatus] = TRANSIENT_ABSENT - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY' @@ -1432,7 +1432,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' @@ -1448,7 +1448,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT' @@ -1464,7 +1464,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = ABSENT - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - nodeNotExistsWithStatusIn_WRITE_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -1481,7 +1481,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY' @@ -1497,7 +1497,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' @@ -1513,7 +1513,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' @@ -1529,7 +1529,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-23.1($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: column dependents exist before column becomes public @@ -1686,7 +1686,7 @@ deprules to: column-Node query: - $column-type[Type] = '*scpb.ColumnType' - - DescriptorIsNotBeingDropped($column-type) + - descriptorIsNotBeingDropped-23.1($column-type) - $column[Type] = '*scpb.Column' - joinOnColumnID($column-type, $column, $table-id, $col-id) - toAbsent($column-type-Target, $column-Target) @@ -2378,7 +2378,7 @@ deprules - $column[Type] = '*scpb.Column' - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) - joinOnColumnID($index-column, $column, $table-id, $column-id) - - DescriptorIsNotBeingDropped($index-column) + - descriptorIsNotBeingDropped-23.1($index-column) - toAbsent($index-Target, $column-Target) - $index-Node[CurrentStatus] = ABSENT - $column-Node[CurrentStatus] = ABSENT @@ -2393,7 +2393,7 @@ deprules - $column[Type] = '*scpb.Column' - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) - joinOnColumnID($index-column, $column, $table-id, $column-id) - - DescriptorIsNotBeingDropped($index-column) + - descriptorIsNotBeingDropped-23.1($index-column) - transient($index-Target, $column-Target) - $index-Node[CurrentStatus] = TRANSIENT_ABSENT - $column-Node[CurrentStatus] = TRANSIENT_ABSENT @@ -2408,7 +2408,7 @@ deprules - $column[Type] = '*scpb.Column' - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) - joinOnColumnID($index-column, $column, $table-id, $column-id) - - DescriptorIsNotBeingDropped($index-column) + - descriptorIsNotBeingDropped-23.1($index-column) - $index-Target[TargetStatus] = TRANSIENT_ABSENT - $index-Node[CurrentStatus] = TRANSIENT_ABSENT - $column-Target[TargetStatus] = ABSENT @@ -2424,7 +2424,7 @@ deprules - $column[Type] = '*scpb.Column' - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) - joinOnColumnID($index-column, $column, $table-id, $column-id) - - DescriptorIsNotBeingDropped($index-column) + - descriptorIsNotBeingDropped-23.1($index-column) - $index-Target[TargetStatus] = ABSENT - $index-Node[CurrentStatus] = ABSENT - $column-Target[TargetStatus] = TRANSIENT_ABSENT @@ -2457,7 +2457,7 @@ deprules to: index-Node query: - $partial-predicate[Type] = '*scpb.SecondaryIndexPartial' - - DescriptorIsNotBeingDropped($partial-predicate) + - descriptorIsNotBeingDropped-23.1($partial-predicate) - $index[Type] = '*scpb.SecondaryIndex' - joinOnIndexID($partial-predicate, $index, $table-id, $index-id) - toAbsent($partial-predicate-Target, $index-Target) @@ -2471,7 +2471,7 @@ deprules to: index-Node query: - $partial-predicate[Type] = '*scpb.SecondaryIndexPartial' - - DescriptorIsNotBeingDropped($partial-predicate) + - descriptorIsNotBeingDropped-23.1($partial-predicate) - $index[Type] = '*scpb.SecondaryIndex' - joinOnIndexID($partial-predicate, $index, $table-id, $index-id) - transient($partial-predicate-Target, $index-Target) @@ -2485,7 +2485,7 @@ deprules to: index-Node query: - $partial-predicate[Type] = '*scpb.SecondaryIndexPartial' - - DescriptorIsNotBeingDropped($partial-predicate) + - descriptorIsNotBeingDropped-23.1($partial-predicate) - $index[Type] = '*scpb.SecondaryIndex' - joinOnIndexID($partial-predicate, $index, $table-id, $index-id) - $partial-predicate-Target[TargetStatus] = TRANSIENT_ABSENT @@ -2500,7 +2500,7 @@ deprules to: index-Node query: - $partial-predicate[Type] = '*scpb.SecondaryIndexPartial' - - DescriptorIsNotBeingDropped($partial-predicate) + - descriptorIsNotBeingDropped-23.1($partial-predicate) - $index[Type] = '*scpb.SecondaryIndex' - joinOnIndexID($partial-predicate, $index, $table-id, $index-id) - $partial-predicate-Target[TargetStatus] = ABSENT @@ -2830,7 +2830,7 @@ deprules - toAbsent($index-Target, $column-Target) - $index-Node[CurrentStatus] = VALIDATED - $column-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($index-column) + - descriptorIsNotBeingDropped-23.1($index-column) - isIndexKeyColumnKey(*scpb.IndexColumn)($index-column) - joinTargetNode($index, $index-Target, $index-Node) - joinTargetNode($column, $column-Target, $column-Node) diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/oprules b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/oprules index b9054d7fcf8a..622bd5bcec4e 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/oprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/oprules @@ -9,15 +9,15 @@ ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id): ColumnInSwappedInPrimaryIndex($index-column, $index, $table-id, $column-id, $index-id): - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) - sourceIndexIsSet($index) -DescriptorIsNotBeingDropped($element): +ToPublicOrTransient($target1, $target2): + - $target1[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] + - $target2[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] +descriptorIsNotBeingDropped-23.1($element): not-join: - $descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] - joinTarget($descriptor, $descriptor-Target) - joinOnDescID($descriptor, $element, $id) - $descriptor-Target[TargetStatus] = ABSENT -ToPublicOrTransient($target1, $target2): - - $target1[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] - - $target2[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] joinOnColumnID($a, $b, $desc-id, $col-id): - joinOnDescID($a, $b, $desc-id) - $a[ColumnID] = $col-id diff --git a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go index e389328e4a44..e26ac11c2f6a 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go @@ -85,7 +85,7 @@ func StatusesTransient( } func JoinOnDescID(a, b NodeVars, descriptorIDVar rel.Var) rel.Clause { - return joinOnDescIDUntyped(a.El, b.El, descriptorIDVar) + return JoinOnDescIDUntyped(a.El, b.El, descriptorIDVar) } func JoinReferencedDescID(a, b NodeVars, descriptorIDVar rel.Var) rel.Clause { @@ -156,7 +156,7 @@ var ( referenced.AttrEqVar(screl.DescID, id), } }) - joinOnDescIDUntyped = screl.Schema.Def3( + JoinOnDescIDUntyped = screl.Schema.Def3( "joinOnDescID", "a", "b", "id", func( a, b, id rel.Var, ) rel.Clauses { @@ -169,7 +169,7 @@ var ( a, b, descID, indexID rel.Var, ) rel.Clauses { return rel.Clauses{ - joinOnDescIDUntyped(a, b, descID), + JoinOnDescIDUntyped(a, b, descID), indexID.Entities(screl.IndexID, a, b), } }, @@ -179,7 +179,7 @@ var ( a, b, descID, colID rel.Var, ) rel.Clauses { return rel.Clauses{ - joinOnDescIDUntyped(a, b, descID), + JoinOnDescIDUntyped(a, b, descID), colID.Entities(screl.ColumnID, a, b), } }, @@ -189,7 +189,7 @@ var ( a, b, descID, constraintID rel.Var, ) rel.Clauses { return rel.Clauses{ - joinOnDescIDUntyped(a, b, descID), + JoinOnDescIDUntyped(a, b, descID), constraintID.Entities(screl.ConstraintID, a, b), } }, @@ -244,7 +244,7 @@ func ForEachElement(fn func(element scpb.Element) error) error { } func ForEachElementInActiveVersion( - fn func(element scpb.Element) error, version clusterversion.ClusterVersion, + version clusterversion.ClusterVersion, fn func(element scpb.Element) error, ) error { var ep scpb.ElementProto vep := reflect.ValueOf(ep) @@ -259,179 +259,6 @@ func ForEachElementInActiveVersion( return nil } -// IsDescriptor returns true for a descriptor-element, i.e. an element which -// owns its corresponding descriptor. -func IsDescriptor(e scpb.Element) bool { - switch e.(type) { - case *scpb.Database, *scpb.Schema, *scpb.Table, *scpb.View, *scpb.Sequence, - *scpb.AliasType, *scpb.EnumType, *scpb.CompositeType: - return true - } - return false -} - -func IsSubjectTo2VersionInvariant(e scpb.Element) bool { - // TODO(ajwerner): This should include constraints and enum values but it - // currently does not because we do not support dropping them unless we're - // dropping the descriptor and we do not support adding them. - return IsIndex(e) || IsColumn(e) || IsSupportedNonIndexBackedConstraint(e) -} - -func IsIndex(e scpb.Element) bool { - switch e.(type) { - case *scpb.PrimaryIndex, *scpb.SecondaryIndex, *scpb.TemporaryIndex: - return true - } - return false -} - -func IsColumn(e scpb.Element) bool { - _, ok := e.(*scpb.Column) - return ok -} - -func IsSimpleDependent(e scpb.Element) bool { - return !IsDescriptor(e) && !IsSubjectTo2VersionInvariant(e) && !IsData(e) -} - -func getTypeT(element scpb.Element) (*scpb.TypeT, error) { - switch e := element.(type) { - case *scpb.ColumnType: - if e == nil { - return nil, nil - } - return &e.TypeT, nil - case *scpb.AliasType: - if e == nil { - return nil, nil - } - return &e.TypeT, nil - } - return nil, errors.AssertionFailedf("element %T does not have an embedded scpb.TypeT", element) -} - -func IsWithTypeT(element scpb.Element) bool { - _, err := getTypeT(element) - return err == nil -} - -func getExpression(element scpb.Element) (*scpb.Expression, error) { - switch e := element.(type) { - case *scpb.ColumnType: - if e == nil { - return nil, nil - } - return e.ComputeExpr, nil - case *scpb.ColumnDefaultExpression: - if e == nil { - return nil, nil - } - return &e.Expression, nil - case *scpb.ColumnOnUpdateExpression: - if e == nil { - return nil, nil - } - return &e.Expression, nil - case *scpb.SecondaryIndexPartial: - if e == nil { - return nil, nil - } - return &e.Expression, nil - case *scpb.CheckConstraint: - if e == nil { - return nil, nil - } - return &e.Expression, nil - } - return nil, errors.AssertionFailedf("element %T does not have an embedded scpb.Expression", element) -} - -func IsWithExpression(element scpb.Element) bool { - _, err := getExpression(element) - return err == nil -} - -func IsTypeDescriptor(element scpb.Element) bool { - switch element.(type) { - case *scpb.EnumType, *scpb.AliasType, *scpb.CompositeType: - return true - default: - return false - } -} - -func IsColumnDependent(e scpb.Element) bool { - switch e.(type) { - case *scpb.ColumnType: - return true - case *scpb.ColumnName, *scpb.ColumnComment, *scpb.IndexColumn: - return true - } - return IsColumnTypeDependent(e) -} - -func IsColumnTypeDependent(e scpb.Element) bool { - switch e.(type) { - case *scpb.SequenceOwner, *scpb.ColumnDefaultExpression, *scpb.ColumnOnUpdateExpression: - return true - } - return false -} - -func IsIndexDependent(e scpb.Element) bool { - switch e.(type) { - case *scpb.IndexName, *scpb.IndexComment, *scpb.IndexColumn: - return true - case *scpb.IndexPartitioning, *scpb.SecondaryIndexPartial: - return true - } - return false -} - -// IsSupportedNonIndexBackedConstraint a non-index-backed constraint is one of {Check, FK, UniqueWithoutIndex}. We only -// support Check for now. -// TODO (xiang): Expand this predicate to include other non-index-backed constraints -// when we properly support adding/dropping them in the new schema changer. -func IsSupportedNonIndexBackedConstraint(e scpb.Element) bool { - switch e.(type) { - case *scpb.CheckConstraint, *scpb.ForeignKeyConstraint, *scpb.UniqueWithoutIndexConstraint: - return true - } - return false -} - -func IsConstraint(e scpb.Element) bool { - switch e.(type) { - case *scpb.PrimaryIndex, *scpb.SecondaryIndex, *scpb.TemporaryIndex: - return true - case *scpb.CheckConstraint, *scpb.UniqueWithoutIndexConstraint, *scpb.ForeignKeyConstraint: - return true - } - return false -} - -func IsConstraintDependent(e scpb.Element) bool { - switch e.(type) { - case *scpb.ConstraintWithoutIndexName: - return true - case *scpb.ConstraintComment: - return true - } - return false -} - -func IsData(e scpb.Element) bool { - switch e.(type) { - case *scpb.DatabaseData: - return true - case *scpb.TableData: - return true - case *scpb.IndexData: - return true - } - return false -} - type elementTypePredicate = func(e scpb.Element) bool func Or(predicates ...elementTypePredicate) elementTypePredicate { @@ -507,23 +334,6 @@ func RegisterDepRuleForDrop( }) } -// descriptorIsNotBeingDropped creates a clause which leads to the outer clause -// failing to unify if the passed element is part of a descriptor and -// that descriptor is being dropped. -var descriptorIsNotBeingDropped = screl.Schema.DefNotJoin1( - "DescriptorIsNotBeingDropped", "element", func( - element rel.Var, - ) rel.Clauses { - descriptor := MkNodeVars("descriptor") - return rel.Clauses{ - descriptor.TypeFilter(IsDescriptor), - descriptor.JoinTarget(), - joinOnDescIDUntyped(descriptor.El, element, "id"), - descriptor.TargetStatus(scpb.ToAbsent), - } - }, -) - // notJoinOnNodeWithStatusIn is a cache to memoize getNotJoinOnNodeWithStatusIn. var notJoinOnNodeWithStatusIn = map[string]rel.Rule1{} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/registry.go b/pkg/sql/schemachanger/scplan/internal/rules/registry.go index d5833dcb99d3..6368472d650f 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/registry.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/registry.go @@ -18,6 +18,7 @@ import ( "context" "sort" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph" @@ -225,12 +226,15 @@ func (v NodeVars) Type(valuesForTypeOf ...interface{}) rel.Clause { // TypeFilter returns a Type clause which binds the element var to elements of // a specific type, filtered by the conjunction of all provided predicates. -func (v NodeVars) TypeFilter(predicatesForTypeOf ...func(element scpb.Element) bool) rel.Clause { +func (v NodeVars) TypeFilter( + version clusterversion.Key, predicatesForTypeOf ...func(element scpb.Element) bool, +) rel.Clause { if len(predicatesForTypeOf) == 0 { panic(errors.AssertionFailedf("empty type predicate for %q", v.El)) } + cv := clusterversion.ClusterVersion{Version: clusterversion.ByKey(version)} var valuesForTypeOf []interface{} - _ = ForEachElement(func(e scpb.Element) error { + _ = ForEachElementInActiveVersion(cv, func(e scpb.Element) error { for _, p := range predicatesForTypeOf { if !p(e) { return nil @@ -261,12 +265,6 @@ func (v NodeVars) ReferencedSequenceIDsContains(containedIDVar rel.Var) rel.Clau return v.El.AttrContainsVar(screl.ReferencedSequenceIDs, containedIDVar) } -// DescriptorIsNotBeingDropped is a type-safe shorthand to invoke the -// rule of the same name on the element. -func (v NodeVars) DescriptorIsNotBeingDropped() rel.Clause { - return descriptorIsNotBeingDropped(v.El) -} - func MkNodeVars(elStr string) NodeVars { el := rel.Var(elStr) return NodeVars{ diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/BUILD.bazel b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/BUILD.bazel index 5228fe6f6523..b1998d175ce9 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/BUILD.bazel +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "dep_drop_object.go", "dep_swap_index.go", "dep_two_version.go", + "helpers.go", "op_drop.go", "op_index_and_column.go", "registry.go", @@ -23,12 +24,14 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules/release_22_2", visibility = ["//pkg/sql/schemachanger/scplan:__subpackages__"], deps = [ + "//pkg/clusterversion", "//pkg/sql/schemachanger/rel", "//pkg/sql/schemachanger/scpb", "//pkg/sql/schemachanger/scplan/internal/opgen", "//pkg/sql/schemachanger/scplan/internal/rules", "//pkg/sql/schemachanger/scplan/internal/scgraph", "//pkg/sql/schemachanger/screl", + "@com_github_cockroachdb_errors//:errors", ], ) diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/assertions_test.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/assertions_test.go index 0227d2439ef8..dc1f6318f69c 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/assertions_test.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/assertions_test.go @@ -44,13 +44,13 @@ func TestRuleAssertions(t *testing.T) { nameParts := strings.Split(fullName, "rules.") shortName := nameParts[len(nameParts)-1] t.Run(shortName, func(t *testing.T) { - _ = ForEachElementInActiveVersion(func(e scpb.Element) error { + _ = ForEachElementInActiveVersion(version, func(e scpb.Element) error { e = nonNilElement(e) if err := fn(e); err != nil { t.Errorf("%T: %+v", e, err) } return nil - }, version) + }) }) } } @@ -62,7 +62,7 @@ func nonNilElement(element scpb.Element) scpb.Element { // Assert that only simple dependents (non-descriptor, non-index, non-column) // have screl.ReferencedDescID attributes. func checkSimpleDependentsReferenceDescID(e scpb.Element) error { - if IsSimpleDependent(e) { + if isSimpleDependent(e) { return nil } if _, ok := e.(*scpb.ForeignKeyConstraint); ok { @@ -85,15 +85,15 @@ func checkToAbsentCategories(e scpb.Element) error { s1 := opgen.NextStatus(e, scpb.Status_ABSENT, s0) switch s1 { case scpb.Status_TXN_DROPPED, scpb.Status_DROPPED: - if IsDescriptor(e) { + if isDescriptor(e) { return nil } case scpb.Status_VALIDATED, scpb.Status_WRITE_ONLY, scpb.Status_DELETE_ONLY: - if IsSubjectTo2VersionInvariant(e) { + if isSubjectTo2VersionInvariant(e) || isSupportedNonIndexBackedConstraint(e) { return nil } case scpb.Status_ABSENT: - if IsSimpleDependent(e) { + if isSimpleDependent(e) { return nil } } @@ -103,7 +103,7 @@ func checkToAbsentCategories(e scpb.Element) error { // Assert that isWithTypeT covers all elements with embedded TypeTs. func checkIsWithTypeT(e scpb.Element) error { return screl.WalkTypes(e, func(t *types.T) error { - if IsWithTypeT(e) { + if isWithTypeT(e) { return nil } return errors.New("should verify isWithTypeT but doesn't") @@ -120,37 +120,37 @@ func checkIsWithExpression(e scpb.Element) error { case *scpb.RowLevelTTL: return nil } - if IsWithExpression(e) { + if isWithExpression(e) { return nil } return errors.New("should verify isWithExpression but doesn't") }) } -// Assert that rules.IsColumnDependent covers all dependent elements of a column +// Assert that rules.isColumnDependent covers all dependent elements of a column // element. func checkIsColumnDependent(e scpb.Element) error { // Exclude columns themselves. - if IsColumn(e) { + if isColumn(e) { return nil } // A column dependent should have a ColumnID attribute. _, err := screl.Schema.GetAttribute(screl.ColumnID, e) - if IsColumnDependent(e) { + if isColumnDependent(e) { if err != nil { - return errors.New("verifies rules.IsColumnDependent but doesn't have ColumnID attr") + return errors.New("verifies rules.isColumnDependent but doesn't have ColumnID attr") } } else if err == nil { - return errors.New("has ColumnID attr but doesn't verify rules.IsColumnDependent") + return errors.New("has ColumnID attr but doesn't verify rules.isColumnDependent") } return nil } -// Assert that rules.IsIndexDependent covers all dependent elements of an index +// Assert that rules.isIndexDependent covers all dependent elements of an index // element. func checkIsIndexDependent(e scpb.Element) error { // Exclude indexes themselves. - if IsIndex(e) || IsSupportedNonIndexBackedConstraint(e) { + if IsIndex(e) || isSupportedNonIndexBackedConstraint(e) { return nil } // Skip check constraints, in 22.2 these didn't have @@ -160,12 +160,12 @@ func checkIsIndexDependent(e scpb.Element) error { } // An index dependent should have an IndexID attribute. _, err := screl.Schema.GetAttribute(screl.IndexID, e) - if IsIndexDependent(e) { + if isIndexDependent(e) { if err != nil { - return errors.New("verifies rules.IsIndexDependent but doesn't have IndexID attr") + return errors.New("verifies rules.isIndexDependent but doesn't have IndexID attr") } } else if err == nil { - return errors.New("has IndexID attr but doesn't verify rules.IsIndexDependent") + return errors.New("has IndexID attr but doesn't verify rules.isIndexDependent") } return nil } @@ -174,17 +174,17 @@ func checkIsIndexDependent(e scpb.Element) error { // element. func checkIsConstraintDependent(e scpb.Element) error { // Exclude constraints themselves. - if IsConstraint(e) { + if isConstraint(e) { return nil } // A constraint dependent should have a ConstraintID attribute. _, err := screl.Schema.GetAttribute(screl.ConstraintID, e) - if IsConstraintDependent(e) { + if isConstraintDependent(e) { if err != nil { - return errors.New("verifies rules.IsConstraintDependent but doesn't have ConstraintID attr") + return errors.New("verifies rules.isConstraintDependent but doesn't have ConstraintID attr") } } else if err == nil { - return errors.New("has ConstraintID attr but doesn't verify rules.IsConstraintDependent") + return errors.New("has ConstraintID attr but doesn't verify rules.isConstraintDependent") } return nil } diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_column.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_column.go index 6eefc4f221da..1149dd21d7fe 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_column.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_column.go @@ -29,7 +29,7 @@ func init() { func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.Type((*scpb.Column)(nil)), - to.TypeFilter(IsColumnDependent), + to.TypeFilter(rulesVersionKey, isColumnDependent), JoinOnColumnID(from, to, "table-id", "col-id"), StatusesToPublicOrTransient(from, scpb.Status_DELETE_ONLY, to, scpb.Status_PUBLIC), } @@ -42,7 +42,7 @@ func init() { "dependent", "column", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsColumnDependent), + from.TypeFilter(rulesVersionKey, isColumnDependent), to.Type((*scpb.Column)(nil)), JoinOnColumnID(from, to, "table-id", "col-id"), StatusesToPublicOrTransient(from, scpb.Status_PUBLIC, to, scpb.Status_PUBLIC), diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_constraint.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_constraint.go index 44e42209d8c8..a034b0dd3ac6 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_constraint.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_constraint.go @@ -27,8 +27,8 @@ func init() { "constraint", "dependent", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsConstraint), - to.TypeFilter(IsConstraintDependent), + from.TypeFilter(rulesVersionKey, isConstraint), + to.TypeFilter(rulesVersionKey, isConstraintDependent), JoinOnConstraintID(from, to, "table-id", "constraint-id"), StatusesToPublicOrTransient(from, scpb.Status_PUBLIC, to, scpb.Status_PUBLIC), } diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_index.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_index.go index 011b82d5a2e6..9be2f88109fa 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_index.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_add_index.go @@ -32,7 +32,7 @@ func init() { (*scpb.PrimaryIndex)(nil), (*scpb.SecondaryIndex)(nil), ), - to.TypeFilter(IsIndexDependent), + to.TypeFilter(rulesVersionKey, isIndexDependent), JoinOnIndexID(from, to, "table-id", "index-id"), StatusesToPublicOrTransient(from, scpb.Status_BACKFILL_ONLY, to, scpb.Status_PUBLIC), } @@ -46,7 +46,7 @@ func init() { func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.Type((*scpb.TemporaryIndex)(nil)), - to.TypeFilter(IsIndexDependent), + to.TypeFilter(rulesVersionKey, isIndexDependent), JoinOnIndexID(from, to, "table-id", "index-id"), StatusesToPublicOrTransient(from, scpb.Status_DELETE_ONLY, to, scpb.Status_PUBLIC), } @@ -59,8 +59,8 @@ func init() { "dependent", "index", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsIndexDependent), - to.TypeFilter(IsIndex), + from.TypeFilter(rulesVersionKey, isIndexDependent), + to.TypeFilter(rulesVersionKey, IsIndex), JoinOnIndexID(from, to, "table-id", "index-id"), StatusesToPublicOrTransient(from, scpb.Status_PUBLIC, to, scpb.Status_PUBLIC), } diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_column.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_column.go index dbb18eafbb14..c54cf575bd98 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_column.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_column.go @@ -29,7 +29,7 @@ func init() { func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.Type((*scpb.Column)(nil)), - to.TypeFilter(IsColumnDependent), + to.TypeFilter(rulesVersionKey, isColumnDependent), JoinOnColumnID(from, to, "table-id", "col-id"), } }, @@ -42,7 +42,7 @@ func init() { scpb.Status_ABSENT, scpb.Status_ABSENT, func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsColumnDependent), + from.TypeFilter(rulesVersionKey, isColumnDependent), to.Type((*scpb.Column)(nil)), JoinOnColumnID(from, to, "table-id", "col-id"), } @@ -59,7 +59,7 @@ func init() { "dependent", "column-type", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsColumnTypeDependent), + from.TypeFilter(rulesVersionKey, isColumnTypeDependent), to.Type((*scpb.ColumnType)(nil)), JoinOnColumnID(from, to, "table-id", "col-id"), StatusesToAbsent(from, scpb.Status_ABSENT, to, scpb.Status_ABSENT), @@ -89,7 +89,7 @@ func init() { func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.Type((*scpb.ColumnType)(nil)), - from.DescriptorIsNotBeingDropped(), + descriptorIsNotBeingDropped(from.El), to.Type((*scpb.Column)(nil)), JoinOnColumnID(from, to, "table-id", "col-id"), StatusesToAbsent(from, scpb.Status_ABSENT, to, scpb.Status_ABSENT), diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_constraint.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_constraint.go index cef8c6e7b089..30d73e248c86 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_constraint.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_constraint.go @@ -28,8 +28,8 @@ func init() { scpb.Status_ABSENT, scpb.Status_ABSENT, func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsConstraintDependent), - to.TypeFilter(IsConstraint, Not(IsIndex)), + from.TypeFilter(rulesVersionKey, isConstraintDependent), + to.TypeFilter(rulesVersionKey, isConstraint, Not(IsIndex)), JoinOnConstraintID(from, to, "table-id", "constraint-id"), } }, @@ -42,8 +42,8 @@ func init() { scpb.Status_VALIDATED, scpb.Status_ABSENT, func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsConstraintDependent), - to.TypeFilter(IsConstraint, IsIndex), + from.TypeFilter(rulesVersionKey, isConstraintDependent), + to.TypeFilter(rulesVersionKey, isConstraint, IsIndex), JoinOnConstraintID(from, to, "table-id", "constraint-id"), } }, diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_index.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_index.go index 9cc37b424752..858391fadb12 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_index.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_index.go @@ -28,8 +28,8 @@ func init() { scpb.Status_VALIDATED, scpb.Status_ABSENT, func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsIndex), - to.TypeFilter(IsIndexDependent), + from.TypeFilter(rulesVersionKey, IsIndex), + to.TypeFilter(rulesVersionKey, isIndexDependent), JoinOnIndexID(from, to, "table-id", "index-id"), } }, @@ -41,8 +41,8 @@ func init() { scpb.Status_ABSENT, scpb.Status_ABSENT, func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsIndexDependent), - to.TypeFilter(IsIndex), + from.TypeFilter(rulesVersionKey, isIndexDependent), + to.TypeFilter(rulesVersionKey, IsIndex), JoinOnIndexID(from, to, "table-id", "index-id"), } }, @@ -72,7 +72,7 @@ func init() { func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.Type((*scpb.IndexColumn)(nil)), - to.TypeFilter(IsIndex), + to.TypeFilter(rulesVersionKey, IsIndex), JoinOnIndexID(from, to, "table-id", "index-id"), } }, @@ -98,7 +98,7 @@ func init() { func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.Type((*scpb.SecondaryIndexPartial)(nil)), - from.DescriptorIsNotBeingDropped(), + descriptorIsNotBeingDropped(from.El), to.Type((*scpb.SecondaryIndex)(nil)), JoinOnIndexID(from, to, "table-id", "index-id"), } diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_index_and_column.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_index_and_column.go index 554a309d5179..ee272e2b92be 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_index_and_column.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_index_and_column.go @@ -36,7 +36,7 @@ func init() { to.Type((*scpb.Column)(nil)), ColumnInIndex(ic, from, relationID, columnID, "index-id"), JoinOnColumnID(ic, to, relationID, columnID), - ic.DescriptorIsNotBeingDropped(), + descriptorIsNotBeingDropped(ic.El), } }) @@ -52,7 +52,7 @@ func init() { ColumnInIndex(ic, from, relationID, columnID, "index-id"), JoinOnColumnID(ic, to, relationID, columnID), StatusesToAbsent(from, scpb.Status_VALIDATED, to, scpb.Status_WRITE_ONLY), - ic.DescriptorIsNotBeingDropped(), + descriptorIsNotBeingDropped(ic.El), rel.Filter("rules.IsIndexKeyColumnKey", ic.El)( func(ic *scpb.IndexColumn) bool { return ic.Kind == scpb.IndexColumn_KEY diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_object.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_object.go index 44ebe1e2794d..56628a0431e5 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_object.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_drop_object.go @@ -39,7 +39,7 @@ func init() { "txn_dropped", "dropped", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsDescriptor), + from.TypeFilter(rulesVersionKey, isDescriptor), from.El.AttrEqVar(screl.DescID, "_"), from.El.AttrEqVar(rel.Self, to.El), StatusesToAbsent(from, scpb.Status_TXN_DROPPED, to, scpb.Status_DROPPED), @@ -51,7 +51,7 @@ func init() { "dropped", "absent", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsDescriptor), + from.TypeFilter(rulesVersionKey, isDescriptor), from.El.AttrEqVar(screl.DescID, "_"), from.El.AttrEqVar(rel.Self, to.El), StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), @@ -64,8 +64,8 @@ func init() { "descriptor", "dependent", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsDescriptor), - to.TypeFilter(IsSimpleDependent), + from.TypeFilter(rulesVersionKey, isDescriptor), + to.TypeFilter(rulesVersionKey, isSimpleDependent), JoinOnDescID(from, to, "desc-id"), StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), FromHasPublicStatusIfFromIsTableAndToIsRowLevelTTL(from.Target, from.El, to.El), @@ -78,8 +78,8 @@ func init() { "descriptor", "idx-or-col", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsDescriptor), - to.TypeFilter(IsSubjectTo2VersionInvariant), + from.TypeFilter(rulesVersionKey, isDescriptor), + to.TypeFilter(rulesVersionKey, isSubjectTo2VersionInvariant), JoinOnDescID(from, to, "desc-id"), StatusesToAbsent(from, scpb.Status_ABSENT, to, scpb.Status_ABSENT), } @@ -104,8 +104,8 @@ func init() { "referenced-descriptor", "referencing-via-attr", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsDescriptor), - to.TypeFilter(IsSimpleDependent), + from.TypeFilter(rulesVersionKey, isDescriptor), + to.TypeFilter(rulesVersionKey, isSimpleDependent), JoinReferencedDescID(to, from, "desc-id"), StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), } @@ -119,11 +119,11 @@ func init() { func(from, to NodeVars) rel.Clauses { fromDescID := rel.Var("fromDescID") return rel.Clauses{ - from.TypeFilter(IsTypeDescriptor), + from.TypeFilter(rulesVersionKey, isTypeDescriptor), from.JoinTargetNode(), from.DescIDEq(fromDescID), to.ReferencedTypeDescIDsContain(fromDescID), - to.TypeFilter(IsSimpleDependent, Or(IsWithTypeT, IsWithExpression)), + to.TypeFilter(rulesVersionKey, isSimpleDependent, Or(isWithTypeT, isWithExpression)), StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), } }, @@ -140,7 +140,7 @@ func init() { from.JoinTargetNode(), from.DescIDEq(seqID), to.ReferencedSequenceIDsContains(seqID), - to.TypeFilter(IsSimpleDependent, IsWithExpression), + to.TypeFilter(rulesVersionKey, isSimpleDependent, isWithExpression), StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), } }, diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_two_version.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_two_version.go index a6e4c9805adb..50f1ea78e7c8 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_two_version.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/dep_two_version.go @@ -69,7 +69,7 @@ func init() { from.Target.AttrEq(screl.TargetStatus, targetStatus.Status()), from.Node.AttrEq(screl.CurrentStatus, t.From()), to.Node.AttrEq(screl.CurrentStatus, t.To()), - from.DescriptorIsNotBeingDropped(), + descriptorIsNotBeingDropped(from.El), } if len(prePrevStatuses) > 0 { clauses = append(clauses, @@ -104,7 +104,7 @@ func init() { } } _ = ForEachElement(func(el scpb.Element) error { - if !IsSubjectTo2VersionInvariant(el) { + if !isSubjectTo2VersionInvariant(el) { return nil } if opgen.HasPublic(el) { diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/helpers.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/helpers.go new file mode 100644 index 000000000000..48c3bf3e4346 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/helpers.go @@ -0,0 +1,206 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package release_22_2 + +import ( + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl" + "github.com/cockroachdb/errors" +) + +const ( + // rulesVersion version of elements that can be appended to rel rule names. + rulesVersion = "-22.2" +) + +// rulesVersionKey version of elements used by this rule set. +var rulesVersionKey = clusterversion.V22_2 + +// descriptorIsNotBeingDropped creates a clause which leads to the outer clause +// failing to unify if the passed element is part of a descriptor and +// that descriptor is being dropped. +var descriptorIsNotBeingDropped = screl.Schema.DefNotJoin1( + "descriptorIsNotBeingDropped"+rulesVersion, "element", func( + element rel.Var, + ) rel.Clauses { + descriptor := rules.MkNodeVars("descriptor") + return rel.Clauses{ + descriptor.TypeFilter(rulesVersionKey, isDescriptor), + descriptor.JoinTarget(), + rules.JoinOnDescIDUntyped(descriptor.El, element, "id"), + descriptor.TargetStatus(scpb.ToAbsent), + } + }, +) + +// isDescriptor returns true for a descriptor-element, i.e. an element which +// owns its corresponding descriptor. +func isDescriptor(e scpb.Element) bool { + switch e.(type) { + case *scpb.Database, *scpb.Schema, *scpb.Table, *scpb.View, *scpb.Sequence, + *scpb.AliasType, *scpb.EnumType: + return true + } + return false +} + +func isSubjectTo2VersionInvariant(e scpb.Element) bool { + // TODO(ajwerner): This should include constraints and enum values but it + // currently does not because we do not support dropping them unless we're + // dropping the descriptor and we do not support adding them. + return IsIndex(e) || isColumn(e) +} + +func IsIndex(e scpb.Element) bool { + switch e.(type) { + case *scpb.PrimaryIndex, *scpb.SecondaryIndex, *scpb.TemporaryIndex: + return true + } + return false +} + +func isColumn(e scpb.Element) bool { + _, ok := e.(*scpb.Column) + return ok +} + +func isSimpleDependent(e scpb.Element) bool { + return !isDescriptor(e) && !isSubjectTo2VersionInvariant(e) +} + +func getTypeT(element scpb.Element) (*scpb.TypeT, error) { + switch e := element.(type) { + case *scpb.ColumnType: + if e == nil { + return nil, nil + } + return &e.TypeT, nil + case *scpb.AliasType: + if e == nil { + return nil, nil + } + return &e.TypeT, nil + } + return nil, errors.AssertionFailedf("element %T does not have an embedded scpb.TypeT", element) +} + +func isWithTypeT(element scpb.Element) bool { + _, err := getTypeT(element) + return err == nil +} + +func getExpression(element scpb.Element) (*scpb.Expression, error) { + switch e := element.(type) { + case *scpb.ColumnType: + if e == nil { + return nil, nil + } + return e.ComputeExpr, nil + case *scpb.ColumnDefaultExpression: + if e == nil { + return nil, nil + } + return &e.Expression, nil + case *scpb.ColumnOnUpdateExpression: + if e == nil { + return nil, nil + } + return &e.Expression, nil + case *scpb.SecondaryIndexPartial: + if e == nil { + return nil, nil + } + return &e.Expression, nil + case *scpb.CheckConstraint: + if e == nil { + return nil, nil + } + return &e.Expression, nil + } + return nil, errors.AssertionFailedf("element %T does not have an embedded scpb.Expression", element) +} + +func isWithExpression(element scpb.Element) bool { + _, err := getExpression(element) + return err == nil +} + +func isTypeDescriptor(element scpb.Element) bool { + switch element.(type) { + case *scpb.EnumType, *scpb.AliasType: + return true + default: + return false + } +} + +func isColumnDependent(e scpb.Element) bool { + switch e.(type) { + case *scpb.ColumnType: + return true + case *scpb.ColumnName, *scpb.ColumnComment, *scpb.IndexColumn: + return true + } + return isColumnTypeDependent(e) +} + +func isColumnTypeDependent(e scpb.Element) bool { + switch e.(type) { + case *scpb.SequenceOwner, *scpb.ColumnDefaultExpression, *scpb.ColumnOnUpdateExpression: + return true + } + return false +} + +func isIndexDependent(e scpb.Element) bool { + switch e.(type) { + case *scpb.IndexName, *scpb.IndexComment, *scpb.IndexColumn: + return true + case *scpb.IndexPartitioning, *scpb.SecondaryIndexPartial: + return true + } + return false +} + +// isSupportedNonIndexBackedConstraint a non-index-backed constraint is one of {Check, FK, UniqueWithoutIndex}. We only +// support Check for now. +// TODO (xiang): Expand this predicate to include other non-index-backed constraints +// when we properly support adding/dropping them in the new schema changer. +func isSupportedNonIndexBackedConstraint(e scpb.Element) bool { + switch e.(type) { + case *scpb.CheckConstraint, *scpb.ForeignKeyConstraint, *scpb.UniqueWithoutIndexConstraint: + return true + } + return false +} + +func isConstraint(e scpb.Element) bool { + switch e.(type) { + case *scpb.PrimaryIndex, *scpb.SecondaryIndex, *scpb.TemporaryIndex: + return true + case *scpb.CheckConstraint, *scpb.UniqueWithoutIndexConstraint, *scpb.ForeignKeyConstraint: + return true + } + return false +} + +func isConstraintDependent(e scpb.Element) bool { + switch e.(type) { + case *scpb.ConstraintWithoutIndexName: + return true + case *scpb.ConstraintComment: + return true + } + return false +} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/op_drop.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/op_drop.go index 3f2a0b70e682..7010ebcd3bb3 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/op_drop.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/op_drop.go @@ -130,7 +130,7 @@ func init() { (*scpb.Table)(nil), (*scpb.View)(nil), ), - index.TypeFilter(IsIndex), + index.TypeFilter(rulesVersionKey, IsIndex), dep.Type( (*scpb.IndexName)(nil), (*scpb.IndexPartitioning)(nil), @@ -230,7 +230,7 @@ func init() { "skip element removal ops on descriptor drop", dep.Node, screl.MustQuery( - desc.TypeFilter(IsDescriptor), + desc.TypeFilter(rulesVersionKey, isDescriptor), dep.Type( (*scpb.ColumnFamily)(nil), (*scpb.Owner)(nil), diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/op_index_and_column.go b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/op_index_and_column.go index 5a9253c4e49a..cd6d4d8d50ff 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/op_index_and_column.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/op_index_and_column.go @@ -29,7 +29,7 @@ func init() { ic.Node, screl.MustQuery( ic.Type((*scpb.IndexColumn)(nil)), - index.TypeFilter(IsIndex), + index.TypeFilter(rulesVersionKey, IsIndex), JoinOnIndexID(ic, index, relationID, indexID), ic.JoinTargetNode(), ic.TargetStatus(scpb.ToAbsent, scpb.Transient), diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/deprules b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/deprules index aab3c15bb41e..7be3fc7afd5b 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/deprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/deprules @@ -9,15 +9,15 @@ ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id): ColumnInSwappedInPrimaryIndex($index-column, $index, $table-id, $column-id, $index-id): - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) - sourceIndexIsSet($index) -DescriptorIsNotBeingDropped($element): +ToPublicOrTransient($target1, $target2): + - $target1[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] + - $target2[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] +descriptorIsNotBeingDropped-22.2($element): not-join: - - $descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] + - $descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType'] - joinTarget($descriptor, $descriptor-Target) - joinOnDescID($descriptor, $element, $id) - $descriptor-Target[TargetStatus] = ABSENT -ToPublicOrTransient($target1, $target2): - - $target1[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] - - $target2[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] fromHasPublicStatusIfFromIsTableAndToIsRowLevelTTL($fromTarget, $fromEl, $toEl): not-join: - $fromEl[Type] = '*scpb.Table' @@ -95,11 +95,6 @@ nodeNotExistsWithStatusIn_VALIDATED_TRANSIENT_WRITE_ONLY_MERGE_ONLY_TRANSIENT_ME - $n[Type] = '*screl.Node' - $n[Target] = $sharedTarget - $n[CurrentStatus] IN [VALIDATED, TRANSIENT_WRITE_ONLY, MERGE_ONLY, TRANSIENT_MERGE_ONLY, MERGED, TRANSIENT_MERGED] -nodeNotExistsWithStatusIn_WRITE_ONLY($sharedTarget): - not-join: - - $n[Type] = '*screl.Node' - - $n[Target] = $sharedTarget - - $n[CurrentStatus] IN [WRITE_ONLY] sourceIndexIsSet($index): - $index[SourceIndexID] != 0 toAbsent($target1, $target2): @@ -111,103 +106,6 @@ transient($target1, $target2): deprules ---- -- name: 'CheckConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' - from: prev-Node - kind: PreviousTransactionPrecedence - to: next-Node - query: - - $prev[Type] = '*scpb.CheckConstraint' - - $next[Type] = '*scpb.CheckConstraint' - - $prev[DescID] = $_ - - $prev[Self] = $next - - $prev-Target[Self] = $next-Target - - $prev-Target[TargetStatus] = ABSENT - - $prev-Node[CurrentStatus] = PUBLIC - - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) - - joinTargetNode($prev, $prev-Target, $prev-Node) - - joinTargetNode($next, $next-Target, $next-Node) -- name: 'CheckConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT' - from: prev-Node - kind: PreviousTransactionPrecedence - to: next-Node - query: - - $prev[Type] = '*scpb.CheckConstraint' - - $next[Type] = '*scpb.CheckConstraint' - - $prev[DescID] = $_ - - $prev[Self] = $next - - $prev-Target[Self] = $next-Target - - $prev-Target[TargetStatus] = ABSENT - - $prev-Node[CurrentStatus] = VALIDATED - - $next-Node[CurrentStatus] = ABSENT - - DescriptorIsNotBeingDropped($prev) - - nodeNotExistsWithStatusIn_WRITE_ONLY($prev-Target) - - joinTargetNode($prev, $prev-Target, $prev-Node) - - joinTargetNode($next, $next-Target, $next-Node) -- name: 'CheckConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED' - from: prev-Node - kind: PreviousTransactionPrecedence - to: next-Node - query: - - $prev[Type] = '*scpb.CheckConstraint' - - $next[Type] = '*scpb.CheckConstraint' - - $prev[DescID] = $_ - - $prev[Self] = $next - - $prev-Target[Self] = $next-Target - - $prev-Target[TargetStatus] = ABSENT - - $prev-Node[CurrentStatus] = WRITE_ONLY - - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) - - joinTargetNode($prev, $prev-Target, $prev-Node) - - joinTargetNode($next, $next-Target, $next-Node) -- name: 'CheckConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY' - from: prev-Node - kind: PreviousTransactionPrecedence - to: next-Node - query: - - $prev[Type] = '*scpb.CheckConstraint' - - $next[Type] = '*scpb.CheckConstraint' - - $prev[DescID] = $_ - - $prev[Self] = $next - - $prev-Target[Self] = $next-Target - - $prev-Target[TargetStatus] = PUBLIC - - $prev-Node[CurrentStatus] = ABSENT - - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) - - joinTargetNode($prev, $prev-Target, $prev-Node) - - joinTargetNode($next, $next-Target, $next-Node) -- name: 'CheckConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' - from: prev-Node - kind: PreviousTransactionPrecedence - to: next-Node - query: - - $prev[Type] = '*scpb.CheckConstraint' - - $next[Type] = '*scpb.CheckConstraint' - - $prev[DescID] = $_ - - $prev[Self] = $next - - $prev-Target[Self] = $next-Target - - $prev-Target[TargetStatus] = PUBLIC - - $prev-Node[CurrentStatus] = VALIDATED - - $next-Node[CurrentStatus] = PUBLIC - - DescriptorIsNotBeingDropped($prev) - - joinTargetNode($prev, $prev-Target, $prev-Node) - - joinTargetNode($next, $next-Target, $next-Node) -- name: 'CheckConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' - from: prev-Node - kind: PreviousTransactionPrecedence - to: next-Node - query: - - $prev[Type] = '*scpb.CheckConstraint' - - $next[Type] = '*scpb.CheckConstraint' - - $prev[DescID] = $_ - - $prev[Self] = $next - - $prev-Target[Self] = $next-Target - - $prev-Target[TargetStatus] = PUBLIC - - $prev-Node[CurrentStatus] = WRITE_ONLY - - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) - - joinTargetNode($prev, $prev-Target, $prev-Node) - - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' from: prev-Node kind: PreviousTransactionPrecedence @@ -221,7 +119,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = ABSENT - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY' @@ -237,7 +135,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY' @@ -253,7 +151,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to PUBLIC uphold 2-version invariant: ABSENT->DELETE_ONLY' @@ -269,7 +167,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY' @@ -285,7 +183,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC' @@ -301,7 +199,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = PUBLIC - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: DEFAULT or ON UPDATE existence precedes writes to column @@ -317,103 +215,6 @@ deprules - $column-Node[CurrentStatus] = WRITE_ONLY - joinTargetNode($expr, $expr-Target, $expr-Node) - joinTargetNode($column, $column-Target, $column-Node) -- name: 'ForeignKeyConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' - from: prev-Node - kind: PreviousTransactionPrecedence - to: next-Node - query: - - $prev[Type] = '*scpb.ForeignKeyConstraint' - - $next[Type] = '*scpb.ForeignKeyConstraint' - - $prev[DescID] = $_ - - $prev[Self] = $next - - $prev-Target[Self] = $next-Target - - $prev-Target[TargetStatus] = ABSENT - - $prev-Node[CurrentStatus] = PUBLIC - - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) - - joinTargetNode($prev, $prev-Target, $prev-Node) - - joinTargetNode($next, $next-Target, $next-Node) -- name: 'ForeignKeyConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT' - from: prev-Node - kind: PreviousTransactionPrecedence - to: next-Node - query: - - $prev[Type] = '*scpb.ForeignKeyConstraint' - - $next[Type] = '*scpb.ForeignKeyConstraint' - - $prev[DescID] = $_ - - $prev[Self] = $next - - $prev-Target[Self] = $next-Target - - $prev-Target[TargetStatus] = ABSENT - - $prev-Node[CurrentStatus] = VALIDATED - - $next-Node[CurrentStatus] = ABSENT - - DescriptorIsNotBeingDropped($prev) - - nodeNotExistsWithStatusIn_WRITE_ONLY($prev-Target) - - joinTargetNode($prev, $prev-Target, $prev-Node) - - joinTargetNode($next, $next-Target, $next-Node) -- name: 'ForeignKeyConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED' - from: prev-Node - kind: PreviousTransactionPrecedence - to: next-Node - query: - - $prev[Type] = '*scpb.ForeignKeyConstraint' - - $next[Type] = '*scpb.ForeignKeyConstraint' - - $prev[DescID] = $_ - - $prev[Self] = $next - - $prev-Target[Self] = $next-Target - - $prev-Target[TargetStatus] = ABSENT - - $prev-Node[CurrentStatus] = WRITE_ONLY - - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) - - joinTargetNode($prev, $prev-Target, $prev-Node) - - joinTargetNode($next, $next-Target, $next-Node) -- name: 'ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY' - from: prev-Node - kind: PreviousTransactionPrecedence - to: next-Node - query: - - $prev[Type] = '*scpb.ForeignKeyConstraint' - - $next[Type] = '*scpb.ForeignKeyConstraint' - - $prev[DescID] = $_ - - $prev[Self] = $next - - $prev-Target[Self] = $next-Target - - $prev-Target[TargetStatus] = PUBLIC - - $prev-Node[CurrentStatus] = ABSENT - - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) - - joinTargetNode($prev, $prev-Target, $prev-Node) - - joinTargetNode($next, $next-Target, $next-Node) -- name: 'ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' - from: prev-Node - kind: PreviousTransactionPrecedence - to: next-Node - query: - - $prev[Type] = '*scpb.ForeignKeyConstraint' - - $next[Type] = '*scpb.ForeignKeyConstraint' - - $prev[DescID] = $_ - - $prev[Self] = $next - - $prev-Target[Self] = $next-Target - - $prev-Target[TargetStatus] = PUBLIC - - $prev-Node[CurrentStatus] = VALIDATED - - $next-Node[CurrentStatus] = PUBLIC - - DescriptorIsNotBeingDropped($prev) - - joinTargetNode($prev, $prev-Target, $prev-Node) - - joinTargetNode($next, $next-Target, $next-Node) -- name: 'ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' - from: prev-Node - kind: PreviousTransactionPrecedence - to: next-Node - query: - - $prev[Type] = '*scpb.ForeignKeyConstraint' - - $next[Type] = '*scpb.ForeignKeyConstraint' - - $prev[DescID] = $_ - - $prev[Self] = $next - - $prev-Target[Self] = $next-Target - - $prev-Target[TargetStatus] = PUBLIC - - $prev-Node[CurrentStatus] = WRITE_ONLY - - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) - - joinTargetNode($prev, $prev-Target, $prev-Node) - - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY' from: prev-Node kind: PreviousTransactionPrecedence @@ -427,7 +228,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY' @@ -443,7 +244,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = BACKFILL_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' @@ -459,7 +260,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = ABSENT - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - nodeNotExistsWithStatusIn_TRANSIENT_DELETE_ONLY_BACKFILLED_TRANSIENT_BACKFILLED_BACKFILL_ONLY_TRANSIENT_BACKFILL_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -476,7 +277,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY' @@ -492,7 +293,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = MERGE_ONLY - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' @@ -508,7 +309,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_ABSENT->ABSENT' @@ -524,7 +325,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_ABSENT - $next-Node[CurrentStatus] = ABSENT - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_BACKFILLED->DELETE_ONLY' @@ -540,7 +341,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_BACKFILL_ONLY->DELETE_ONLY' @@ -556,7 +357,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_BACKFILL_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->DELETE_ONLY' @@ -572,7 +373,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_MERGED->WRITE_ONLY' @@ -588,7 +389,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_MERGE_ONLY->WRITE_ONLY' @@ -604,7 +405,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_MERGE_ONLY - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_VALIDATED->VALIDATED' @@ -620,7 +421,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_VALIDATED - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_WRITE_ONLY->WRITE_ONLY' @@ -636,7 +437,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_WRITE_ONLY - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY' @@ -652,7 +453,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - nodeNotExistsWithStatusIn_TRANSIENT_VALIDATED($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -669,7 +470,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - nodeNotExistsWithStatusIn_VALIDATED_TRANSIENT_WRITE_ONLY_MERGE_ONLY_TRANSIENT_MERGE_ONLY_MERGED_TRANSIENT_MERGED($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -686,7 +487,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = BACKFILL_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY' @@ -702,7 +503,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED' @@ -718,7 +519,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = BACKFILL_ONLY - $next-Node[CurrentStatus] = BACKFILLED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY' @@ -734,7 +535,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = MERGE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY' @@ -750,7 +551,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED' @@ -766,7 +567,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = MERGE_ONLY - $next-Node[CurrentStatus] = MERGED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' @@ -782,7 +583,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' @@ -798,7 +599,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->BACKFILL_ONLY' @@ -814,7 +615,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = BACKFILL_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY' @@ -830,7 +631,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED' @@ -846,7 +647,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = BACKFILL_ONLY - $next-Node[CurrentStatus] = BACKFILLED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY' @@ -862,7 +663,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = MERGE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: MERGED->WRITE_ONLY' @@ -878,7 +679,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: MERGE_ONLY->MERGED' @@ -894,7 +695,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = MERGE_ONLY - $next-Node[CurrentStatus] = MERGED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: PUBLIC->TRANSIENT_VALIDATED' @@ -910,7 +711,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = TRANSIENT_VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_BACKFILLED->TRANSIENT_DELETE_ONLY' @@ -926,7 +727,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_BACKFILLED - $next-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_BACKFILL_ONLY->TRANSIENT_DELETE_ONLY' @@ -942,7 +743,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_BACKFILL_ONLY - $next-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT' @@ -958,7 +759,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY - $next-Node[CurrentStatus] = TRANSIENT_ABSENT - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - nodeNotExistsWithStatusIn_TRANSIENT_BACKFILLED_TRANSIENT_BACKFILL_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -975,7 +776,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_MERGED - $next-Node[CurrentStatus] = TRANSIENT_WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_MERGE_ONLY->TRANSIENT_WRITE_ONLY' @@ -991,7 +792,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_MERGE_ONLY - $next-Node[CurrentStatus] = TRANSIENT_WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_VALIDATED->TRANSIENT_WRITE_ONLY' @@ -1007,7 +808,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_VALIDATED - $next-Node[CurrentStatus] = TRANSIENT_WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_WRITE_ONLY->TRANSIENT_DELETE_ONLY' @@ -1023,7 +824,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_WRITE_ONLY - $next-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - nodeNotExistsWithStatusIn_TRANSIENT_VALIDATED_TRANSIENT_MERGE_ONLY_TRANSIENT_MERGED($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -1040,7 +841,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED' @@ -1056,7 +857,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY' @@ -1072,7 +873,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY' @@ -1088,7 +889,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = BACKFILL_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' @@ -1104,7 +905,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = ABSENT - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - nodeNotExistsWithStatusIn_BACKFILLED_BACKFILL_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -1121,7 +922,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY' @@ -1137,7 +938,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = MERGE_ONLY - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' @@ -1153,7 +954,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = PUBLIC - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY' @@ -1169,7 +970,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY' @@ -1185,7 +986,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - nodeNotExistsWithStatusIn_VALIDATED_MERGE_ONLY_MERGED($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -1202,7 +1003,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = BACKFILL_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY' @@ -1218,7 +1019,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = BACKFILLED - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED' @@ -1234,7 +1035,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = BACKFILL_ONLY - $next-Node[CurrentStatus] = BACKFILLED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY' @@ -1250,7 +1051,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = MERGE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY' @@ -1266,7 +1067,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = MERGED - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED' @@ -1282,7 +1083,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = MERGE_ONLY - $next-Node[CurrentStatus] = MERGED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' @@ -1298,7 +1099,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = VALIDATED - $next-Node[CurrentStatus] = PUBLIC - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' @@ -1314,7 +1115,7 @@ deprules - $prev-Target[TargetStatus] = PUBLIC - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' @@ -1330,7 +1131,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = ABSENT - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - nodeNotExistsWithStatusIn_TRANSIENT_DELETE_ONLY($prev-Target) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) @@ -1347,7 +1148,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_ABSENT - $next-Node[CurrentStatus] = ABSENT - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->DELETE_ONLY' @@ -1363,7 +1164,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY' @@ -1379,7 +1180,7 @@ deprules - $prev-Target[TargetStatus] = ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY' @@ -1395,7 +1196,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = ABSENT - $next-Node[CurrentStatus] = DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY' @@ -1411,7 +1212,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = DELETE_ONLY - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT' @@ -1427,7 +1228,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY - $next-Node[CurrentStatus] = TRANSIENT_ABSENT - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY' @@ -1443,104 +1244,7 @@ deprules - $prev-Target[TargetStatus] = TRANSIENT_ABSENT - $prev-Node[CurrentStatus] = WRITE_ONLY - $next-Node[CurrentStatus] = TRANSIENT_DELETE_ONLY - - DescriptorIsNotBeingDropped($prev) - - joinTargetNode($prev, $prev-Target, $prev-Node) - - joinTargetNode($next, $next-Target, $next-Node) -- name: 'UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' - from: prev-Node - kind: PreviousTransactionPrecedence - to: next-Node - query: - - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $prev[DescID] = $_ - - $prev[Self] = $next - - $prev-Target[Self] = $next-Target - - $prev-Target[TargetStatus] = ABSENT - - $prev-Node[CurrentStatus] = PUBLIC - - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) - - joinTargetNode($prev, $prev-Target, $prev-Node) - - joinTargetNode($next, $next-Target, $next-Node) -- name: 'UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT' - from: prev-Node - kind: PreviousTransactionPrecedence - to: next-Node - query: - - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $prev[DescID] = $_ - - $prev[Self] = $next - - $prev-Target[Self] = $next-Target - - $prev-Target[TargetStatus] = ABSENT - - $prev-Node[CurrentStatus] = VALIDATED - - $next-Node[CurrentStatus] = ABSENT - - DescriptorIsNotBeingDropped($prev) - - nodeNotExistsWithStatusIn_WRITE_ONLY($prev-Target) - - joinTargetNode($prev, $prev-Target, $prev-Node) - - joinTargetNode($next, $next-Target, $next-Node) -- name: 'UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED' - from: prev-Node - kind: PreviousTransactionPrecedence - to: next-Node - query: - - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $prev[DescID] = $_ - - $prev[Self] = $next - - $prev-Target[Self] = $next-Target - - $prev-Target[TargetStatus] = ABSENT - - $prev-Node[CurrentStatus] = WRITE_ONLY - - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) - - joinTargetNode($prev, $prev-Target, $prev-Node) - - joinTargetNode($next, $next-Target, $next-Node) -- name: 'UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY' - from: prev-Node - kind: PreviousTransactionPrecedence - to: next-Node - query: - - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $prev[DescID] = $_ - - $prev[Self] = $next - - $prev-Target[Self] = $next-Target - - $prev-Target[TargetStatus] = PUBLIC - - $prev-Node[CurrentStatus] = ABSENT - - $next-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($prev) - - joinTargetNode($prev, $prev-Target, $prev-Node) - - joinTargetNode($next, $next-Target, $next-Node) -- name: 'UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' - from: prev-Node - kind: PreviousTransactionPrecedence - to: next-Node - query: - - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $prev[DescID] = $_ - - $prev[Self] = $next - - $prev-Target[Self] = $next-Target - - $prev-Target[TargetStatus] = PUBLIC - - $prev-Node[CurrentStatus] = VALIDATED - - $next-Node[CurrentStatus] = PUBLIC - - DescriptorIsNotBeingDropped($prev) - - joinTargetNode($prev, $prev-Target, $prev-Node) - - joinTargetNode($next, $next-Target, $next-Node) -- name: 'UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' - from: prev-Node - kind: PreviousTransactionPrecedence - to: next-Node - query: - - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' - - $prev[DescID] = $_ - - $prev[Self] = $next - - $prev-Target[Self] = $next-Target - - $prev-Target[TargetStatus] = PUBLIC - - $prev-Node[CurrentStatus] = WRITE_ONLY - - $next-Node[CurrentStatus] = VALIDATED - - DescriptorIsNotBeingDropped($prev) + - descriptorIsNotBeingDropped-22.2($prev) - joinTargetNode($prev, $prev-Target, $prev-Node) - joinTargetNode($next, $next-Target, $next-Node) - name: column dependents exist before column becomes public @@ -1697,7 +1401,7 @@ deprules to: column-Node query: - $column-type[Type] = '*scpb.ColumnType' - - DescriptorIsNotBeingDropped($column-type) + - descriptorIsNotBeingDropped-22.2($column-type) - $column[Type] = '*scpb.Column' - joinOnColumnID($column-type, $column, $table-id, $col-id) - toAbsent($column-type-Target, $column-Target) @@ -1939,7 +1643,7 @@ deprules kind: PreviousTransactionPrecedence to: absent-Node query: - - $dropped[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] + - $dropped[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType'] - $dropped[DescID] = $_ - $dropped[Self] = $absent - toAbsent($dropped-Target, $absent-Target) @@ -1952,7 +1656,7 @@ deprules kind: PreviousStagePrecedence to: dropped-Node query: - - $txn_dropped[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] + - $txn_dropped[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType'] - $txn_dropped[DescID] = $_ - $txn_dropped[Self] = $dropped - toAbsent($txn_dropped-Target, $dropped-Target) @@ -1965,8 +1669,8 @@ deprules kind: SameStagePrecedence to: dependent-Node query: - - $descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] - - $dependent[Type] IN ['*scpb.ColumnFamily', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TablePartitioning', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaParent', '*scpb.SchemaComment', '*scpb.ObjectParent', '*scpb.EnumTypeValue', '*scpb.CompositeTypeAttrType', '*scpb.CompositeTypeAttrName'] + - $descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType'] + - $dependent[Type] IN ['*scpb.ColumnFamily', '*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaParent', '*scpb.SchemaComment', '*scpb.ObjectParent', '*scpb.EnumTypeValue'] - joinOnDescID($descriptor, $dependent, $desc-id) - toAbsent($descriptor-Target, $dependent-Target) - $descriptor-Node[CurrentStatus] = DROPPED @@ -1979,8 +1683,8 @@ deprules kind: SameStagePrecedence to: referencing-via-attr-Node query: - - $referenced-descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] - - $referencing-via-attr[Type] IN ['*scpb.ColumnFamily', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TablePartitioning', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaParent', '*scpb.SchemaComment', '*scpb.ObjectParent', '*scpb.EnumTypeValue', '*scpb.CompositeTypeAttrType', '*scpb.CompositeTypeAttrName'] + - $referenced-descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType'] + - $referencing-via-attr[Type] IN ['*scpb.ColumnFamily', '*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaParent', '*scpb.SchemaComment', '*scpb.ObjectParent', '*scpb.EnumTypeValue'] - joinReferencedDescID($referencing-via-attr, $referenced-descriptor, $desc-id) - toAbsent($referenced-descriptor-Target, $referencing-via-attr-Target) - $referenced-descriptor-Node[CurrentStatus] = DROPPED @@ -1996,7 +1700,7 @@ deprules - joinTargetNode($referenced-descriptor, $referenced-descriptor-Target, $referenced-descriptor-Node) - $referenced-descriptor[DescID] = $seqID - $referencing-via-expr[ReferencedSequenceIDs] CONTAINS $seqID - - $referencing-via-expr[Type] IN ['*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SecondaryIndexPartial'] + - $referencing-via-expr[Type] IN ['*scpb.CheckConstraint', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SecondaryIndexPartial'] - toAbsent($referenced-descriptor-Target, $referencing-via-expr-Target) - $referenced-descriptor-Node[CurrentStatus] = DROPPED - $referencing-via-expr-Node[CurrentStatus] = ABSENT @@ -2007,11 +1711,11 @@ deprules kind: SameStagePrecedence to: referencing-via-type-Node query: - - $referenced-descriptor[Type] IN ['*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] + - $referenced-descriptor[Type] IN ['*scpb.EnumType', '*scpb.AliasType'] - joinTargetNode($referenced-descriptor, $referenced-descriptor-Target, $referenced-descriptor-Node) - $referenced-descriptor[DescID] = $fromDescID - $referencing-via-type[ReferencedTypeIDs] CONTAINS $fromDescID - - $referencing-via-type[Type] IN ['*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SecondaryIndexPartial'] + - $referencing-via-type[Type] IN ['*scpb.CheckConstraint', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SecondaryIndexPartial'] - toAbsent($referenced-descriptor-Target, $referencing-via-type-Target) - $referenced-descriptor-Node[CurrentStatus] = DROPPED - $referencing-via-type-Node[CurrentStatus] = ABSENT @@ -2022,8 +1726,8 @@ deprules kind: SameStagePrecedence to: idx-or-col-Node query: - - $descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] - - $idx-or-col[Type] IN ['*scpb.Column', '*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex', '*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] + - $descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType'] + - $idx-or-col[Type] IN ['*scpb.Column', '*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] - joinOnDescID($descriptor, $idx-or-col, $desc-id) - toAbsent($descriptor-Target, $idx-or-col-Target) - $descriptor-Node[CurrentStatus] = ABSENT @@ -2174,7 +1878,7 @@ deprules - $column[Type] = '*scpb.Column' - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) - joinOnColumnID($index-column, $column, $table-id, $column-id) - - DescriptorIsNotBeingDropped($index-column) + - descriptorIsNotBeingDropped-22.2($index-column) - toAbsent($index-Target, $column-Target) - $index-Node[CurrentStatus] = ABSENT - $column-Node[CurrentStatus] = ABSENT @@ -2189,7 +1893,7 @@ deprules - $column[Type] = '*scpb.Column' - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) - joinOnColumnID($index-column, $column, $table-id, $column-id) - - DescriptorIsNotBeingDropped($index-column) + - descriptorIsNotBeingDropped-22.2($index-column) - transient($index-Target, $column-Target) - $index-Node[CurrentStatus] = TRANSIENT_ABSENT - $column-Node[CurrentStatus] = TRANSIENT_ABSENT @@ -2204,7 +1908,7 @@ deprules - $column[Type] = '*scpb.Column' - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) - joinOnColumnID($index-column, $column, $table-id, $column-id) - - DescriptorIsNotBeingDropped($index-column) + - descriptorIsNotBeingDropped-22.2($index-column) - $index-Target[TargetStatus] = TRANSIENT_ABSENT - $index-Node[CurrentStatus] = TRANSIENT_ABSENT - $column-Target[TargetStatus] = ABSENT @@ -2220,7 +1924,7 @@ deprules - $column[Type] = '*scpb.Column' - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) - joinOnColumnID($index-column, $column, $table-id, $column-id) - - DescriptorIsNotBeingDropped($index-column) + - descriptorIsNotBeingDropped-22.2($index-column) - $index-Target[TargetStatus] = ABSENT - $index-Node[CurrentStatus] = ABSENT - $column-Target[TargetStatus] = TRANSIENT_ABSENT @@ -2253,7 +1957,7 @@ deprules to: index-Node query: - $partial-predicate[Type] = '*scpb.SecondaryIndexPartial' - - DescriptorIsNotBeingDropped($partial-predicate) + - descriptorIsNotBeingDropped-22.2($partial-predicate) - $index[Type] = '*scpb.SecondaryIndex' - joinOnIndexID($partial-predicate, $index, $table-id, $index-id) - toAbsent($partial-predicate-Target, $index-Target) @@ -2267,7 +1971,7 @@ deprules to: index-Node query: - $partial-predicate[Type] = '*scpb.SecondaryIndexPartial' - - DescriptorIsNotBeingDropped($partial-predicate) + - descriptorIsNotBeingDropped-22.2($partial-predicate) - $index[Type] = '*scpb.SecondaryIndex' - joinOnIndexID($partial-predicate, $index, $table-id, $index-id) - transient($partial-predicate-Target, $index-Target) @@ -2281,7 +1985,7 @@ deprules to: index-Node query: - $partial-predicate[Type] = '*scpb.SecondaryIndexPartial' - - DescriptorIsNotBeingDropped($partial-predicate) + - descriptorIsNotBeingDropped-22.2($partial-predicate) - $index[Type] = '*scpb.SecondaryIndex' - joinOnIndexID($partial-predicate, $index, $table-id, $index-id) - $partial-predicate-Target[TargetStatus] = TRANSIENT_ABSENT @@ -2296,7 +2000,7 @@ deprules to: index-Node query: - $partial-predicate[Type] = '*scpb.SecondaryIndexPartial' - - DescriptorIsNotBeingDropped($partial-predicate) + - descriptorIsNotBeingDropped-22.2($partial-predicate) - $index[Type] = '*scpb.SecondaryIndex' - joinOnIndexID($partial-predicate, $index, $table-id, $index-id) - $partial-predicate-Target[TargetStatus] = ABSENT @@ -2449,7 +2153,7 @@ deprules - toAbsent($index-Target, $column-Target) - $index-Node[CurrentStatus] = VALIDATED - $column-Node[CurrentStatus] = WRITE_ONLY - - DescriptorIsNotBeingDropped($index-column) + - descriptorIsNotBeingDropped-22.2($index-column) - rules.IsIndexKeyColumnKey(*scpb.IndexColumn)($index-column) - joinTargetNode($index, $index-Target, $index-Node) - joinTargetNode($column, $column-Target, $column-Node) diff --git a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/oprules b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/oprules index 37682f812530..1ef348ddbb18 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/oprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/release_22_2/testdata/oprules @@ -9,15 +9,15 @@ ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id): ColumnInSwappedInPrimaryIndex($index-column, $index, $table-id, $column-id, $index-id): - ColumnInIndex($index-column, $index, $table-id, $column-id, $index-id) - sourceIndexIsSet($index) -DescriptorIsNotBeingDropped($element): +ToPublicOrTransient($target1, $target2): + - $target1[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] + - $target2[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] +descriptorIsNotBeingDropped-22.2($element): not-join: - - $descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] + - $descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType'] - joinTarget($descriptor, $descriptor-Target) - joinOnDescID($descriptor, $element, $id) - $descriptor-Target[TargetStatus] = ABSENT -ToPublicOrTransient($target1, $target2): - - $target1[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] - - $target2[TargetStatus] IN [PUBLIC, TRANSIENT_ABSENT] fromHasPublicStatusIfFromIsTableAndToIsRowLevelTTL($fromTarget, $fromEl, $toEl): not-join: - $fromEl[Type] = '*scpb.Table' @@ -95,11 +95,6 @@ nodeNotExistsWithStatusIn_VALIDATED_TRANSIENT_WRITE_ONLY_MERGE_ONLY_TRANSIENT_ME - $n[Type] = '*screl.Node' - $n[Target] = $sharedTarget - $n[CurrentStatus] IN [VALIDATED, TRANSIENT_WRITE_ONLY, MERGE_ONLY, TRANSIENT_MERGE_ONLY, MERGED, TRANSIENT_MERGED] -nodeNotExistsWithStatusIn_WRITE_ONLY($sharedTarget): - not-join: - - $n[Type] = '*screl.Node' - - $n[Target] = $sharedTarget - - $n[CurrentStatus] IN [WRITE_ONLY] sourceIndexIsSet($index): - $index[SourceIndexID] != 0 toAbsent($target1, $target2): @@ -163,7 +158,7 @@ oprules - name: skip element removal ops on descriptor drop from: dep-Node query: - - $desc[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] + - $desc[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType'] - $dep[Type] IN ['*scpb.ColumnFamily', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.EnumTypeValue'] - joinOnDescID($desc, $dep, $desc-id) - joinTarget($desc, $desc-Target) diff --git a/pkg/sql/schemachanger/scplan/internal/scstage/BUILD.bazel b/pkg/sql/schemachanger/scplan/internal/scstage/BUILD.bazel index ddef73d0121a..eaa8cecec91d 100644 --- a/pkg/sql/schemachanger/scplan/internal/scstage/BUILD.bazel +++ b/pkg/sql/schemachanger/scplan/internal/scstage/BUILD.bazel @@ -15,7 +15,7 @@ go_library( "//pkg/sql/catalog/descpb", "//pkg/sql/schemachanger/scop", "//pkg/sql/schemachanger/scpb", - "//pkg/sql/schemachanger/scplan/internal/rules", + "//pkg/sql/schemachanger/scplan/internal/rules/current", "//pkg/sql/schemachanger/scplan/internal/scgraph", "//pkg/sql/schemachanger/screl", "//pkg/util/iterutil", diff --git a/pkg/sql/schemachanger/scplan/internal/scstage/build.go b/pkg/sql/schemachanger/scplan/internal/scstage/build.go index 3bfdb54568e1..b531ff8bd966 100644 --- a/pkg/sql/schemachanger/scplan/internal/scstage/build.go +++ b/pkg/sql/schemachanger/scplan/internal/scstage/build.go @@ -21,7 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" - "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules/current" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/screl" "github.com/cockroachdb/cockroach/pkg/util/iterutil" @@ -876,7 +876,7 @@ func (bc buildContext) makeDescriptorStates(cur, next *Stage) map[descpb.ID]*scp // if we haven't reached the non-revertible post-commit phase yet. if cur.Phase == scop.PostCommitNonRevertiblePhase { for i, t := range bc.targetState.Targets { - if !rules.IsDescriptor(t.Element()) || t.TargetStatus != scpb.Status_ABSENT { + if !current.IsDescriptor(t.Element()) || t.TargetStatus != scpb.Status_ABSENT { continue } descID := screl.GetDescID(t.Element()) diff --git a/pkg/sql/sqlinstance/instancestorage/BUILD.bazel b/pkg/sql/sqlinstance/instancestorage/BUILD.bazel index 7d4b827648a4..9025d034d662 100644 --- a/pkg/sql/sqlinstance/instancestorage/BUILD.bazel +++ b/pkg/sql/sqlinstance/instancestorage/BUILD.bazel @@ -28,6 +28,7 @@ go_library( "//pkg/sql/catalog/systemschema", "//pkg/sql/enum", "//pkg/sql/rowenc/valueside", + "//pkg/sql/sem/catid", "//pkg/sql/sem/tree", "//pkg/sql/sqlinstance", "//pkg/sql/sqlliveness", diff --git a/pkg/sql/sqlinstance/instancestorage/instancereader.go b/pkg/sql/sqlinstance/instancestorage/instancereader.go index a52d20b2ac1a..216cc5fa0e7e 100644 --- a/pkg/sql/sqlinstance/instancestorage/instancereader.go +++ b/pkg/sql/sqlinstance/instancestorage/instancereader.go @@ -17,9 +17,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sqlinstance" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" "github.com/cockroachdb/cockroach/pkg/util/grpcutil" @@ -121,6 +123,50 @@ func (r *Reader) WaitForStarted(ctx context.Context) error { } } +func makeInstanceInfo(row instancerow) sqlinstance.InstanceInfo { + return sqlinstance.InstanceInfo{ + InstanceID: row.instanceID, + InstanceRPCAddr: row.rpcAddr, + InstanceSQLAddr: row.sqlAddr, + SessionID: row.sessionID, + Locality: row.locality, + } +} + +func makeInstanceInfos(rows []instancerow) []sqlinstance.InstanceInfo { + ret := make([]sqlinstance.InstanceInfo, len(rows)) + for i := range rows { + ret[i] = makeInstanceInfo(rows[i]) + } + return ret +} + +// GetAllInstancesUsingTxn reads all instances using the given transaction and returns +// live instances only. +func (r *Reader) GetAllInstancesUsingTxn( + ctx context.Context, codec keys.SQLCodec, tableID catid.DescID, txn *kv.Txn, +) ([]sqlinstance.InstanceInfo, error) { + instancesTablePrefix := codec.TablePrefix(uint32(tableID)) + rows, err := txn.Scan(ctx, instancesTablePrefix, instancesTablePrefix.PrefixEnd(), 0 /* maxRows */) + if err != nil { + return nil, err + } + rowcodec := makeRowCodec(codec, tableID) + decodedRows := make([]instancerow, 0, len(rows)) + for _, row := range rows { + decodedRow, err := rowcodec.decodeRow(row.Key, row.Value) + if err != nil { + return nil, err + } + decodedRows = append(decodedRows, decodedRow) + } + filteredRows, err := selectDistinctLiveRows(ctx, r.slReader, decodedRows) + if err != nil { + return nil, err + } + return makeInstanceInfos(filteredRows), nil +} + func (r *Reader) startRangeFeed(ctx context.Context) { updateCacheFn := func( ctx context.Context, keyVal *roachpb.RangeFeedValue, @@ -202,31 +248,22 @@ func (r *Reader) GetInstance( // This method does not block as the underlying sqlliveness.Reader // being used (outside of test environment) is a cached reader which // does not perform any RPCs in its `isAlive()` calls. -func (r *Reader) GetAllInstances( - ctx context.Context, -) (sqlInstances []sqlinstance.InstanceInfo, _ error) { +func (r *Reader) GetAllInstances(ctx context.Context) ([]sqlinstance.InstanceInfo, error) { if err := r.initialScanErr(); err != nil { return nil, err } - liveInstances, err := r.getAllLiveInstances(ctx) + liveInstances, err := selectDistinctLiveRows(ctx, r.slReader, r.getAllInstanceRows()) if err != nil { return nil, err } - for _, liveInstance := range liveInstances { - instanceInfo := sqlinstance.InstanceInfo{ - InstanceID: liveInstance.instanceID, - InstanceRPCAddr: liveInstance.rpcAddr, - InstanceSQLAddr: liveInstance.sqlAddr, - SessionID: liveInstance.sessionID, - Locality: liveInstance.locality, - } - sqlInstances = append(sqlInstances, instanceInfo) - } - return sqlInstances, nil + return makeInstanceInfos(liveInstances), nil } -func (r *Reader) getAllLiveInstances(ctx context.Context) ([]instancerow, error) { - rows := r.getAllInstanceRows() +// selectDistinctLiveRows modifies the given slice in-place and returns +// the selected rows. +func selectDistinctLiveRows( + ctx context.Context, slReader sqlliveness.Reader, rows []instancerow, +) ([]instancerow, error) { // Filter inactive instances. { truncated := rows[:0] @@ -235,7 +272,7 @@ func (r *Reader) getAllLiveInstances(ctx context.Context) ([]instancerow, error) if row.isAvailable() { continue } - isAlive, err := r.slReader.IsAlive(ctx, row.sessionID) + isAlive, err := slReader.IsAlive(ctx, row.sessionID) if err != nil { return nil, err } diff --git a/pkg/sql/sqlinstance/instancestorage/instancereader_test.go b/pkg/sql/sqlinstance/instancestorage/instancereader_test.go index c3902b41b05a..b628548f834c 100644 --- a/pkg/sql/sqlinstance/instancestorage/instancereader_test.go +++ b/pkg/sql/sqlinstance/instancestorage/instancereader_test.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" @@ -151,30 +152,80 @@ func TestReader(t *testing.T) { {Tiers: []roachpb.Tier{{Key: "region", Value: "region3"}}}, } - testOutputFn := func(expectedIDs []base.SQLInstanceID, expectedRPCAddresses, expectedSQLAddresses []string, expectedSessionIDs []sqlliveness.SessionID, expectedLocalities []roachpb.Locality, actualInstances []sqlinstance.InstanceInfo) error { - if len(expectedIDs) != len(actualInstances) { - return errors.Newf("expected %d instances, got %d instances", len(expectedIDs), len(actualInstances)) + type expectations struct { + instanceIDs []base.SQLInstanceID + rpcAddresses []string + sqlAddresses []string + sessionIDs []sqlliveness.SessionID + localities []roachpb.Locality + } + + testOutputFn := func(exp expectations, actualInstances []sqlinstance.InstanceInfo) error { + if len(exp.instanceIDs) != len(actualInstances) { + return errors.Newf("expected %d instances, got %d instances", len(exp.instanceIDs), len(actualInstances)) } for index, instance := range actualInstances { - if expectedIDs[index] != instance.InstanceID { - return errors.Newf("expected instance ID %d != actual instance ID %d", expectedIDs[index], instance.InstanceID) + if exp.instanceIDs[index] != instance.InstanceID { + return errors.Newf("expected instance ID %d != actual instance ID %d", exp.instanceIDs[index], instance.InstanceID) } - if expectedRPCAddresses[index] != instance.InstanceRPCAddr { - return errors.Newf("expected instance address %s != actual instance address %s", expectedRPCAddresses[index], instance.InstanceRPCAddr) + if exp.rpcAddresses[index] != instance.InstanceRPCAddr { + return errors.Newf("expected instance address %s != actual instance address %s", exp.rpcAddresses[index], instance.InstanceRPCAddr) } - if expectedSQLAddresses[index] != instance.InstanceSQLAddr { - return errors.Newf("expected instance address %s != actual instance address %s", expectedSQLAddresses[index], instance.InstanceSQLAddr) + if exp.sqlAddresses[index] != instance.InstanceSQLAddr { + return errors.Newf("expected instance address %s != actual instance address %s", exp.sqlAddresses[index], instance.InstanceSQLAddr) } - if expectedSessionIDs[index] != instance.SessionID { - return errors.Newf("expected session ID %s != actual session ID %s", expectedSessionIDs[index], instance.SessionID) + if exp.sessionIDs[index] != instance.SessionID { + return errors.Newf("expected session ID %s != actual session ID %s", exp.sessionIDs[index], instance.SessionID) } - if !expectedLocalities[index].Equals(instance.Locality) { - return errors.Newf("expected instance locality %s != actual instance locality %s", expectedLocalities[index], instance.Locality) + if !exp.localities[index].Equals(instance.Locality) { + return errors.Newf("expected instance locality %s != actual instance locality %s", exp.localities[index], instance.Locality) } } return nil } + + getInstancesUsingTxn := func(t *testing.T) ([]sqlinstance.InstanceInfo, error) { + var instancesUsingTxn []sqlinstance.InstanceInfo + if err := s.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + var err error + instancesUsingTxn, err = reader.GetAllInstancesUsingTxn(ctx, s.Codec(), getTableID(t, tDB, t.Name(), "sql_instances"), txn) + return err + }); err != nil { + return nil, err + } + return instancesUsingTxn, nil + } + + verifyInstancesWithGetter := func(t *testing.T, name string, exp expectations, getInstances func() ([]sqlinstance.InstanceInfo, error)) error { + instances, err := getInstances() + if err != nil { + return errors.Wrapf(err, "%s", name) + } + sortInstances(instances) + return errors.Wrapf(testOutputFn(exp, instances), "%s", name) + } + verifyInstances := func(t *testing.T, exp expectations) error { + if err := verifyInstancesWithGetter(t, "reader", exp, func() ([]sqlinstance.InstanceInfo, error) { + return reader.GetAllInstances(ctx) + }); err != nil { + return err + } + return verifyInstancesWithGetter(t, "txn", exp, func() ([]sqlinstance.InstanceInfo, error) { + return getInstancesUsingTxn(t) + }) + } + + expectationsFromOffset := func(offset int) expectations { + return expectations{ + instanceIDs: instanceIDs[offset:], + rpcAddresses: rpcAddresses[offset:], + sqlAddresses: sqlAddresses[offset:], + sessionIDs: sessionIDs[offset:], + localities: localities[offset:], + } + } + { // Set up mock data within instance and session storage. for index, rpcAddr := range rpcAddresses { @@ -190,12 +241,7 @@ func TestReader(t *testing.T) { } // Verify all instances are returned by GetAllInstances. testutils.SucceedsSoon(t, func() error { - instances, err := reader.GetAllInstances(ctx) - if err != nil { - return err - } - sortInstances(instances) - return testOutputFn(instanceIDs, rpcAddresses, sqlAddresses, sessionIDs, localities, instances) + return verifyInstances(t, expectationsFromOffset(0)) }) } @@ -206,12 +252,7 @@ func TestReader(t *testing.T) { t.Fatal(err) } testutils.SucceedsSoon(t, func() error { - instances, err := reader.GetAllInstances(ctx) - if err != nil { - return err - } - sortInstances(instances) - return testOutputFn(instanceIDs[1:], rpcAddresses[1:], sqlAddresses[1:], sessionIDs[1:], localities[1:], instances) + return verifyInstances(t, expectationsFromOffset(1)) }) } @@ -222,12 +263,7 @@ func TestReader(t *testing.T) { t.Fatal(err) } testutils.SucceedsSoon(t, func() error { - instances, err := reader.GetAllInstances(ctx) - if err != nil { - return err - } - sortInstances(instances) - return testOutputFn(instanceIDs[2:], rpcAddresses[2:], sqlAddresses[2:], sessionIDs[2:], localities[2:], instances) + return verifyInstances(t, expectationsFromOffset(2)) }) } @@ -247,19 +283,13 @@ func TestReader(t *testing.T) { t.Fatal(err) } testutils.SucceedsSoon(t, func() error { - instances, err := reader.GetAllInstances(ctx) - if err != nil { - return err - } - sortInstances(instances) - return testOutputFn( - []base.SQLInstanceID{instance.InstanceID}, - []string{rpcAddresses[2]}, - []string{sqlAddresses[2]}, - []sqlliveness.SessionID{sessionID}, - []roachpb.Locality{locality}, - instances, - ) + return verifyInstances(t, expectations{ + []base.SQLInstanceID{instance.InstanceID}, /* instanceIDs */ + []string{rpcAddresses[2]}, /* rpcAddresses */ + []string{sqlAddresses[2]}, /* sqlAddresses */ + []sqlliveness.SessionID{sessionID}, /* sessionIDs */ + []roachpb.Locality{locality}, /* localities */ + }) }) } }) diff --git a/pkg/sql/sqlinstance/instancestorage/test_helpers.go b/pkg/sql/sqlinstance/instancestorage/test_helpers.go index b0da5e1b9f93..db673a8f1cec 100644 --- a/pkg/sql/sqlinstance/instancestorage/test_helpers.go +++ b/pkg/sql/sqlinstance/instancestorage/test_helpers.go @@ -138,25 +138,15 @@ func (s *Storage) GetInstanceDataForTest( // testing purposes. func (s *Storage) GetAllInstancesDataForTest( ctx context.Context, -) (instances []sqlinstance.InstanceInfo, err error) { +) ([]sqlinstance.InstanceInfo, error) { var rows []instancerow ctx = multitenant.WithTenantCostControlExemption(ctx) - err = s.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + if err := s.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + var err error rows, err = s.getInstanceRows(ctx, nil /*global*/, txn, lock.WaitPolicy_Block) return err - }) - if err != nil { + }); err != nil { return nil, err } - for _, instance := range rows { - instanceInfo := sqlinstance.InstanceInfo{ - InstanceID: instance.instanceID, - InstanceRPCAddr: instance.rpcAddr, - InstanceSQLAddr: instance.sqlAddr, - SessionID: instance.sessionID, - Locality: instance.locality, - } - instances = append(instances, instanceInfo) - } - return instances, nil + return makeInstanceInfos(rows), nil } diff --git a/pkg/sql/table.go b/pkg/sql/table.go index 0c7a9747bbcd..c5cc7a17ae4b 100644 --- a/pkg/sql/table.go +++ b/pkg/sql/table.go @@ -43,7 +43,7 @@ func (p *planner) createDropDatabaseJob( tableDropDetails []jobspb.DroppedTableDetails, typesToDrop []*typedesc.Mutable, jobDesc string, -) error { +) { // TODO (lucy): This should probably be deleting the queued jobs for all the // tables being dropped, so that we don't have duplicate schema changers. tableIDs := make([]descpb.ID, 0, len(tableDropDetails)) @@ -54,7 +54,7 @@ func (p *planner) createDropDatabaseJob( for _, t := range typesToDrop { typeIDs = append(typeIDs, t.ID) } - jobRecord := jobs.Record{ + jobRecord := &jobs.Record{ Description: jobDesc, Username: p.User(), DescriptorIDs: tableIDs, @@ -68,12 +68,8 @@ func (p *planner) createDropDatabaseJob( Progress: jobspb.SchemaChangeProgress{}, NonCancelable: true, } - newJob, err := p.extendedEvalCtx.QueueJob(ctx, p.InternalSQLTxn(), jobRecord) - if err != nil { - return err - } - log.Infof(ctx, "queued new drop database job %d for database %d", newJob.ID(), databaseID) - return nil + jobID := p.extendedEvalCtx.QueueJob(jobRecord) + log.Infof(ctx, "queued new drop database job %d for database %d", jobID, databaseID) } // CreateNonDropDatabaseChangeJob covers all database descriptor updates other @@ -82,8 +78,8 @@ func (p *planner) createDropDatabaseJob( // don't queue multiple jobs for the same database. func (p *planner) createNonDropDatabaseChangeJob( ctx context.Context, databaseID descpb.ID, jobDesc string, -) error { - jobRecord := jobs.Record{ +) { + jobRecord := &jobs.Record{ Description: jobDesc, Username: p.User(), Details: jobspb.SchemaChangeDetails{ @@ -93,12 +89,8 @@ func (p *planner) createNonDropDatabaseChangeJob( Progress: jobspb.SchemaChangeProgress{}, NonCancelable: true, } - newJob, err := p.extendedEvalCtx.QueueJob(ctx, p.InternalSQLTxn(), jobRecord) - if err != nil { - return err - } - log.Infof(ctx, "queued new database schema change job %d for database %d", newJob.ID(), databaseID) - return nil + jobID := p.extendedEvalCtx.QueueJob(jobRecord) + log.Infof(ctx, "queued new database schema change job %d for database %d", jobID, databaseID) } // createOrUpdateSchemaChangeJob queues a new job for the schema change if there @@ -116,7 +108,7 @@ func (p *planner) createOrUpdateSchemaChangeJob( return scerrors.ConcurrentSchemaChangeError(tableDesc) } - record, recordExists := p.extendedEvalCtx.SchemaChangeJobRecords[tableDesc.ID] + record, recordExists := p.extendedEvalCtx.jobs.uniqueToCreate[tableDesc.ID] if p.extendedEvalCtx.ExecCfg.TestingKnobs.RunAfterSCJobsCacheLookup != nil { p.extendedEvalCtx.ExecCfg.TestingKnobs.RunAfterSCJobsCacheLookup(record) } @@ -174,7 +166,7 @@ func (p *planner) createOrUpdateSchemaChangeJob( // have mutations, e.g., in CREATE TABLE AS VALUES. NonCancelable: mutationID == descpb.InvalidMutationID && !tableDesc.Adding(), } - p.extendedEvalCtx.SchemaChangeJobRecords[tableDesc.ID] = &newRecord + p.extendedEvalCtx.jobs.uniqueToCreate[tableDesc.ID] = &newRecord // Only add a MutationJob if there's an associated mutation. // TODO (lucy): get rid of this when we get rid of MutationJobs. if mutationID != descpb.InvalidMutationID { diff --git a/pkg/sql/tenant_deletion.go b/pkg/sql/tenant_deletion.go index eca0554e4be6..3fa5aa0f2ea9 100644 --- a/pkg/sql/tenant_deletion.go +++ b/pkg/sql/tenant_deletion.go @@ -45,7 +45,7 @@ func (p *planner) DropTenantByID( p.ExecCfg().Settings, p.InternalSQLTxn(), p.ExecCfg().JobRegistry, - p.extendedEvalCtx.Jobs, + p.extendedEvalCtx.jobs, p.User(), info, synchronousImmediateDrop, @@ -70,7 +70,7 @@ func dropTenantInternal( settings *cluster.Settings, txn isql.Txn, jobRegistry *jobs.Registry, - sessionJobs *jobsCollection, + sessionJobs *txnJobsCollection, user username.SQLUsername, info *mtinfopb.TenantInfo, synchronousImmediateDrop bool, @@ -124,7 +124,7 @@ func dropTenantInternal( return errors.Wrap(err, "scheduling gc job") } if synchronousImmediateDrop { - sessionJobs.add(jobID) + sessionJobs.addCreatedJobID(jobID) } return nil } diff --git a/pkg/sql/type_change.go b/pkg/sql/type_change.go index e0263c89b715..e13d3c59f517 100644 --- a/pkg/sql/type_change.go +++ b/pkg/sql/type_change.go @@ -103,7 +103,7 @@ func (p *planner) writeTypeSchemaChange( ctx context.Context, typeDesc *typedesc.Mutable, jobDesc string, ) error { // Check if there is a cached specification for this type, otherwise create one. - record, recordExists := p.extendedEvalCtx.SchemaChangeJobRecords[typeDesc.ID] + record, recordExists := p.extendedEvalCtx.jobs.uniqueToCreate[typeDesc.ID] transitioningMembers, beingDropped := findTransitioningMembers(typeDesc) if recordExists { // Update it. @@ -142,7 +142,7 @@ func (p *planner) writeTypeSchemaChange( // a transition that drops an enum member. NonCancelable: !beingDropped, } - p.extendedEvalCtx.SchemaChangeJobRecords[typeDesc.ID] = &newRecord + p.extendedEvalCtx.jobs.uniqueToCreate[typeDesc.ID] = &newRecord log.Infof(ctx, "queued new type change job %d for type %d", newRecord.JobID, typeDesc.ID) } diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index 31c3282a4db0..7f3c47eeb1b2 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -987,12 +987,29 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (p *Pebble, err error) { // disk is stalled. While the logging subsystem should also be robust to // stalls and crash the process if unable to write logs, there's less risk // to sequencing the crashing listener first. + // + // For the same reason, make the logging call asynchronous for DiskSlow events. + // This prevents slow logging calls during a disk slow/stall event from holding + // up Pebble's internal disk health checking, and better obeys the + // EventListener contract for not having any functions block or take a while to + // run. Creating goroutines is acceptable given their low cost, and the low + // write concurrency to Pebble's FS (Pebble compactions + flushes + SQL + // spilling to disk). If the maximum concurrency of DiskSlow events increases + // significantly in the future, we can improve the logic here by queueing up + // most of the logging work (except for the Fatalf call), and have it be done + // by a single goroutine. + lel := pebble.MakeLoggingEventListener(pebbleLogger{ + ctx: logCtx, + depth: 2, // skip over the EventListener stack frame + }) + oldDiskSlow := lel.DiskSlow + lel.DiskSlow = func(info pebble.DiskSlowInfo) { + // Run oldDiskSlow asynchronously. + go oldDiskSlow(info) + } el := pebble.TeeEventListener( p.makeMetricEtcEventListener(ctx), - pebble.MakeLoggingEventListener(pebbleLogger{ - ctx: logCtx, - depth: 2, // skip over the EventListener stack frame - }), + lel, ) p.eventListener = &el @@ -1071,10 +1088,16 @@ func (p *Pebble) makeMetricEtcEventListener(ctx context.Context) pebble.EventLis atomic.AddInt64(&p.diskStallCount, 1) // Note that the below log messages go to the main cockroach log, not // the pebble-specific log. + // + // Run non-fatal log.* calls in separate goroutines as they could block + // if the logging device is also slow/stalling, preventing pebble's disk + // health checking from functioning correctly. See the comment in + // pebble.EventListener on why it's important for this method to return + // quickly. if fatalOnExceeded { log.Fatalf(ctx, "file write stall detected: %s", info) } else { - log.Errorf(ctx, "file write stall detected: %s", info) + go log.Errorf(ctx, "file write stall detected: %s", info) } return }