diff --git a/Makefile b/Makefile index 0408cf500eea..e26a6850ebd9 100644 --- a/Makefile +++ b/Makefile @@ -887,12 +887,14 @@ EXECGEN_TARGETS = \ pkg/sql/colexec/colexecwindow/lag.eg.go \ pkg/sql/colexec/colexecwindow/last_value.eg.go \ pkg/sql/colexec/colexecwindow/lead.eg.go \ + pkg/sql/colexec/colexecwindow/min_max_removable_agg.eg.go \ pkg/sql/colexec/colexecwindow/ntile.eg.go \ pkg/sql/colexec/colexecwindow/nth_value.eg.go \ pkg/sql/colexec/colexecwindow/range_offset_handler.eg.go \ pkg/sql/colexec/colexecwindow/rank.eg.go \ pkg/sql/colexec/colexecwindow/relative_rank.eg.go \ pkg/sql/colexec/colexecwindow/row_number.eg.go \ + pkg/sql/colexec/colexecwindow/window_aggregator.eg.go \ pkg/sql/colexec/colexecwindow/window_framer.eg.go \ pkg/sql/colexec/colexecwindow/window_peer_grouper.eg.go diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index c91492e00a9b..2b537facc2cd 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -152,4 +152,4 @@ trace.datadog.project string CockroachDB the project under which traces will be trace.debug.enable boolean false if set, traces for recent requests can be seen at https:///debug/requests trace.lightstep.token string if set, traces go to Lightstep using this token trace.zipkin.collector string if set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'). Only one tracer can be configured at a time. -version version 21.1-130 set the active cluster version in the format '.' +version version 21.1-132 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 842d0ba157d1..586c80212b0c 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -156,6 +156,6 @@ trace.debug.enablebooleanfalseif set, traces for recent requests can be seen at https:///debug/requests trace.lightstep.tokenstringif set, traces go to Lightstep using this token trace.zipkin.collectorstringif set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'). Only one tracer can be configured at a time. -versionversion21.1-130set the active cluster version in the format '.' +versionversion21.1-132set the active cluster version in the format '.' diff --git a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go index 3ad38a870b29..e7b01e6b5b57 100644 --- a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go +++ b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go @@ -665,9 +665,10 @@ func TestClusterRestoreFailCleanup(t *testing.T) { }, } } - // The initial restore will return an error, and restart. sqlDBRestore.ExpectErr(t, `injected error: restarting in background`, `RESTORE FROM $1`, LocalFoo) + // Reduce retry delays. + sqlDBRestore.Exec(t, "SET CLUSTER SETTING jobs.registry.retry.initial_delay = '1ms'") // Expect the restore to succeed. sqlDBRestore.CheckQueryResultsRetry(t, `SELECT count(*) FROM [SHOW JOBS] WHERE job_type = 'RESTORE' AND status = 'succeeded'`, diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index b227578ab66e..c417ac945141 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -280,6 +280,8 @@ const ( // PostSeparatedIntentsMigration runs a cleanup migration after the main // SeparatedIntentsMigration. PostSeparatedIntentsMigration + // RetryJobsWithExponentialBackoff retries failed jobs with exponential delays. + RetryJobsWithExponentialBackoff // Step (1): Add new versions here. ) @@ -461,6 +463,10 @@ var versionsSingleton = keyedVersions{ Key: PostSeparatedIntentsMigration, Version: roachpb.Version{Major: 21, Minor: 1, Internal: 130}, }, + { + Key: RetryJobsWithExponentialBackoff, + Version: roachpb.Version{Major: 21, Minor: 1, Internal: 132}, + }, // Step (2): Add new versions here. } diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index f11b817a5cdc..e4e41a461e62 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -44,11 +44,12 @@ func _() { _ = x[AlterSystemWebSessionsCreateIndexes-33] _ = x[SeparatedIntentsMigration-34] _ = x[PostSeparatedIntentsMigration-35] + _ = x[RetryJobsWithExponentialBackoff-36] } -const _Key_name = "Start20_2NodeMembershipStatusMinPasswordLengthAbortSpanBytesCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1CPutInlineReplicaVersionsreplacedTruncatedAndRangeAppliedStateMigrationreplacedPostTruncatedAndRangeAppliedStateMigrationTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigrationSeparatedIntentsTracingVerbosityIndependentSemanticsClosedTimestampsRaftTransportPriorReadSummariesNonVotingReplicasV21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsSQLStatsTableDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrorsAlterSystemWebSessionsCreateIndexesSeparatedIntentsMigrationPostSeparatedIntentsMigration" +const _Key_name = "Start20_2NodeMembershipStatusMinPasswordLengthAbortSpanBytesCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1CPutInlineReplicaVersionsreplacedTruncatedAndRangeAppliedStateMigrationreplacedPostTruncatedAndRangeAppliedStateMigrationTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigrationSeparatedIntentsTracingVerbosityIndependentSemanticsClosedTimestampsRaftTransportPriorReadSummariesNonVotingReplicasV21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsSQLStatsTableDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrorsAlterSystemWebSessionsCreateIndexesSeparatedIntentsMigrationPostSeparatedIntentsMigrationRetryJobsWithExponentialBackoff" -var _Key_index = [...]uint16{0, 9, 29, 46, 60, 80, 92, 97, 106, 116, 131, 177, 227, 265, 307, 323, 359, 388, 406, 423, 428, 441, 450, 465, 494, 511, 528, 577, 591, 604, 624, 640, 657, 684, 719, 744, 773} +var _Key_index = [...]uint16{0, 9, 29, 46, 60, 80, 92, 97, 106, 116, 131, 177, 227, 265, 307, 323, 359, 388, 406, 423, 428, 441, 450, 465, 494, 511, 528, 577, 591, 604, 624, 640, 657, 684, 719, 744, 773, 804} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/jobs/BUILD.bazel b/pkg/jobs/BUILD.bazel index 1102f052152b..97512a009155 100644 --- a/pkg/jobs/BUILD.bazel +++ b/pkg/jobs/BUILD.bazel @@ -24,6 +24,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", + "//pkg/clusterversion", "//pkg/jobs/jobspb", "//pkg/kv", "//pkg/roachpb:with-mocks", diff --git a/pkg/jobs/adopt.go b/pkg/jobs/adopt.go index 6decbd6bb469..dc890868451d 100644 --- a/pkg/jobs/adopt.go +++ b/pkg/jobs/adopt.go @@ -16,6 +16,7 @@ import ( "strconv" "sync" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -43,16 +44,16 @@ const ( // NonTerminalStatusTupleString is a sql tuple corresponding to statuses of // non-terminal jobs. NonTerminalStatusTupleString = `(` + nonTerminalStatusList + `)` -) -const claimQuery = ` + claimQuery = ` UPDATE system.jobs SET claim_session_id = $1, claim_instance_id = $2 - WHERE (claim_session_id IS NULL) - AND (status IN ` + claimableStatusTupleString + `) + WHERE ((claim_session_id IS NULL) + AND (status IN ` + claimableStatusTupleString + `)) ORDER BY created DESC LIMIT $3 RETURNING id;` +) func (r *Registry) maybeDumpTrace( resumerCtx context.Context, resumer Resumer, jobID, traceID int64, jobErr error, @@ -96,11 +97,11 @@ func (r *Registry) claimJobs(ctx context.Context, s sqlliveness.Session) error { } numRows, err := r.ex.Exec( ctx, "claim-jobs", txn, claimQuery, - s.ID().UnsafeBytes(), r.ID(), maxAdoptionsPerLoop, - ) + s.ID().UnsafeBytes(), r.ID(), maxAdoptionsPerLoop) if err != nil { return errors.Wrap(err, "could not query jobs table") } + r.metrics.ClaimedJobs.Inc(int64(numRows)) if log.ExpensiveLogEnabled(ctx, 1) || numRows > 0 { log.Infof(ctx, "claimed %d jobs", numRows) } @@ -108,14 +109,75 @@ func (r *Registry) claimJobs(ctx context.Context, s sqlliveness.Session) error { }) } +const ( + // processQueryStatusTupleString includes the states of a job in which a + // job can be claimed and resumed. + processQueryStatusTupleString = `(` + + `'` + string(StatusRunning) + `', ` + + `'` + string(StatusReverting) + `'` + + `)` + + // canRunArgs are used in canRunClause, which specify whether a job can be + // run now or not. + canRunArgs = `(SELECT $3::TIMESTAMP AS ts, $4::FLOAT AS initial_delay, $5::FLOAT AS max_delay) args` + canRunClause = ` +args.ts >= COALESCE(last_run, created) + least( + IF( + args.initial_delay * (power(2, least(62, COALESCE(num_runs, 0))) - 1)::FLOAT >= 0.0, + args.initial_delay * (power(2, least(62, COALESCE(num_runs, 0))) - 1)::FLOAT, + args.max_delay + ), + args.max_delay +)::INTERVAL +` + // processQueryBase and processQueryWhereBase select IDs of the jobs that + // can be processed among the claimed jobs. + processQueryBase = `SELECT id FROM system.jobs` + processQueryWhereBase = ` status IN ` + processQueryStatusTupleString + ` AND (claim_session_id = $1 AND claim_instance_id = $2)` + + processQueryWithoutBackoff = processQueryBase + " WHERE " + processQueryWhereBase + processQueryWithBackoff = processQueryBase + ", " + canRunArgs + + " WHERE " + processQueryWhereBase + " AND " + canRunClause + + resumeQueryBaseCols = "status, payload, progress, crdb_internal.sql_liveness_is_alive(claim_session_id)" + resumeQueryWhereBase = `id = $1 AND claim_session_id = $2` + resumeQueryWithoutBackoff = `SELECT ` + resumeQueryBaseCols + ` FROM system.jobs WHERE ` + resumeQueryWhereBase + resumeQueryWithBackoff = `SELECT ` + resumeQueryBaseCols + `, ` + canRunClause + ` AS can_run` + + ` FROM system.jobs, ` + canRunArgs + " WHERE " + resumeQueryWhereBase +) + +// getProcessQuery returns the query that selects the jobs that are claimed +// by this node. +func getProcessQuery( + ctx context.Context, s sqlliveness.Session, r *Registry, +) (string, []interface{}) { + // Select the running or reverting jobs that this node has claimed. + query := processQueryWithoutBackoff + args := []interface{}{s.ID().UnsafeBytes(), r.ID()} + // Gating the version that introduced job retries with exponential backoff. + if r.settings.Version.IsActive(ctx, clusterversion.RetryJobsWithExponentialBackoff) { + // Select only those jobs that can be executed right now. + query = processQueryWithBackoff + initDelay := retryInitialDelaySetting.Get(&r.settings.SV).Seconds() + maxDelay := retryMaxDelaySetting.Get(&r.settings.SV).Seconds() + if r.knobs.IntervalOverrides.RetryInitialDelay != nil { + initDelay = r.knobs.IntervalOverrides.RetryInitialDelay.Seconds() + } + if r.knobs.IntervalOverrides.RetryMaxDelay != nil { + maxDelay = r.knobs.IntervalOverrides.RetryMaxDelay.Seconds() + } + args = append(args, r.clock.Now().GoTime(), initDelay, maxDelay) + } + return query, args +} + // processClaimedJobs processes all jobs currently claimed by the registry. func (r *Registry) processClaimedJobs(ctx context.Context, s sqlliveness.Session) error { + query, args := getProcessQuery(ctx, s, r) + it, err := r.ex.QueryIteratorEx( ctx, "select-running/get-claimed-jobs", nil, - sessiondata.InternalExecutorOverride{User: security.NodeUserName()}, ` -SELECT id FROM system.jobs -WHERE (status = $1 OR status = $2) AND (claim_session_id = $3 AND claim_instance_id = $4)`, - StatusRunning, StatusReverting, s.ID().UnsafeBytes(), r.ID(), + sessiondata.InternalExecutorOverride{User: security.NodeUserName()}, query, args..., ) if err != nil { return errors.Wrapf(err, "could not query for claimed jobs") @@ -134,7 +196,6 @@ WHERE (status = $1 OR status = $2) AND (claim_session_id = $3 AND claim_instance if err != nil { return errors.Wrapf(err, "could not query for claimed jobs") } - r.filterAlreadyRunningAndCancelFromPreviousSessions(ctx, s, claimedToResume) r.resumeClaimedJobs(ctx, s, claimedToResume) return nil @@ -190,12 +251,24 @@ func (r *Registry) filterAlreadyRunningAndCancelFromPreviousSessions( // resumeJob resumes a claimed job. func (r *Registry) resumeJob(ctx context.Context, jobID jobspb.JobID, s sqlliveness.Session) error { log.Infof(ctx, "job %d: resuming execution", jobID) + resumeQuery := resumeQueryWithoutBackoff + args := []interface{}{jobID, s.ID().UnsafeBytes()} + backoffIsActive := r.settings.Version.IsActive(ctx, clusterversion.RetryJobsWithExponentialBackoff) + if backoffIsActive { + resumeQuery = resumeQueryWithBackoff + initDelay := retryInitialDelaySetting.Get(&r.settings.SV).Seconds() + maxDelay := retryMaxDelaySetting.Get(&r.settings.SV).Seconds() + if r.knobs.IntervalOverrides.RetryInitialDelay != nil { + initDelay = r.knobs.IntervalOverrides.RetryInitialDelay.Seconds() + } + if r.knobs.IntervalOverrides.RetryMaxDelay != nil { + maxDelay = r.knobs.IntervalOverrides.RetryMaxDelay.Seconds() + } + args = append(args, r.clock.Now().GoTime(), initDelay, maxDelay) + } row, err := r.ex.QueryRowEx( ctx, "get-job-row", nil, - sessiondata.InternalExecutorOverride{User: security.NodeUserName()}, ` -SELECT status, payload, progress, crdb_internal.sql_liveness_is_alive(claim_session_id) -FROM system.jobs WHERE id = $1 AND claim_session_id = $2`, - jobID, s.ID().UnsafeBytes(), + sessiondata.InternalExecutorOverride{User: security.NodeUserName()}, resumeQuery, args..., ) if err != nil { return errors.Wrapf(err, "job %d: could not query job table row", jobID) @@ -218,6 +291,26 @@ FROM system.jobs WHERE id = $1 AND claim_session_id = $2`, return errors.Errorf("job %d: claim with session id %s has expired", jobID, s.ID()) } + if backoffIsActive { + // It's too soon to run the job. + // + // We need this check to address a race between adopt-loop and an existing + // resumer, e.g., in the following schedule: + // Adopt loop: Cl(j,n1) St(r1) Cl(j, n1) St(r2) + // Resumer 1: Rg(j) Up(n1->n2) Fl(j) Ur(j) + // Resumer 2: x-| Starting too soon + // Where: + // - Cl(j,nx): claim job j when num_runs is x + // - St(r1): start resumer r1 + // - Rg(j): Add jobID of j in adoptedJobs, disabling further resumers + // - Ur(j): Remove jobID of j from adoptedJobs, enabling further resumers + // - Up(n1->2): Update number of runs from 1 to 2 + // - Fl(j): Job j fails + if !(*row[4].(*tree.DBool)) { + return nil + } + } + payload, err := UnmarshalPayload(row[1]) if err != nil { return err @@ -248,6 +341,7 @@ FROM system.jobs WHERE id = $1 AND claim_session_id = $2`, aj := &adoptedJob{sid: s.ID(), cancel: cancel} r.addAdoptedJob(jobID, aj) + r.metrics.ResumedJobs.Inc(1) if err := r.stopper.RunAsyncTask(ctx, job.taskName(), func(ctx context.Context) { // Wait for the job to finish. No need to print the error because if there // was one it's been set in the job status already. @@ -266,6 +360,9 @@ func (r *Registry) removeAdoptedJob(jobID jobspb.JobID) { } func (r *Registry) addAdoptedJob(jobID jobspb.JobID, aj *adoptedJob) { + // TODO(sajjad): We should check whether adoptedJobs already has jobID or not. If + // the ID exists, we should not add it again and the caller should not start + // another resumer. r.mu.Lock() defer r.mu.Unlock() r.mu.adoptedJobs[jobID] = aj @@ -334,16 +431,18 @@ func (r *Registry) runJob( return err } -const cancelQuery = ` -UPDATE system.jobs -SET status = - CASE - WHEN status = $1 THEN $2 - WHEN status = $3 THEN $4 - ELSE status - END -WHERE (status IN ($1, $3)) AND ((claim_session_id = $5) AND (claim_instance_id = $6)) -RETURNING id, status` +const pauseAndCancelUpdate = ` + UPDATE system.jobs + SET status = + CASE + WHEN status = '` + string(StatusPauseRequested) + `' THEN '` + string(StatusPaused) + `' + WHEN status = '` + string(StatusCancelRequested) + `' THEN '` + string(StatusReverting) + `' + ELSE status + END + WHERE (status IN ('` + string(StatusPauseRequested) + `', '` + string(StatusCancelRequested) + `')) + AND ((claim_session_id = $1) AND (claim_instance_id = $2)) +RETURNING id, status +` func (r *Registry) servePauseAndCancelRequests(ctx context.Context, s sqlliveness.Session) error { return r.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { @@ -358,10 +457,7 @@ func (r *Registry) servePauseAndCancelRequests(ctx context.Context, s sqllivenes // registry). rows, err := r.ex.QueryBufferedEx( ctx, "cancel/pause-requested", txn, sessiondata.InternalExecutorOverride{User: security.NodeUserName()}, - cancelQuery, - StatusPauseRequested, StatusPaused, - StatusCancelRequested, StatusReverting, - s.ID().UnsafeBytes(), r.ID(), + pauseAndCancelUpdate, s.ID().UnsafeBytes(), r.ID(), ) if err != nil { return errors.Wrap(err, "could not query jobs table") @@ -381,6 +477,12 @@ func (r *Registry) servePauseAndCancelRequests(ctx context.Context, s sqllivenes encodedErr := errors.EncodeError(ctx, errJobCanceled) md.Payload.FinalResumeError = &encodedErr ju.UpdatePayload(md.Payload) + if r.settings.Version.IsActive(ctx, clusterversion.RetryJobsWithExponentialBackoff) { + // When we cancel a job, we want to reset its last_run and num_runs + // so that the job can be picked-up in the next adopt-loop, sooner + // than its current next-retry time. + ju.UpdateRunStats(0 /* numRuns */, r.clock.Now().GoTime() /* lastRun */) + } return nil }); err != nil { return errors.Wrapf(err, "job %d: tried to cancel but could not mark as reverting: %s", id, err) diff --git a/pkg/jobs/config.go b/pkg/jobs/config.go index 1d759ae96cdc..f618ec2297af 100644 --- a/pkg/jobs/config.go +++ b/pkg/jobs/config.go @@ -20,32 +20,45 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) -const intervalBaseSettingKey = "jobs.registry.interval.base" -const adoptIntervalSettingKey = "jobs.registry.interval.adopt" -const cancelIntervalSettingKey = "jobs.registry.interval.cancel" -const gcIntervalSettingKey = "jobs.registry.interval.gc" -const retentionTimeSettingKey = "jobs.retention_time" -const cancelUpdateLimitKey = "jobs.cancel_update_limit" +const ( + intervalBaseSettingKey = "jobs.registry.interval.base" + adoptIntervalSettingKey = "jobs.registry.interval.adopt" + cancelIntervalSettingKey = "jobs.registry.interval.cancel" + gcIntervalSettingKey = "jobs.registry.interval.gc" + retentionTimeSettingKey = "jobs.retention_time" + cancelUpdateLimitKey = "jobs.cancel_update_limit" + retryInitialDelaySettingKey = "jobs.registry.retry.initial_delay" + retryMaxDelaySettingKey = "jobs.registry.retry.max_delay" +) + +var ( + // defaultAdoptInterval is the default adopt interval. + defaultAdoptInterval = 30 * time.Second -// defaultAdoptInterval is the default adopt interval. -var defaultAdoptInterval = 30 * time.Second + // defaultCancelInterval is the default cancel interval. + defaultCancelInterval = 10 * time.Second -// defaultCancelInterval is the default cancel interval. -var defaultCancelInterval = 10 * time.Second + // defaultGcInterval is the default GC Interval. + defaultGcInterval = 1 * time.Hour -// defaultGcInterval is the default GC Interval. -var defaultGcInterval = 1 * time.Hour + // defaultIntervalBase is the default interval base. + defaultIntervalBase = 1.0 -// defaultIntervalBase is the default interval base. -var defaultIntervalBase = 1.0 + // defaultRetentionTime is the default duration for which terminal jobs are + // kept in the records. + defaultRetentionTime = 14 * 24 * time.Hour -// defaultRetentionTime is the default duration for which terminal jobs are -// kept in the records. -var defaultRetentionTime = 14 * 24 * time.Hour + // defaultCancellationsUpdateLimit is the default number of jobs that can be + // updated when canceling jobs concurrently from dead sessions. + defaultCancellationsUpdateLimit int64 = 1000 -// defaultCancellationsUpdateLimit is the default number of jobs that can be -// updated when canceling jobs concurrently from dead sessions. -var defaultCancellationsUpdateLimit int64 = 1000 + // defaultRetryInitialDelay is the initial delay in the calculation of exponentially + // increasing delays to retry failed jobs. + defaultRetryInitialDelay = 30 * time.Second + + // defaultRetryMaxDelay is the maximum delay to retry a failed job. + defaultRetryMaxDelay = 24 * time.Hour +) var ( intervalBaseSetting = settings.RegisterFloatSetting( @@ -93,6 +106,22 @@ var ( defaultCancellationsUpdateLimit, settings.NonNegativeInt, ) + + retryInitialDelaySetting = settings.RegisterDurationSetting( + retryInitialDelaySettingKey, + "the starting duration of exponential-backoff delay"+ + " to retry a job which encountered a retryable error or had its coordinator"+ + " fail. The delay doubles after each retry.", + defaultRetryInitialDelay, + settings.NonNegativeDuration, + ) + + retryMaxDelaySetting = settings.RegisterDurationSetting( + retryMaxDelaySettingKey, + "the maximum duration by which a job can be delayed to retry", + defaultRetryMaxDelay, + settings.PositiveDuration, + ) ) // jitter adds a small jitter in the given duration. diff --git a/pkg/jobs/executor_impl_test.go b/pkg/jobs/executor_impl_test.go index 45e1a725e351..a7414d8e67ec 100644 --- a/pkg/jobs/executor_impl_test.go +++ b/pkg/jobs/executor_impl_test.go @@ -28,7 +28,7 @@ func TestInlineExecutorFailedJobsHandling(t *testing.T) { defer log.Scope(t).Close(t) argsFn := func(args *base.TestServerArgs) { - args.Knobs.JobsTestingKnobs = NewTestingKnobsWithIntervals(time.Millisecond, time.Millisecond) + args.Knobs.JobsTestingKnobs = NewTestingKnobsWithShortIntervals() } h, cleanup := newTestHelperWithServerArgs(t, argsFn) diff --git a/pkg/jobs/helpers_test.go b/pkg/jobs/helpers_test.go index c5baa5048ecd..8bcbf7b49d7f 100644 --- a/pkg/jobs/helpers_test.go +++ b/pkg/jobs/helpers_test.go @@ -92,10 +92,14 @@ func (j *Job) Succeeded(ctx context.Context) error { var ( AdoptQuery = claimQuery - CancelQuery = cancelQuery + CancelQuery = pauseAndCancelUpdate GcQuery = expiredJobsQuery + RemoveClaimsQuery = removeClaimsQuery + + ProcessJobsQuery = processQueryWithBackoff + IntervalBaseSettingKey = intervalBaseSettingKey AdoptIntervalSettingKey = adoptIntervalSettingKey diff --git a/pkg/jobs/jobs.go b/pkg/jobs/jobs.go index f53c97025b7e..547e98255b58 100644 --- a/pkg/jobs/jobs.go +++ b/pkg/jobs/jobs.go @@ -16,6 +16,7 @@ import ( "reflect" "sync/atomic" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security" @@ -273,17 +274,21 @@ func (j *Job) taskName() string { return fmt.Sprintf(`job-%d`, j.ID()) } -// Started marks the tracked job as started. +// Started marks the tracked job as started by updating status to running in +// jobs table. func (j *Job) started(ctx context.Context, txn *kv.Txn) error { return j.Update(ctx, txn, func(_ *kv.Txn, md JobMetadata, ju *JobUpdater) error { if md.Status != StatusPending && md.Status != StatusRunning { return errors.Errorf("job with status %s cannot be marked started", md.Status) } - // TODO(spaskob): Remove this status change after we stop supporting - // pending job states. - ju.UpdateStatus(StatusRunning) - md.Payload.StartedMicros = timeutil.ToUnixMicros(j.registry.clock.Now().GoTime()) - ju.UpdatePayload(md.Payload) + if md.Payload.StartedMicros == 0 { + ju.UpdateStatus(StatusRunning) + md.Payload.StartedMicros = timeutil.ToUnixMicros(j.registry.clock.Now().GoTime()) + ju.UpdatePayload(md.Payload) + } + if j.registry.settings.Version.IsActive(ctx, clusterversion.RetryJobsWithExponentialBackoff) { + ju.UpdateRunStats(md.RunStats.NumRuns+1, j.registry.clock.Now().GoTime()) + } return nil }) } @@ -464,7 +469,8 @@ func (j *Job) cancelRequested( } if md.Status == StatusPaused && md.Payload.FinalResumeError != nil { decodedErr := errors.DecodeError(ctx, *md.Payload.FinalResumeError) - return fmt.Errorf("job %d is paused and has non-nil FinalResumeError %s hence cannot be canceled and should be reverted", j.ID(), decodedErr.Error()) + return fmt.Errorf("job %d is paused and has non-nil FinalResumeError "+ + "%s hence cannot be canceled and should be reverted", j.ID(), decodedErr.Error()) } if fn != nil { if err := fn(ctx, txn); err != nil { @@ -529,29 +535,43 @@ func (j *Job) reverted( ctx context.Context, txn *kv.Txn, err error, fn func(context.Context, *kv.Txn) error, ) error { return j.Update(ctx, txn, func(txn *kv.Txn, md JobMetadata, ju *JobUpdater) error { - if md.Status == StatusReverting { - return nil - } - if md.Status != StatusCancelRequested && md.Status != StatusRunning && md.Status != StatusPending { + if md.Status != StatusReverting && + md.Status != StatusCancelRequested && + md.Status != StatusRunning && + md.Status != StatusPending { return fmt.Errorf("job with status %s cannot be reverted", md.Status) } - if fn != nil { - if err := fn(ctx, txn); err != nil { - return err + if md.Status != StatusReverting { + if fn != nil { + if err := fn(ctx, txn); err != nil { + return err + } + } + if err != nil { + md.Payload.Error = err.Error() + encodedErr := errors.EncodeError(ctx, err) + md.Payload.FinalResumeError = &encodedErr + ju.UpdatePayload(md.Payload) + } else { + if md.Payload.FinalResumeError == nil { + return errors.AssertionFailedf( + "tried to mark job as reverting, but no error was provided or recorded") + } } + ju.UpdateStatus(StatusReverting) } - if err != nil { - md.Payload.Error = err.Error() - encodedErr := errors.EncodeError(ctx, err) - md.Payload.FinalResumeError = &encodedErr - ju.UpdatePayload(md.Payload) - } else { - if md.Payload.FinalResumeError == nil { - return errors.AssertionFailedf( - "tried to mark job as reverting, but no error was provided or recorded") + if j.registry.settings.Version.IsActive(ctx, clusterversion.RetryJobsWithExponentialBackoff) { + // We can reach here due to a failure or due to the job being canceled. + // We should reset the exponential backoff parameters if the job was not + // canceled. Note that md.Status will be StatusReverting if the job + // was canceled. + numRuns := md.RunStats.NumRuns + 1 + if md.Status != StatusReverting { + // Reset the number of runs to speed up reverting. + numRuns = 1 } + ju.UpdateRunStats(numRuns, j.registry.clock.Now().GoTime()) } - ju.UpdateStatus(StatusReverting) return nil }) } @@ -594,6 +614,9 @@ func (j *Job) failed( return err } } + // TODO (sajjad): We don't have any checks for state transitions here. Consequently, + // a pause-requested job can transition to failed, which may or may not be + // acceptable depending on the job. ju.UpdateStatus(StatusFailed) md.Payload.Error = err.Error() md.Payload.FinishedMicros = timeutil.ToUnixMicros(j.registry.clock.Now().GoTime()) diff --git a/pkg/jobs/jobs_test.go b/pkg/jobs/jobs_test.go index 7af9302d68c5..6dfe250ea238 100644 --- a/pkg/jobs/jobs_test.go +++ b/pkg/jobs/jobs_test.go @@ -2288,7 +2288,7 @@ func TestJobInTxn(t *testing.T) { // Set the adoption interval to be very long to test the adoption channel. args := base.TestServerArgs{Knobs: base.TestingKnobs{ - JobsTestingKnobs: jobs.NewTestingKnobsWithIntervals(time.Hour, time.Hour)}, + JobsTestingKnobs: jobs.NewTestingKnobsWithIntervals(time.Hour, time.Hour, time.Hour, time.Hour)}, } ctx := context.Background() s, sqlDB, _ := serverutils.StartServer(t, args) @@ -2885,7 +2885,7 @@ func TestMetrics(t *testing.T) { ) { jobConstructorCleanup := jobs.ResetConstructors() args := base.TestServerArgs{Knobs: base.TestingKnobs{ - JobsTestingKnobs: jobs.NewTestingKnobsWithIntervals(time.Millisecond, time.Millisecond)}, + JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()}, } s, db, _ = serverutils.StartServer(t, args) r = s.JobRegistry().(*jobs.Registry) @@ -3054,7 +3054,7 @@ func TestLoseLeaseDuringExecution(t *testing.T) { defer jobs.ResetConstructors()() // Disable the loops from messing with the job execution. - knobs := base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithIntervals(time.Hour, time.Hour)} + knobs := base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithIntervals(time.Hour, time.Hour, time.Hour, time.Hour)} ctx := context.Background() diff --git a/pkg/jobs/jobspb/jobs.pb.go b/pkg/jobs/jobspb/jobs.pb.go index a4d488b4d84e..076591bea886 100644 --- a/pkg/jobs/jobspb/jobs.pb.go +++ b/pkg/jobs/jobspb/jobs.pb.go @@ -2561,7 +2561,7 @@ func (*Progress) XXX_OneofWrappers() []interface{} { type Job struct { Id JobID `protobuf:"varint,1,opt,name=id,proto3,customtype=JobID" json:"id"` - // Keep progress first as it may bre more relevant to see when looking at a + // Keep progress first as it may be more relevant to see when looking at a // running job. Progress *Progress `protobuf:"bytes,2,opt,name=progress,proto3" json:"progress,omitempty"` Payload *Payload `protobuf:"bytes,3,opt,name=payload,proto3" json:"payload,omitempty"` diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index 341a1b548c1c..a6cf624a26d3 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -760,7 +760,7 @@ enum Type { message Job { int64 id = 1 [(gogoproto.nullable) = false, (gogoproto.customtype) = "JobID"]; - // Keep progress first as it may bre more relevant to see when looking at a + // Keep progress first as it may be more relevant to see when looking at a // running job. Progress progress = 2; Payload payload = 3; diff --git a/pkg/jobs/lease_test.go b/pkg/jobs/lease_test.go index e82cd5f3259f..ba9c2d027fcf 100644 --- a/pkg/jobs/lease_test.go +++ b/pkg/jobs/lease_test.go @@ -14,11 +14,13 @@ import ( "context" "strings" "testing" + "time" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/stretchr/testify/require" ) @@ -32,20 +34,25 @@ func TestJobsTableClaimFamily(t *testing.T) { sqlDB := sqlutils.MakeSQLRunner(db) var table, schema string sqlDB.QueryRow(t, `SHOW CREATE system.jobs`).Scan(&table, &schema) - if !strings.Contains(schema, `FAMILY claim (claim_session_id, claim_instance_id)`) { + if !strings.Contains( + schema, `FAMILY claim (claim_session_id, claim_instance_id, num_runs, last_run)`, + ) { t.Fatalf("expected claim family, got %q", schema) } + now := timeutil.Now() _ = sqlDB.Query(t, ` -INSERT INTO system.jobs (id, status, payload, claim_session_id, claim_instance_id) -VALUES (1, 'running', '@!%$%45', 'foo', 101)`) - +INSERT INTO system.jobs (id, status, payload, claim_session_id, claim_instance_id, num_runs, last_run) +VALUES (1, 'running', '@!%$%45', 'foo', 101, 100, $1)`, now) var status, sessionID string - var instanceID int64 - const stmt = "SELECT status, claim_session_id, claim_instance_id FROM system.jobs WHERE id = $1" - sqlDB.QueryRow(t, stmt, 1).Scan(&status, &sessionID, &instanceID) + var instanceID, numRuns int64 + var lastRun time.Time + const stmt = "SELECT status, claim_session_id, claim_instance_id, num_runs, last_run FROM system.jobs WHERE id = $1" + sqlDB.QueryRow(t, stmt, 1).Scan(&status, &sessionID, &instanceID, &numRuns, &lastRun) require.Equal(t, "running", status) require.Equal(t, "foo", sessionID) require.Equal(t, int64(101), instanceID) + require.Equal(t, int64(100), numRuns) + require.Equal(t, timeutil.ToUnixMicros(now), timeutil.ToUnixMicros(lastRun)) } diff --git a/pkg/jobs/metrics.go b/pkg/jobs/metrics.go index 00d46df58484..339c10ccbfc7 100644 --- a/pkg/jobs/metrics.go +++ b/pkg/jobs/metrics.go @@ -26,6 +26,17 @@ type Metrics struct { JobMetrics [jobspb.NumJobTypes]*JobTypeMetrics Changefeed metric.Struct + + // AdoptIterations counts the number of adopt loops executed by Registry. + AdoptIterations *metric.Counter + + // ClaimedJobs counts the number of jobs claimed in adopt loops. + ClaimedJobs *metric.Counter + + // ResumedJobs counts the number of jobs resumed by Registry. It doesn't + // correlate with the ClaimedJobs counter because a job can be resumed + // without an adopt loop, e.g., through a StartableJob. + ResumedJobs *metric.Counter } // JobTypeMetrics is a metric.Struct containing metrics for each type of job. @@ -122,6 +133,32 @@ func makeMetaFailOrCancelFailed(typeStr string) metric.Metadata { } } +var ( + metaAdoptIterations = metric.Metadata{ + Name: "jobs.adopt_iterations", + Help: "number of job-adopt iterations performed by the registry", + Measurement: "iterations", + Unit: metric.Unit_COUNT, + MetricType: io_prometheus_client.MetricType_GAUGE, + } + + metaClaimedJobs = metric.Metadata{ + Name: "jobs.claimed_jobs", + Help: "number of jobs claimed in job-adopt iterations", + Measurement: "jobs", + Unit: metric.Unit_COUNT, + MetricType: io_prometheus_client.MetricType_GAUGE, + } + + metaResumedClaimedJobs = metric.Metadata{ + Name: "jobs.resumed_claimed_jobs", + Help: "number of claimed-jobs resumed in job-adopt iterations", + Measurement: "jobs", + Unit: metric.Unit_COUNT, + MetricType: io_prometheus_client.MetricType_GAUGE, + } +) + // MetricStruct implements the metric.Struct interface. func (Metrics) MetricStruct() {} @@ -130,6 +167,9 @@ func (m *Metrics) init(histogramWindowInterval time.Duration) { if MakeChangefeedMetricsHook != nil { m.Changefeed = MakeChangefeedMetricsHook(histogramWindowInterval) } + m.AdoptIterations = metric.NewCounter(metaAdoptIterations) + m.ClaimedJobs = metric.NewCounter(metaClaimedJobs) + m.ResumedJobs = metric.NewCounter(metaResumedClaimedJobs) for i := 0; i < jobspb.NumJobTypes; i++ { jt := jobspb.Type(i) if jt == jobspb.TypeUnspecified { // do not track TypeUnspecified diff --git a/pkg/jobs/registry.go b/pkg/jobs/registry.go index bf1bfd94525a..e86a528d19d8 100644 --- a/pkg/jobs/registry.go +++ b/pkg/jobs/registry.go @@ -195,6 +195,9 @@ func MakeRegistry( } if knobs != nil { r.knobs = *knobs + if knobs.TimeSource != nil { + r.clock = knobs.TimeSource + } } r.mu.adoptedJobs = make(map[jobspb.JobID]*adoptedJob) r.metrics.init(histogramWindowInterval) @@ -540,7 +543,6 @@ VALUES ($1, $2, $3, $4, $5, $6)`, jobID, StatusRunning, payloadBytes, progressBy ); err != nil { return nil, err } - return j, nil } @@ -711,8 +713,20 @@ func (r *Registry) UpdateJobWithTxn( return j.update(ctx, txn, useReadLock, updateFunc) } +// TODO (sajjad): make maxAdoptionsPerLoop a cluster setting. var maxAdoptionsPerLoop = envutil.EnvOrDefaultInt(`COCKROACH_JOB_ADOPTIONS_PER_PERIOD`, 10) +const removeClaimsQuery = ` +UPDATE system.jobs + SET claim_session_id = NULL + WHERE claim_session_id in ( +SELECT claim_session_id + WHERE claim_session_id <> $1 + AND status IN ` + claimableStatusTupleString + ` + AND NOT crdb_internal.sql_liveness_is_alive(claim_session_id) + FETCH FIRST $2 ROWS ONLY) +` + // Start polls the current node for liveness failures and cancels all registered // jobs if it observes a failure. Otherwise it starts all the main daemons of // registry that poll the jobs table and start/cancel/gc jobs. @@ -747,16 +761,10 @@ func (r *Registry) Start(ctx context.Context, stopper *stop.Stopper) error { } _, err := r.ex.ExecEx( ctx, "expire-sessions", nil, - sessiondata.InternalExecutorOverride{User: security.RootUserName()}, ` -UPDATE system.jobs - SET claim_session_id = NULL -WHERE claim_session_id in ( -SELECT claim_session_id - WHERE claim_session_id <> $1 - AND status IN `+claimableStatusTupleString+` - AND NOT crdb_internal.sql_liveness_is_alive(claim_session_id) FETCH - FIRST `+strconv.Itoa(int(cancellationsUpdateLimitSetting.Get(&r.settings.SV)))+` ROWS ONLY)`, + sessiondata.InternalExecutorOverride{User: security.RootUserName()}, + removeClaimsQuery, s.ID().UnsafeBytes(), + cancellationsUpdateLimitSetting.Get(&r.settings.SV), ) return err }); err != nil { @@ -779,6 +787,7 @@ SELECT claim_session_id // claimJobs iterates the set of jobs which are not currently claimed and // claims jobs up to maxAdoptionsPerLoop. claimJobs := withSession(func(ctx context.Context, s sqlliveness.Session) { + r.metrics.AdoptIterations.Inc(1) if err := r.claimJobs(ctx, s); err != nil { log.Errorf(ctx, "error claiming jobs: %s", err) } @@ -907,6 +916,8 @@ func (r *Registry) cleanupOldJobs(ctx context.Context, olderThan time.Time) erro } } +// TODO (sajjad): Why are we returning column 'created' in this query? It's not +// being used. const expiredJobsQuery = "SELECT id, payload, status, created FROM system.jobs " + "WHERE (created < $1) AND (id > $2) " + "ORDER BY id " + // the ordering is important as we keep track of the maximum ID we've seen @@ -1154,6 +1165,7 @@ func (r *Registry) stepThroughStateMachine( jobType := payload.Type() log.Infof(ctx, "%s job %d: stepping through state %s with error: %+v", jobType, job.ID(), status, jobErr) jm := r.metrics.JobMetrics[jobType] + switch status { case StatusRunning: if jobErr != nil { @@ -1161,11 +1173,11 @@ func (r *Registry) stepThroughStateMachine( "job %d: resuming with non-nil error", job.ID()) } resumeCtx := logtags.AddTag(ctx, "job", job.ID()) - if payload.StartedMicros == 0 { - if err := job.started(ctx, nil /* txn */); err != nil { - return err - } + + if err := job.started(ctx, nil /* txn */); err != nil { + return err } + var err error func() { jm.CurrentlyRunning.Inc(1) @@ -1186,8 +1198,6 @@ func (r *Registry) stepThroughStateMachine( return errors.Errorf("job %d: node liveness error: restarting in background", job.ID()) } // TODO(spaskob): enforce a limit on retries. - // TODO(spaskob,lucy): Add metrics on job retries. Consider having a backoff - // mechanism (possibly combined with a retry limit). if errors.Is(err, retryJobErrorSentinel) { jm.ResumeRetryError.Inc(1) return errors.Errorf("job %d: %s: restarting in background", job.ID(), err) diff --git a/pkg/jobs/registry_external_test.go b/pkg/jobs/registry_external_test.go index 11b71c36272e..05c96767f6f8 100644 --- a/pkg/jobs/registry_external_test.go +++ b/pkg/jobs/registry_external_test.go @@ -85,7 +85,7 @@ func TestExpiringSessionsAndClaimJobsDoesNotTouchTerminalJobs(t *testing.T) { adopt := 10 * time.Hour cancel := 10 * time.Millisecond args := base.TestServerArgs{Knobs: base.TestingKnobs{ - JobsTestingKnobs: jobs.NewTestingKnobsWithIntervals(adopt, cancel), + JobsTestingKnobs: jobs.NewTestingKnobsWithIntervals(adopt, cancel, adopt, adopt), }} ctx := context.Background() diff --git a/pkg/jobs/registry_test.go b/pkg/jobs/registry_test.go index 498ae65cf9cc..124ea7078ad7 100644 --- a/pkg/jobs/registry_test.go +++ b/pkg/jobs/registry_test.go @@ -12,8 +12,12 @@ package jobs import ( "context" + gosql "database/sql" "fmt" + "math" "strconv" + "strings" + "sync/atomic" "testing" "time" @@ -28,13 +32,18 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -347,3 +356,496 @@ func TestBatchJobsCreation(t *testing.T) { }) } } + +// TestRetriesWithExponentialBackoff tests the working of exponential delays +// when jobs are retried. Moreover, it tests the effectiveness of the upper +// bound on the retry delay. +func TestRetriesWithExponentialBackoff(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + const ( + // Number of retries should be reasonably large such that they are sufficient + // to test long delays but not too many to increase the test time. + retries = 50 + + unitTime = time.Millisecond + // initDelay and maxDelay can be large as we jump in time through a fake clock. + initialDelay = time.Second + maxDelay = time.Hour + + pause = true + cancel = false + ) + + // createJob creates a mock job. + createJob := func( + ctx context.Context, s serverutils.TestServerInterface, r *Registry, tdb *sqlutils.SQLRunner, kvDB *kv.DB, + ) (jobspb.JobID, time.Time) { + jobID := r.MakeJobID() + require.NoError(t, kvDB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + _, err := r.CreateJobWithTxn(ctx, Record{ + Details: jobspb.ImportDetails{}, + Progress: jobspb.ImportProgress{}, + }, jobID, txn) + return err + })) + var lastRun time.Time + tdb.QueryRow(t, + "SELECT created FROM system.jobs where id = $1", jobID, + ).Scan(&lastRun) + return jobID, lastRun + } + validateResumeCounts := func(t *testing.T, expected, found int64) { + require.Equal(t, expected, found, "unexpected number of jobs resumed") + } + waitUntilCount := func(t *testing.T, counter *metric.Counter, count int64) { + testutils.SucceedsSoon(t, func() error { + cnt := counter.Count() + if cnt >= count { + return nil + } + return errors.Errorf( + "waiting for %v to reach %d, currently at %d", counter.GetName(), count, cnt, + ) + }) + } + waitUntilStatus := func(t *testing.T, tdb *sqlutils.SQLRunner, jobID jobspb.JobID, status Status) { + tdb.CheckQueryResultsRetry(t, + fmt.Sprintf("SELECT status FROM [SHOW JOBS] WHERE job_id = %d", jobID), + [][]string{{string(status)}}) + } + // pauseOrCancelJob pauses or cancels a job. If pauseJob is true, the job is paused, + // otherwise the job is canceled. + pauseOrCancelJob := func( + t *testing.T, ctx context.Context, db *kv.DB, registry *Registry, jobID jobspb.JobID, pauseJob bool, + ) { + assert.NoError(t, db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + if pauseJob { + return registry.PauseRequested(ctx, txn, jobID) + } + return registry.CancelRequested(ctx, txn, jobID) + })) + } + // nextDelay returns the next delay based calculated from the given retryCnt + // and exponential-backoff parameters. + nextDelay := func(retryCnt int, initDelay time.Duration, maxDelay time.Duration) time.Duration { + delay := initDelay * ((1 << int(math.Min(62, float64(retryCnt)))) - 1) + if delay < 0 { + delay = maxDelay + } + return time.Duration(math.Min(float64(delay), float64(maxDelay))) + } + + type BackoffTestInfra struct { + s serverutils.TestServerInterface + tdb *sqlutils.SQLRunner + kvDB *kv.DB + registry *Registry + clock *timeutil.ManualTime + resumeCh chan struct{} + failOrCancelCh chan struct{} + transitionCh chan struct{} + errCh chan error + done atomic.Value + jobMetrics *JobTypeMetrics + adopted *metric.Counter + resumed *metric.Counter + afterJobStateMachineKnob func() + } + testInfraSetUp := func(ctx context.Context, bti *BackoffTestInfra) func() { + // We use a manual clock to control and evaluate job execution times. + // We initialize the clock with Now() because the job-creation timestamp, + // 'created' column in system.jobs, of a new job is set from txn's time. + bti.clock = timeutil.NewManualTime(timeutil.Now()) + timeSource := hlc.NewClock(func() int64 { + return bti.clock.Now().UnixNano() + }, base.DefaultMaxClockOffset) + // Set up the test cluster. + knobs := &TestingKnobs{ + TimeSource: timeSource, + } + if bti.afterJobStateMachineKnob != nil { + knobs.AfterJobStateMachine = bti.afterJobStateMachineKnob + } + cs := cluster.MakeTestingClusterSettings() + // Set a small adopt and cancel intervals to reduce test time. + adoptIntervalSetting.Override(ctx, &cs.SV, unitTime) + cancelIntervalSetting.Override(ctx, &cs.SV, unitTime) + retryInitialDelaySetting.Override(ctx, &cs.SV, initialDelay) + retryMaxDelaySetting.Override(ctx, &cs.SV, maxDelay) + args := base.TestServerArgs{ + Settings: cs, + Knobs: base.TestingKnobs{ + JobsTestingKnobs: knobs, + }, + } + var sqlDB *gosql.DB + bti.s, sqlDB, bti.kvDB = serverutils.StartServer(t, args) + cleanup := func() { + close(bti.errCh) + close(bti.resumeCh) + close(bti.failOrCancelCh) + bti.s.Stopper().Stop(ctx) + } + bti.tdb = sqlutils.MakeSQLRunner(sqlDB) + bti.registry = bti.s.JobRegistry().(*Registry) + bti.resumeCh = make(chan struct{}) + bti.failOrCancelCh = make(chan struct{}) + bti.transitionCh = make(chan struct{}) + bti.errCh = make(chan error) + bti.done.Store(false) + bti.jobMetrics = bti.registry.metrics.JobMetrics[jobspb.TypeImport] + bti.adopted = bti.registry.metrics.AdoptIterations + bti.resumed = bti.registry.metrics.ResumedJobs + RegisterConstructor(jobspb.TypeImport, func(job *Job, cs *cluster.Settings) Resumer { + return FakeResumer{ + OnResume: func(ctx context.Context) error { + if bti.done.Load().(bool) { + return nil + } + bti.resumeCh <- struct{}{} + return <-bti.errCh + }, + FailOrCancel: func(ctx context.Context) error { + if bti.done.Load().(bool) { + return nil + } + bti.failOrCancelCh <- struct{}{} + return <-bti.errCh + }, + } + }) + return cleanup + } + + runTest := func(t *testing.T, jobID jobspb.JobID, retryCnt int, expectedResumed int64, lastRun time.Time, bti *BackoffTestInfra, waitFn func(int64)) { + // We retry for a fixed number. For each retry: + // - We first advance the clock such that it is a little behind the + // next retry time. + // - We then wait until a few adopt-loops complete to ensure that + // the registry gets a chance to pick up a job if it can. + // - We validate that the number of resumed jobs has not increased to + // ensure that a job is not started/resumed/reverted before its next + // retry time. + // - We then advance the clock to the exact next retry time. Now the job + // can be retried in the next adopt-loop. + // - We wait until the resumer completes one execution and the job needs + // to be picked up again in the next adopt-loop. + // - Now we validate that resumedJobs counter has increment, which ensures + // that the job has completed only one cycle in this time. + // + // If retries do not happen based on exponential-backoff times, our counters + // will not match, causing the test to fail in validateCount. + + for i := 0; i < retries; i++ { + // Exponential delay in the next retry. + delay := nextDelay(retryCnt, initialDelay, maxDelay) + // The delay must not exceed the max delay setting. It ensures that + // we are expecting correct timings from our test code, which in turn + // ensures that the jobs are resumed with correct exponential delays. + require.GreaterOrEqual(t, maxDelay, delay, "delay exceeds the max") + // Advance the clock such that it is before the next expected retry time. + bti.clock.AdvanceTo(lastRun.Add(delay - unitTime)) + // This allows adopt-loops to run for a few times, which ensures that + // adopt-loops do not resume jobs without correctly following the job + // schedules. + waitUntilCount(t, bti.adopted, bti.adopted.Count()+2) + // Validate that the job is not resumed yet. + validateResumeCounts(t, expectedResumed, bti.resumed.Count()) + // Advance the clock by delta from the expected time of next retry. + bti.clock.Advance(unitTime) + // Wait until the resumer completes its execution. + waitFn(int64(retryCnt)) + expectedResumed++ + retryCnt++ + // Validate that the job is resumed only once. + validateResumeCounts(t, expectedResumed, bti.resumed.Count()) + lastRun = bti.clock.Now() + } + bti.done.Store(true) + // Let the job be retried one more time. + bti.clock.Advance(nextDelay(retryCnt, initialDelay, maxDelay)) + // Wait until the job completes. + testutils.SucceedsSoon(t, func() error { + var found Status + bti.tdb.QueryRow(t, "SELECT status FROM system.jobs WHERE id = $1", jobID).Scan(&found) + if found.Terminal() { + return nil + } + retryCnt++ + bti.clock.Advance(nextDelay(retryCnt, initialDelay, maxDelay)) + return errors.Errorf("waiting job %d to reach a terminal state, currently %s", jobID, found) + }) + } + + t.Run("running", func(t *testing.T) { + ctx := context.Background() + bti := BackoffTestInfra{} + bti.afterJobStateMachineKnob = func() { + if bti.done.Load().(bool) { + return + } + bti.transitionCh <- struct{}{} + } + cleanup := testInfraSetUp(ctx, &bti) + defer cleanup() + + jobID, lastRun := createJob(ctx, bti.s, bti.registry, bti.tdb, bti.kvDB) + retryCnt := 0 + expectedResumed := int64(0) + runTest(t, jobID, retryCnt, expectedResumed, lastRun, &bti, func(_ int64) { + <-bti.resumeCh + bti.errCh <- NewRetryJobError("injecting error to retry running") + <-bti.transitionCh + }) + }) + + t.Run("pause running", func(t *testing.T) { + ctx := context.Background() + bti := BackoffTestInfra{} + bti.afterJobStateMachineKnob = func() { + if bti.done.Load().(bool) { + return + } + bti.transitionCh <- struct{}{} + } + cleanup := testInfraSetUp(ctx, &bti) + defer cleanup() + + jobID, lastRun := createJob(ctx, bti.s, bti.registry, bti.tdb, bti.kvDB) + retryCnt := 0 + expectedResumed := int64(0) + runTest(t, jobID, retryCnt, expectedResumed, lastRun, &bti, func(_ int64) { + <-bti.resumeCh + pauseOrCancelJob(t, ctx, bti.kvDB, bti.registry, jobID, pause) + bti.errCh <- nil + <-bti.transitionCh + waitUntilStatus(t, bti.tdb, jobID, StatusPaused) + require.NoError(t, bti.registry.Unpause(ctx, nil, jobID)) + }) + }) + + t.Run("revert on fail", func(t *testing.T) { + ctx := context.Background() + bti := BackoffTestInfra{} + bti.afterJobStateMachineKnob = func() { + if bti.done.Load().(bool) { + return + } + bti.transitionCh <- struct{}{} + } + cleanup := testInfraSetUp(ctx, &bti) + defer cleanup() + + jobID, lastRun := createJob(ctx, bti.s, bti.registry, bti.tdb, bti.kvDB) + bti.clock.AdvanceTo(lastRun) + <-bti.resumeCh + bti.errCh <- errors.Errorf("injecting error to revert") + <-bti.failOrCancelCh + bti.errCh <- NewRetryJobError("injecting error in reverting state to retry") + <-bti.transitionCh + expectedResumed := bti.resumed.Count() + retryCnt := 1 + runTest(t, jobID, retryCnt, expectedResumed, lastRun, &bti, func(_ int64) { + <-bti.failOrCancelCh + bti.errCh <- NewRetryJobError("injecting error in reverting state to retry") + <-bti.transitionCh + }) + }) + + t.Run("revert on cancel", func(t *testing.T) { + ctx := context.Background() + bti := BackoffTestInfra{} + cleanup := testInfraSetUp(ctx, &bti) + defer cleanup() + + jobID, lastRun := createJob(ctx, bti.s, bti.registry, bti.tdb, bti.kvDB) + bti.clock.AdvanceTo(lastRun) + <-bti.resumeCh + pauseOrCancelJob(t, ctx, bti.kvDB, bti.registry, jobID, cancel) + bti.errCh <- nil + <-bti.failOrCancelCh + bti.errCh <- NewRetryJobError("injecting error in reverting state") + expectedResumed := bti.resumed.Count() + retryCnt := 1 + runTest(t, jobID, retryCnt, expectedResumed, lastRun, &bti, func(retryCnt int64) { + <-bti.failOrCancelCh + bti.errCh <- NewRetryJobError("injecting error in reverting state") + waitUntilCount(t, bti.jobMetrics.FailOrCancelRetryError, retryCnt+1) + }) + }) + + t.Run("pause reverting", func(t *testing.T) { + ctx := context.Background() + bti := BackoffTestInfra{} + bti.afterJobStateMachineKnob = func() { + if bti.done.Load().(bool) { + return + } + bti.transitionCh <- struct{}{} + } + cleanup := testInfraSetUp(ctx, &bti) + defer cleanup() + + jobID, lastRun := createJob(ctx, bti.s, bti.registry, bti.tdb, bti.kvDB) + bti.clock.AdvanceTo(lastRun) + <-bti.resumeCh + bti.errCh <- errors.Errorf("injecting error to revert") + <-bti.failOrCancelCh + bti.errCh <- NewRetryJobError("injecting error in reverting state to retry") + <-bti.transitionCh + expectedResumed := bti.resumed.Count() + retryCnt := 1 + runTest(t, jobID, retryCnt, expectedResumed, lastRun, &bti, func(_ int64) { + <-bti.failOrCancelCh + pauseOrCancelJob(t, ctx, bti.kvDB, bti.registry, jobID, pause) + // We have to return error here because, otherwise, the job will be marked as + // failed regardless of the fact that it is currently pause-requested in the + // jobs table. This is because we currently do not check the current status + // of a job before marking it as failed. + bti.errCh <- NewRetryJobError("injecting error in reverting state to retry") + <-bti.transitionCh + waitUntilStatus(t, bti.tdb, jobID, StatusPaused) + require.NoError(t, bti.registry.Unpause(ctx, nil, jobID)) + }) + }) +} + +// TestExponentialBackoffSettings tests the cluster settings of exponential backoff delays. +func TestExponentialBackoffSettings(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + for _, test := range [...]struct { + name string // Test case ID. + // The setting to test. + settingKey string + // The value of the setting to set. + value time.Duration + }{ + { + name: "backoff initial delay setting", + settingKey: retryInitialDelaySettingKey, + value: 2 * time.Millisecond, + }, + { + name: "backoff max delay setting", + settingKey: retryMaxDelaySettingKey, + value: 2 * time.Millisecond, + }, + } { + t.Run(test.name, func(t *testing.T) { + ctx := context.Background() + var tdb *sqlutils.SQLRunner + var finished atomic.Value + finished.Store(false) + var intercepted atomic.Value + intercepted.Store(false) + intercept := func(orig, updated JobMetadata) error { + // If this updated is not to mark as succeeded or the test has already failed. + if updated.Status != StatusSucceeded { + return nil + } + + // If marking the first time, prevent the marking and update the cluster + // setting based on test params. The setting value should be reduced + // from a large value to a small value. + if !intercepted.Load().(bool) { + tdb.Exec(t, fmt.Sprintf("SET CLUSTER SETTING %s = '%v'", test.settingKey, test.value)) + intercepted.Store(true) + return errors.Errorf("preventing the job from succeeding") + } + // Let the job to succeed. As we began with a long interval and prevented + // the job than succeeding in the first attempt, its re-execution + // indicates that the setting is updated successfully and is in effect. + finished.Store(true) + return nil + } + + // Setup the test cluster. + cs := cluster.MakeTestingClusterSettings() + // Set a small adopt interval to reduce test time. + adoptIntervalSetting.Override(ctx, &cs.SV, 2*time.Millisecond) + // Begin with a long delay. + retryInitialDelaySetting.Override(ctx, &cs.SV, time.Hour) + retryMaxDelaySetting.Override(ctx, &cs.SV, time.Hour) + args := base.TestServerArgs{ + Settings: cs, + Knobs: base.TestingKnobs{JobsTestingKnobs: &TestingKnobs{BeforeUpdate: intercept}}, + } + s, sdb, kvDB := serverutils.StartServer(t, args) + defer s.Stopper().Stop(ctx) + tdb = sqlutils.MakeSQLRunner(sdb) + // Create and run a dummy job. + RegisterConstructor(jobspb.TypeImport, func(_ *Job, cs *cluster.Settings) Resumer { + return FakeResumer{} + }) + registry := s.JobRegistry().(*Registry) + id := registry.MakeJobID() + require.NoError(t, kvDB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + _, err := registry.CreateJobWithTxn(ctx, Record{ + // Job does not accept an empty Details field, so arbitrarily provide + // ImportDetails. + Details: jobspb.ImportDetails{}, + Progress: jobspb.ImportProgress{}, + }, id, txn) + return err + })) + + // Wait for the job to be succeed. + testutils.SucceedsSoon(t, func() error { + if finished.Load().(bool) { + return nil + } + return errors.Errorf("waiting for the job to complete") + }) + }) + } +} + +func TestRegistryUsePartialIndex(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + // Mock args. + const ( + id = 1 + sid = "bytes" + iid = 1 + lim = 10 + d = time.Millisecond + ) + ts := timeutil.Now() + + for _, test := range []struct { + name string + query string + queryArgs []interface{} + }{ + {"remove claims", RemoveClaimsQuery, []interface{}{sid, lim}}, + {"claim jobs", AdoptQuery, []interface{}{sid, iid, lim}}, + {"process claimed jobs", ProcessJobsQuery, []interface{}{sid, iid, ts, d, d}}, + {"serve cancel and pause", CancelQuery, []interface{}{sid, iid}}, + } { + t.Run(test.name, func(t *testing.T) { + ctx := context.Background() + s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer s.Stopper().Stop(ctx) + tdb := sqlutils.MakeSQLRunner(sqlDB) + + usingIndex := false + rows := tdb.Query(t, "EXPLAIN "+test.query, test.queryArgs...) + for rows.Next() { + var line string + require.NoError(t, rows.Scan(&line)) + if strings.Contains(line, "table: jobs@jobs_run_stats_idx (partial index)") { + usingIndex = true + break + } + } + require.NoError(t, rows.Close()) + require.True(t, usingIndex, "partial index is not used") + }) + } +} diff --git a/pkg/jobs/schedule_control_test.go b/pkg/jobs/schedule_control_test.go index 6b8a7bb26c88..01dd44ac0ba3 100644 --- a/pkg/jobs/schedule_control_test.go +++ b/pkg/jobs/schedule_control_test.go @@ -254,7 +254,7 @@ func TestFilterJobsControlForSchedules(t *testing.T) { argsFn := func(args *base.TestServerArgs) { // Prevent registry from changing job state while running this test. interval := 24 * time.Hour - args.Knobs.JobsTestingKnobs = NewTestingKnobsWithIntervals(interval, interval) + args.Knobs.JobsTestingKnobs = NewTestingKnobsWithIntervals(interval, interval, interval, interval) } th, cleanup := newTestHelperForTables(t, jobstest.UseSystemTables, argsFn) defer cleanup() diff --git a/pkg/jobs/testing_knobs.go b/pkg/jobs/testing_knobs.go index 241978ac53c3..03d2e34ff9f5 100644 --- a/pkg/jobs/testing_knobs.go +++ b/pkg/jobs/testing_knobs.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/scheduledjobs" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util/hlc" ) // TestingKnobs are base.ModuleTestingKnobs for testing jobs related infra. @@ -59,8 +60,14 @@ type TestingKnobs struct { // returned from the state machine that transitions it from one state to // another. AfterJobStateMachine func() + + // TimeSource replaces registry's clock. + TimeSource *hlc.Clock } +// ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. +func (*TestingKnobs) ModuleTestingKnobs() {} + // TestingIntervalOverrides contains variables to override the intervals and // settings of periodic tasks. type TestingIntervalOverrides struct { @@ -73,30 +80,36 @@ type TestingIntervalOverrides struct { // Gc overrides the gcIntervalSetting cluster setting. Gc *time.Duration - // Base overrides the intervalBaseSetting cluster setting. - Base *float64 - // RetentionTime overrides the retentionTimeSetting cluster setting. RetentionTime *time.Duration -} -// ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. -func (*TestingKnobs) ModuleTestingKnobs() {} + // RetryInitialDelay overrides retryInitialDelaySetting cluster setting. + RetryInitialDelay *time.Duration + + // RetryMaxDelay overrides retryMaxDelaySetting cluster setting. + RetryMaxDelay *time.Duration +} // NewTestingKnobsWithShortIntervals return a TestingKnobs structure with // overrides for short adopt and cancel intervals. func NewTestingKnobsWithShortIntervals() *TestingKnobs { const defaultShortInterval = 10 * time.Millisecond - return NewTestingKnobsWithIntervals(defaultShortInterval, defaultShortInterval) + return NewTestingKnobsWithIntervals( + defaultShortInterval, defaultShortInterval, defaultShortInterval, defaultShortInterval, + ) } // NewTestingKnobsWithIntervals return a TestingKnobs structure with overrides // for adopt and cancel intervals. -func NewTestingKnobsWithIntervals(adopt, cancel time.Duration) *TestingKnobs { +func NewTestingKnobsWithIntervals( + adopt, cancel, initialDelay, maxDelay time.Duration, +) *TestingKnobs { return &TestingKnobs{ IntervalOverrides: TestingIntervalOverrides{ - Adopt: &adopt, - Cancel: &cancel, + Adopt: &adopt, + Cancel: &cancel, + RetryInitialDelay: &initialDelay, + RetryMaxDelay: &maxDelay, }, } } diff --git a/pkg/jobs/update.go b/pkg/jobs/update.go index cb2c24d0be94..a83058499aa9 100644 --- a/pkg/jobs/update.go +++ b/pkg/jobs/update.go @@ -15,7 +15,9 @@ import ( "context" "fmt" "strings" + "time" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security" @@ -36,12 +38,19 @@ import ( // changes will be ignored unless JobUpdater is used). type UpdateFn func(txn *kv.Txn, md JobMetadata, ju *JobUpdater) error +// RunStats consists of job-run statistics: num of runs and last-run timestamp. +type RunStats struct { + LastRun time.Time + NumRuns int +} + // JobMetadata groups the job metadata values passed to UpdateFn. type JobMetadata struct { ID jobspb.JobID Status Status Payload *jobspb.Payload Progress *jobspb.Progress + RunStats *RunStats } // CheckRunningOrReverting returns an InvalidStatusError if md.Status is not @@ -80,6 +89,15 @@ func (ju *JobUpdater) hasUpdates() bool { return ju.md != JobMetadata{} } +// UpdateRunStats is used to update the exponential-backoff parameters last_run and +// num_runs in system.jobs table. +func (ju *JobUpdater) UpdateRunStats(numRuns int, lastRun time.Time) { + ju.md.RunStats = &RunStats{ + NumRuns: numRuns, + LastRun: lastRun, + } +} + // UpdateHighwaterProgressed updates job updater progress with the new high water mark. func UpdateHighwaterProgressed(highWater hlc.Timestamp, md JobMetadata, ju *JobUpdater) error { if err := md.CheckRunningOrReverting(); err != nil { @@ -124,13 +142,14 @@ func (j *Job) update(ctx context.Context, txn *kv.Txn, useReadLock bool, updateF var payload *jobspb.Payload var progress *jobspb.Progress + backoffIsActive := j.registry.settings.Version.IsActive(ctx, clusterversion.RetryJobsWithExponentialBackoff) if err := j.runInTxn(ctx, txn, func(ctx context.Context, txn *kv.Txn) error { var err error var row tree.Datums row, err = j.registry.ex.QueryRowEx( ctx, "log-job", txn, sessiondata.InternalExecutorOverride{User: security.RootUserName()}, - getSelectStmtForJobUpdate(j.sessionID != "", useReadLock), j.ID(), + getSelectStmtForJobUpdate(j.sessionID != "", useReadLock, backoffIsActive), j.ID(), ) if err != nil { return err @@ -144,6 +163,7 @@ func (j *Job) update(ctx context.Context, txn *kv.Txn, useReadLock bool, updateF return errors.AssertionFailedf("job %d: expected string status, but got %T", j.ID(), statusString) } + status := Status(*statusString) if j.sessionID != "" { if row[3] == tree.DNull { return errors.Errorf( @@ -157,8 +177,6 @@ func (j *Job) update(ctx context.Context, txn *kv.Txn, useReadLock bool, updateF j.ID(), statusString, j.sessionID, storedSession) } } - - status := Status(*statusString) if payload, err = UnmarshalPayload(row[1]); err != nil { return err } @@ -172,6 +190,28 @@ func (j *Job) update(ctx context.Context, txn *kv.Txn, useReadLock bool, updateF Payload: payload, Progress: progress, } + + if backoffIsActive { + offset := 0 + if j.sessionID != "" { + offset = 1 + } + var lastRun *tree.DTimestamp + lastRun, ok = row[3+offset].(*tree.DTimestamp) + if !ok { + return errors.AssertionFailedf("job %d: expected timestamp last_run, but got %T", j.ID(), lastRun) + } + var numRuns *tree.DInt + numRuns, ok = row[4+offset].(*tree.DInt) + if !ok { + return errors.AssertionFailedf("job %d: expected int num_runs, but got %T", j.ID(), numRuns) + } + md.RunStats = &RunStats{ + NumRuns: int(*numRuns), + LastRun: lastRun.Time, + } + } + var ju JobUpdater if err := updateFn(txn, md, &ju); err != nil { return err @@ -181,6 +221,7 @@ func (j *Job) update(ctx context.Context, txn *kv.Txn, useReadLock bool, updateF return err } } + if !ju.hasUpdates() { return nil } @@ -226,6 +267,11 @@ func (j *Job) update(ctx context.Context, txn *kv.Txn, useReadLock bool, updateF addSetter("progress", progressBytes) } + if backoffIsActive && ju.md.RunStats != nil { + addSetter("last_run", ju.md.RunStats.LastRun) + addSetter("num_runs", ju.md.RunStats.NumRuns) + } + updateStmt := fmt.Sprintf( "UPDATE system.jobs SET %s WHERE id = $1", strings.Join(setters, ", "), @@ -257,21 +303,23 @@ func (j *Job) update(ctx context.Context, txn *kv.Txn, useReadLock bool, updateF } // getSelectStmtForJobUpdate constructs the select statement used in Job.update. -func getSelectStmtForJobUpdate(hasSessionID, useReadLock bool) string { +func getSelectStmtForJobUpdate(hasSessionID, useReadLock, backoffIsActive bool) string { const ( selectWithoutSession = `SELECT status, payload, progress` selectWithSession = selectWithoutSession + `, claim_session_id` from = ` FROM system.jobs WHERE id = $1` fromForUpdate = from + ` FOR UPDATE` + backoffColumns = ", COALESCE(last_run, created), COALESCE(num_runs, 0)" ) + stmt := selectWithoutSession if hasSessionID { - if useReadLock { - return selectWithSession + fromForUpdate - } - return selectWithSession + from + stmt = selectWithSession + } + if backoffIsActive { + stmt = stmt + backoffColumns } if useReadLock { - return selectWithoutSession + fromForUpdate + return stmt + fromForUpdate } - return selectWithoutSession + from + return stmt + from } diff --git a/pkg/migration/BUILD.bazel b/pkg/migration/BUILD.bazel index 483b7b03c8fd..0581e804b3d0 100644 --- a/pkg/migration/BUILD.bazel +++ b/pkg/migration/BUILD.bazel @@ -7,11 +7,14 @@ go_library( "migration.go", "system_migration.go", "tenant_migration.go", + "testing_knobs.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/migration", visibility = ["//visibility:public"], deps = [ + "//pkg/base", "//pkg/clusterversion", + "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/kvcoord:with-mocks", diff --git a/pkg/migration/migrationjob/migration_job.go b/pkg/migration/migrationjob/migration_job.go index 6cb74a4c45bf..5ea1fc1d9c4f 100644 --- a/pkg/migration/migrationjob/migration_job.go +++ b/pkg/migration/migrationjob/migration_job.go @@ -56,7 +56,6 @@ type resumer struct { var _ jobs.Resumer = (*resumer)(nil) func (r resumer) Resume(ctx context.Context, execCtxI interface{}) error { - execCtx := execCtxI.(sql.JobExecContext) pl := r.j.Payload() cv := *pl.GetMigration().ClusterVersion @@ -88,6 +87,7 @@ func (r resumer) Resume(ctx context.Context, execCtxI interface{}) error { CollectionFactory: execCtx.ExecCfg().CollectionFactory, LeaseManager: execCtx.ExecCfg().LeaseManager, InternalExecutor: execCtx.ExecCfg().InternalExecutor, + TestingKnobs: execCtx.ExecCfg().MigrationTestingKnobs, }) default: return errors.AssertionFailedf("unknown migration type %T", m) diff --git a/pkg/migration/migrationmanager/BUILD.bazel b/pkg/migration/migrationmanager/BUILD.bazel index c73df824c655..3d6d768d0566 100644 --- a/pkg/migration/migrationmanager/BUILD.bazel +++ b/pkg/migration/migrationmanager/BUILD.bazel @@ -2,14 +2,10 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "migrationmanager", - srcs = [ - "manager.go", - "testing_knobs.go", - ], + srcs = ["manager.go"], importpath = "github.com/cockroachdb/cockroach/pkg/migration/migrationmanager", visibility = ["//visibility:public"], deps = [ - "//pkg/base", "//pkg/clusterversion", "//pkg/jobs", "//pkg/jobs/jobspb", @@ -40,7 +36,6 @@ go_test( "manager_external_test.go", ], deps = [ - ":migrationmanager", "//pkg/base", "//pkg/clusterversion", "//pkg/jobs", diff --git a/pkg/migration/migrationmanager/manager.go b/pkg/migration/migrationmanager/manager.go index b826dc8c19b8..abdf7c7f4c4d 100644 --- a/pkg/migration/migrationmanager/manager.go +++ b/pkg/migration/migrationmanager/manager.go @@ -46,7 +46,7 @@ type Manager struct { jr *jobs.Registry codec keys.SQLCodec settings *cluster.Settings - knobs TestingKnobs + knobs migration.TestingKnobs } // GetMigration returns the migration associated with this key. @@ -74,9 +74,9 @@ func NewManager( jr *jobs.Registry, codec keys.SQLCodec, settings *cluster.Settings, - testingKnobs *TestingKnobs, + testingKnobs *migration.TestingKnobs, ) *Manager { - var knobs TestingKnobs + var knobs migration.TestingKnobs if testingKnobs != nil { knobs = *testingKnobs } diff --git a/pkg/migration/migrationmanager/manager_external_test.go b/pkg/migration/migrationmanager/manager_external_test.go index 97e317cad78b..48f6a73d999b 100644 --- a/pkg/migration/migrationmanager/manager_external_test.go +++ b/pkg/migration/migrationmanager/manager_external_test.go @@ -25,7 +25,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/migration" - "github.com/cockroachdb/cockroach/pkg/migration/migrationmanager" "github.com/cockroachdb/cockroach/pkg/migration/migrations" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" @@ -70,7 +69,7 @@ func TestAlreadyRunningJobsAreHandledProperly(t *testing.T) { BinaryVersionOverride: startCV.Version, DisableAutomaticVersionUpgrade: 1, }, - MigrationManager: &migrationmanager.TestingKnobs{ + MigrationManager: &migration.TestingKnobs{ ListBetweenOverride: func(from, to clusterversion.ClusterVersion) []clusterversion.ClusterVersion { return []clusterversion.ClusterVersion{to} }, @@ -204,7 +203,7 @@ func TestMigrateUpdatesReplicaVersion(t *testing.T) { BinaryVersionOverride: startCV.Version, DisableAutomaticVersionUpgrade: 1, }, - MigrationManager: &migrationmanager.TestingKnobs{ + MigrationManager: &migration.TestingKnobs{ ListBetweenOverride: func(from, to clusterversion.ClusterVersion) []clusterversion.ClusterVersion { return []clusterversion.ClusterVersion{from, to} }, @@ -322,7 +321,7 @@ func TestConcurrentMigrationAttempts(t *testing.T) { BinaryVersionOverride: versions[0].Version, DisableAutomaticVersionUpgrade: 1, }, - MigrationManager: &migrationmanager.TestingKnobs{ + MigrationManager: &migration.TestingKnobs{ ListBetweenOverride: func(from, to clusterversion.ClusterVersion) []clusterversion.ClusterVersion { return versions }, @@ -404,7 +403,7 @@ func TestPauseMigration(t *testing.T) { BinaryVersionOverride: startCV.Version, DisableAutomaticVersionUpgrade: 1, }, - MigrationManager: &migrationmanager.TestingKnobs{ + MigrationManager: &migration.TestingKnobs{ ListBetweenOverride: func(from, to clusterversion.ClusterVersion) []clusterversion.ClusterVersion { return []clusterversion.ClusterVersion{to} }, @@ -517,7 +516,7 @@ func TestPrecondition(t *testing.T) { }, // Inject a migration which would run to upgrade the cluster. // We'll validate that we never create a job for this migration. - MigrationManager: &migrationmanager.TestingKnobs{ + MigrationManager: &migration.TestingKnobs{ ListBetweenOverride: func(from, to clusterversion.ClusterVersion) []clusterversion.ClusterVersion { start := sort.Search(len(versions), func(i int) bool { return from.Less(versions[i].Version) }) end := sort.Search(len(versions), func(i int) bool { return to.Less(versions[i].Version) }) diff --git a/pkg/migration/migrations/BUILD.bazel b/pkg/migration/migrations/BUILD.bazel index 5370981663f8..65c82d44e539 100644 --- a/pkg/migration/migrations/BUILD.bazel +++ b/pkg/migration/migrations/BUILD.bazel @@ -8,6 +8,7 @@ go_library( "fix_descriptor_migration.go", "join_tokens.go", "migrations.go", + "retry_jobs_with_exponential_backoff.go", "separated_intents.go", "sql_instances.go", "sql_stats.go", @@ -18,19 +19,23 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/clusterversion", + "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/kvcoord:with-mocks", "//pkg/kv/kvserver/intentresolver", "//pkg/migration", "//pkg/roachpb:with-mocks", + "//pkg/security", "//pkg/server/serverpb", + "//pkg/sql/catalog", "//pkg/sql/catalog/catalogkeys", "//pkg/sql/catalog/catalogkv", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", "//pkg/sql/catalog/systemschema", "//pkg/sql/sem/tree", + "//pkg/sql/sessiondata", "//pkg/sql/sqlutil", "//pkg/startupmigrations", "//pkg/storage/enginepb", @@ -42,6 +47,7 @@ go_library( "//pkg/util/syncutil", "//pkg/util/uuid", "@com_github_cockroachdb_errors//:errors", + "@com_github_kr_pretty//:pretty", ], ) @@ -50,7 +56,9 @@ go_test( srcs = [ "delete_deprecated_namespace_tabledesc_external_test.go", "fix_descriptor_migration_external_test.go", + "helpers_test.go", "main_test.go", + "retry_jobs_with_exponential_backoff_external_test.go", "separated_intents_external_test.go", "separated_intents_test.go", "truncated_state_external_test.go", @@ -60,21 +68,30 @@ go_test( deps = [ "//pkg/base", "//pkg/clusterversion", + "//pkg/jobs", + "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/kvcoord:with-mocks", "//pkg/kv/kvserver", "//pkg/kv/kvserver/intentresolver", "//pkg/kv/kvserver/stateloader", + "//pkg/migration", "//pkg/roachpb:with-mocks", "//pkg/security", "//pkg/security/securitytest", "//pkg/server", "//pkg/settings/cluster", + "//pkg/sql/catalog", "//pkg/sql/catalog/catalogkeys", "//pkg/sql/catalog/catalogkv", "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/descs", "//pkg/sql/catalog/systemschema", + "//pkg/sql/catalog/tabledesc", + "//pkg/sql/sem/tree", + "//pkg/sql/sqlutil", + "//pkg/sql/types", "//pkg/storage", "//pkg/storage/enginepb", "//pkg/testutils", @@ -93,6 +110,7 @@ go_test( "//pkg/util/uuid", "@com_github_cockroachdb_datadriven//:datadriven", "@com_github_cockroachdb_errors//:errors", + "@com_github_stretchr_testify//assert", "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/migration/migrations/helpers_test.go b/pkg/migration/migrations/helpers_test.go new file mode 100644 index 000000000000..7ee1dd00376d --- /dev/null +++ b/pkg/migration/migrations/helpers_test.go @@ -0,0 +1,24 @@ +// Copyright 2021 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 migrations + +import "github.com/cockroachdb/cockroach/pkg/sql/catalog" + +func HasBackoffCols(storedTable, expectedTable catalog.TableDescriptor, col string) (bool, error) { + return hasColumn(storedTable, expectedTable, col) +} + +func HasBackoffIndex(jobsTable, expectedTable catalog.TableDescriptor, index string) (bool, error) { + return hasIndex(jobsTable, expectedTable, index) +} + +const AddColsQuery = addColsQuery +const AddIndexQuery = addIndexQuery diff --git a/pkg/migration/migrations/migrations.go b/pkg/migration/migrations/migrations.go index 12b8a81626a2..dcf014e3ab1c 100644 --- a/pkg/migration/migrations/migrations.go +++ b/pkg/migration/migrations/migrations.go @@ -105,6 +105,11 @@ var migrations = []migration.Migration{ "run no-op migrate command on all ranges after lock table migration", toCV(clusterversion.PostSeparatedIntentsMigration), postSeparatedIntentsMigration), + migration.NewTenantMigration( + "add last_run and num_runs columns to system.jobs", + toCV(clusterversion.RetryJobsWithExponentialBackoff), + NoPrecondition, + retryJobsWithExponentialBackoff), } func init() { diff --git a/pkg/migration/migrations/retry_jobs_with_exponential_backoff.go b/pkg/migration/migrations/retry_jobs_with_exponential_backoff.go new file mode 100644 index 000000000000..a64eefeca3cb --- /dev/null +++ b/pkg/migration/migrations/retry_jobs_with_exponential_backoff.go @@ -0,0 +1,286 @@ +// Copyright 2021 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 migrations + +import ( + "bytes" + "context" + "fmt" + "strings" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/migration" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/errors" + "github.com/kr/pretty" +) + +// Target schema changes in the system.jobs table, adding two columns and an +// index for job retries with exponential backoff functionality. +const ( + addColsQuery = ` +ALTER TABLE system.jobs + ADD COLUMN num_runs INT8 FAMILY claim, + ADD COLUMN last_run TIMESTAMP FAMILY claim` + addIndexQuery = ` +CREATE INDEX jobs_run_stats_idx + ON system.jobs (claim_session_id, status, created) + STORING (last_run, num_runs, claim_instance_id) + WHERE ` + systemschema.JobsRunStatsIdxPredicate +) + +// retryJobsWithExponentialBackoff changes the schema of system.jobs table in +// two steps. It first adds two new columns and then an index. +func retryJobsWithExponentialBackoff( + ctx context.Context, cs clusterversion.ClusterVersion, d migration.TenantDeps, +) error { + jobsTable := systemschema.JobsTable + ops := [...]struct { + // Operation name. + name string + // List of schema names, e.g., column names, which are modified in the query. + schemaList []string + // Schema change query. + query string + // Function to check existing schema. + schemaExistsFn func(catalog.TableDescriptor, catalog.TableDescriptor, string) (bool, error) + }{ + {"jobs-add-columns", []string{"num_runs", "last_run"}, addColsQuery, hasColumn}, + {"jobs-add-index", []string{"jobs_run_stats_idx"}, addIndexQuery, hasIndex}, + } + for _, op := range ops { + if err := migrateTable(ctx, cs, d, op.name, keys.JobsTableID, op.query, + func(storedTable catalog.TableDescriptor) (bool, error) { + // Expect all or none. + var exists bool + for i, schemaName := range op.schemaList { + hasSchema, err := op.schemaExistsFn(storedTable, jobsTable, schemaName) + if err != nil { + return false, err + } + if i > 0 && exists != hasSchema { + return false, errors.Errorf("observed partial schema exists while performing %v", op.name) + } + exists = hasSchema + } + return exists, nil + }); err != nil { + return err + } + } + return nil +} + +// migrateTable is run during a migration to a new version and changes an existing +// table's schema based on schemaChangeQuery. The schema-change is ignored if the +// table already has the required changes. +// +// This function reads the existing table descriptor from storage and passes it +// to schemaExists function to verify whether the schema-change already exists or +// not. If the change is already done, the function does not perform the change +// again, which makes migrateTable idempotent. +// +// schemaExists function should be customized based on the table being modified, +// ignoring the fields in the descriptor that do not +// +// If multiple changes are done in the same query, e.g., if multiple columns are +// added, the function should check all changes to exist or absent, returning +// an error if changes exist partially. +func migrateTable( + ctx context.Context, + _ clusterversion.ClusterVersion, + d migration.TenantDeps, + opTag string, + tableID descpb.ID, + schemaChangeQuery string, + schemaExists func(descriptor catalog.TableDescriptor) (bool, error), +) error { + for { + // - Fetch the table, reading its descriptor from storage. + // - Check if any mutation jobs exist for the table. These mutations can + // belong to a previous migration attempt that failed. + // - If any mutation job exists: + // - Wait for the ongoing mutations to complete. + // - Continue to the beginning of the loop to cater for the mutations + // that may have started while waiting for existing mutations to complete. + // - Check if the intended schema-changes already exist. + // - If the changes already exist, skip the schema-change and return as + // the changes are already done in a previous migration attempt. + // - Otherwise, perform the schema-change and return. + + log.Infof(ctx, "performing table migration operation %v", opTag) + // Retrieve the table. + jt, err := readTableDescriptor(ctx, d, tableID) + if err != nil { + return err + } + // Wait for any in-flight schema changes to complete. + if mutations := jt.GetMutationJobs(); len(mutations) > 0 { + for _, mutation := range mutations { + log.Infof(ctx, "waiting for the mutation job %v to complete", mutation.JobID) + if d.TestingKnobs.BeforeWaitInRetryJobsWithExponentialBackoffMigration != nil { + d.TestingKnobs.BeforeWaitInRetryJobsWithExponentialBackoffMigration(jobspb.JobID(mutation.JobID)) + } + if _, err := d.InternalExecutor.Exec(ctx, "migration-mutations-wait", + nil, "SHOW JOB WHEN COMPLETE $1", mutation.JobID); err != nil { + return err + } + } + continue + } + + // Ignore the schema change if the table already has the required schema. + if ok, err := schemaExists(jt); err != nil { + return errors.Wrapf(err, "error while validating descriptors during"+ + " operation %s", opTag) + } else if ok { + log.Infof(ctx, "skipping %s operation as the schema change already exists.", opTag) + if d.TestingKnobs != nil && d.TestingKnobs.SkippedMutation != nil { + d.TestingKnobs.SkippedMutation() + } + return nil + } + + // Modify the table. + log.Infof(ctx, "performing operation: %s", opTag) + if _, err := d.InternalExecutor.ExecEx( + ctx, + fmt.Sprintf("migration-alter-table-%d", tableID), + nil, /* txn */ + sessiondata.InternalExecutorOverride{User: security.NodeUserName()}, + schemaChangeQuery); err != nil { + return err + } + return nil + } +} + +func readTableDescriptor( + ctx context.Context, d migration.TenantDeps, tableID descpb.ID, +) (catalog.TableDescriptor, error) { + var jt catalog.TableDescriptor + + if err := d.CollectionFactory.Txn(ctx, d.InternalExecutor, d.DB, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, + ) (err error) { + jt, err = descriptors.GetImmutableTableByID(ctx, txn, tableID, tree.ObjectLookupFlags{ + CommonLookupFlags: tree.CommonLookupFlags{ + AvoidCached: true, + Required: true, + }, + }) + return err + }); err != nil { + return nil, err + } + return jt, nil +} + +// hasColumn returns true if storedTable already has the given column, comparing +// with expectedTable. +// storedTable descriptor must be read from system storage as compared to reading +// from the systemschema package. On the contrary, expectedTable must be accessed +// directly from systemschema package. +// This function returns an error if the column exists but doesn't match with the +// expectedTable descriptor. The comparison is not strict as several descriptor +// fields are ignored. +func hasColumn(storedTable, expectedTable catalog.TableDescriptor, colName string) (bool, error) { + storedCol, err := storedTable.FindColumnWithName(tree.Name(colName)) + if err != nil { + if strings.Contains(err.Error(), "does not exist") { + return false, nil + } + return false, err + } + + expectedCol, err := expectedTable.FindColumnWithName(tree.Name(colName)) + if err != nil { + return false, errors.Wrapf(err, "columns name %s is invalid.", colName) + } + + expectedCopy := expectedCol.ColumnDescDeepCopy() + storedCopy := storedCol.ColumnDescDeepCopy() + + storedCopy.ID = 0 + expectedCopy.ID = 0 + + if err = ensureProtoMessagesAreEqual(&expectedCopy, &storedCopy); err != nil { + return false, err + } + return true, nil +} + +// hasIndex returns true if storedTable already has the given index, comparing +// with expectedTable. +// storedTable descriptor must be read from system storage as compared to reading +// from the systemschema package. On the contrary, expectedTable must be accessed +// directly from systemschema package. +// This function returns an error if the index exists but doesn't match with the +// expectedTable descriptor. The comparison is not strict as several descriptor +// fields are ignored. +func hasIndex(storedTable, expectedTable catalog.TableDescriptor, indexName string) (bool, error) { + storedIdx, err := storedTable.FindIndexWithName(indexName) + if err != nil { + if strings.Contains(err.Error(), "does not exist") { + return false, nil + } + return false, err + } + expectedIdx, err := expectedTable.FindIndexWithName(indexName) + if err != nil { + return false, errors.Wrapf(err, "index name %s is invalid", indexName) + } + storedCopy := storedIdx.IndexDescDeepCopy() + expectedCopy := expectedIdx.IndexDescDeepCopy() + // Ignore the fields that don't matter in the comparison. + storedCopy.ID = 0 + expectedCopy.ID = 0 + storedCopy.Version = 0 + expectedCopy.Version = 0 + storedCopy.CreatedExplicitly = false + expectedCopy.CreatedExplicitly = false + storedCopy.StoreColumnIDs = []descpb.ColumnID{0, 0, 0} + expectedCopy.StoreColumnIDs = []descpb.ColumnID{0, 0, 0} + + if err = ensureProtoMessagesAreEqual(&expectedCopy, &storedCopy); err != nil { + return false, err + } + return true, nil +} + +// ensureProtoMessagesAreEqual verifies whether the given protobufs are equal or +// not, returning an error if they are not equal. +func ensureProtoMessagesAreEqual(expected, found protoutil.Message) error { + expectedBytes, err := protoutil.Marshal(expected) + if err != nil { + return err + } + foundBytes, err := protoutil.Marshal(found) + if err != nil { + return err + } + if bytes.Equal(expectedBytes, foundBytes) { + return nil + } + return errors.Errorf("expected descriptor doesn't match "+ + "with found descriptor: %s", strings.Join(pretty.Diff(expected, found), "\n")) +} diff --git a/pkg/migration/migrations/retry_jobs_with_exponential_backoff_external_test.go b/pkg/migration/migrations/retry_jobs_with_exponential_backoff_external_test.go new file mode 100644 index 000000000000..e7db24a1eab2 --- /dev/null +++ b/pkg/migration/migrations/retry_jobs_with_exponential_backoff_external_test.go @@ -0,0 +1,671 @@ +// Copyright 2021 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 migrations_test + +import ( + "context" + gosql "database/sql" + "regexp" + "strings" + "sync/atomic" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/migration" + "github.com/cockroachdb/cockroach/pkg/migration/migrations" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +// TestExponentialBackoffMigration tests modification of system.jobs table +// during migration. It does not test the migration success during failures. +func TestExponentialBackoffMigration(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + clusterArgs := base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: 1, + BinaryVersionOverride: clusterversion.ByKey( + clusterversion.RetryJobsWithExponentialBackoff - 1), + }, + JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), + }, + }, + } + + ctx := context.Background() + tc := testcluster.StartTestCluster(t, 1, clusterArgs) + defer tc.Stopper().Stop(ctx) + s := tc.Server(0) + sqlDB := tc.ServerConn(0) + tdb := sqlutils.MakeSQLRunner(sqlDB) + + // Inject the old copy of the descriptor. + injectLegacyTable(t, ctx, s) + // Validate that the jobs table has old schema. + validateSchemaExists(t, ctx, s, sqlDB, false) + // Run the migration. + migrate(t, sqlDB, false, nil) + // Validate that the jobs table has new schema. + validateSchemaExists(t, ctx, s, sqlDB, true) + // Make sure that jobs work by running a job. + runGcJob(t, tdb) +} + +type updateEvent struct { + orig, updated jobs.JobMetadata + errChan chan error +} + +// TestMigrationWithFailures tests modification of system.jobs table during +// migration with different failures. It tests the system behavior with failure +// combinations of the migration job and schema-change jobs at different stages +// in their progress. +func TestMigrationWithFailures(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + // The tests follows the following procedure. + // + // Inject the old table descriptor and ensure that the system is using the + // deprecated jobs-table. + // + // Start migration, which initiates two schema-change jobs one by one. Test + // the system for each schema-change job separately. Later on, we inject + // failure in this migration, causing it to fail. + // + // Depending on the test setting, intercept the target schema-change job, + // preventing the job from progressing. We may cancel this schema-change or + // let it succeed to test different scenarios. + // + // Cancel the migration, causing the migration to revert and fail. + // + // Wait for the canceled migration-job to finish, expecting its failure. The + // schema-change job is still not progressing to control what the restarted + // migration will observe. + // + // Restart the migration, expecting it to succeed. Depending on the test setting, + // the intercepted schema-change job may wail for the migration job to resume. + // If it does, the migration job is expected to observe the ongoing schema-change. + // The ongoing schema-change is canceled or not, depending on the test case. + // In either case, we expect the correct number of mutations to be skipped + // during the migration. + // + // If we canceled the schema-job, expect it to rerun + // as part of the migration. Otherwise, expect the schema-change to be ignored + // during the migration. + // + // Finally, we validate that the schema changes are in effect by reading the new + // columns and the index, and by running a job that is failed and retried to + // practice exponential-backoff machinery. + + for _, test := range []struct { + // Test identifier. + name string + // Job status when the job is intercepted while transitioning to the intercepted status. + query string + // Whether the schema-change job should wait for the migration to restart + // after failure before proceeding. + waitForMigrationRestart bool + // Cancel the intercepted schema-change to inject a failure during migration. + cancelSchemaJob bool + // Expected number of schema-changes that are skipped during migration. + expectedSkipped int + }{ + { + name: "adding columns", + query: migrations.AddColsQuery, + waitForMigrationRestart: false, // Does not matter. + cancelSchemaJob: false, // Does not matter. + expectedSkipped: 0, // Will be ignored. + }, + { + name: "adding index", + query: migrations.AddIndexQuery, + waitForMigrationRestart: false, // Does not matter. + cancelSchemaJob: false, // Does not matter. + expectedSkipped: 0, // Will be ignored. + }, + { + name: "fail adding columns", + query: migrations.AddColsQuery, + waitForMigrationRestart: true, // Need to wait to observe failing schema change. + cancelSchemaJob: true, // To fail adding columns. + expectedSkipped: 0, + }, + { + name: "fail adding index", + query: migrations.AddIndexQuery, + waitForMigrationRestart: true, // Need to wait to observe failing schema change. + cancelSchemaJob: true, // To fail adding index. + expectedSkipped: 1, // Columns must not be added again. + }, + { + name: "skip none", + query: migrations.AddColsQuery, + waitForMigrationRestart: true, // Need to wait to observe schema change and have correct expectedSkipped count. + cancelSchemaJob: true, // To fail adding index and skip adding column. + expectedSkipped: 0, // Both columns and index must be added. + }, + { + name: "skip adding columns", + query: migrations.AddIndexQuery, + waitForMigrationRestart: true, // Need to wait to observe schema change and have correct expectedSkipped count. + cancelSchemaJob: true, // To fail adding index and skip adding column. + expectedSkipped: 1, // Columns must not be added again. + }, + { + name: "skip adding columns and index", + query: migrations.AddIndexQuery, + waitForMigrationRestart: true, // Need to wait to observe schema change and have correct expectedSkipped count. + cancelSchemaJob: false, // To fail adding index and skip adding column. + expectedSkipped: 2, // Both columns and index must not be added again. + }, + } { + t.Run(test.name, func(t *testing.T) { + ctx := context.Background() + // To intercept the schema-change and the migration job. + updateEventChan := make(chan updateEvent) + beforeUpdate := func(orig, updated jobs.JobMetadata) error { + ue := updateEvent{ + orig: orig, + updated: updated, + errChan: make(chan error), + } + updateEventChan <- ue + return <-ue.errChan + } + + var schemaEvent updateEvent + migrationWaitCh := make(chan struct{}) + beforeMutationWait := func(jobID jobspb.JobID) { + if !test.waitForMigrationRestart || jobID != schemaEvent.orig.ID { + return + } + migrationWaitCh <- struct{}{} + } + + // Number of schema-change jobs that are skipped. + skippedCnt := int32(0) + ignoredMutationObserver := func() { + atomic.AddInt32(&skippedCnt, 1) + } + + shortInterval := 2 * time.Millisecond + clusterArgs := base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: 1, + BinaryVersionOverride: clusterversion.ByKey( + clusterversion.RetryJobsWithExponentialBackoff - 1), + }, + JobsTestingKnobs: &jobs.TestingKnobs{ + IntervalOverrides: jobs.TestingIntervalOverrides{ + Adopt: &shortInterval, + Cancel: &shortInterval, + }, + BeforeUpdate: beforeUpdate, + }, + MigrationManager: &migration.TestingKnobs{ + BeforeWaitInRetryJobsWithExponentialBackoffMigration: beforeMutationWait, + SkippedMutation: ignoredMutationObserver, + }, + }, + }, + } + tc := testcluster.StartTestCluster(t, 1, clusterArgs) + defer tc.Stopper().Stop(ctx) + s := tc.Server(0) + sqlDB := tc.ServerConn(0) + tdb := sqlutils.MakeSQLRunner(sqlDB) + + tdb.Exec(t, "SET CLUSTER SETTING jobs.registry.interval.gc = '2ms'") + // Inject the old copy of the descriptor. + injectLegacyTable(t, ctx, s) + // Validate that the jobs-table has old schema. + validateSchemaExists(t, ctx, s, sqlDB, false) + // Run the migration, expecting failure. + t.Log("trying migration, expecting to fail") + // Channel to wait for the migration job to complete. + finishChan := make(chan struct{}) + go migrate(t, sqlDB, true, finishChan) + + var migJobID jobspb.JobID + // Intercept the target schema-change job and get migration-job's ID. + t.Log("intercepting the schema job") + for { + e := <-updateEventChan + // The migration job creates schema-change jobs. Therefore, we are guaranteed + // to get the migration-job's ID before canceling the job later on. + if e.orig.Payload.Type() == jobspb.TypeMigration { + migJobID = e.orig.ID + e.errChan <- nil + continue + } + schemaQuery := strings.Replace(e.orig.Payload.Description, "system.public.jobs", "system.jobs", -1) + testQuery := removeSpaces(test.query) + testQuery = strings.ReplaceAll(testQuery, ":::STRING", "") + if testQuery == schemaQuery { + // Intercepted the target schema-change. + schemaEvent = e + t.Logf("intercepted schema change job: %v", e.orig.ID) + break + } + // Ignore all other job updates. + e.errChan <- nil + } + // Cancel the migration job. + t.Log("canceling the migration job") + go cancelJob(t, ctx, s, migJobID) + + // Wait for the migration job to finish while preventing the intercepted + // schema-change job from progressing. + t.Log("waiting for the migration job to finish.") + testutils.SucceedsSoon(t, func() error { + for { + select { + case <-finishChan: + return nil + case e := <-updateEventChan: + e.errChan <- nil + default: + return errors.Errorf("waiting for the migration job to finish.") + } + } + }) + + // Channel to finish the goroutine when the test completes. + done := make(chan struct{}) + // Let all jobs to continue until test's completion, except the intercepted + // schema-change job that we resume later on. + go func() { + for { + select { + case e := <-updateEventChan: + e.errChan <- nil + case <-done: + return + } + } + }() + + // Restart the migration job. + t.Log("retrying migration, expecting to succeed") + go migrate(t, sqlDB, false, finishChan) + + // Wait until the new migration job observes an existing mutation job. + if test.waitForMigrationRestart { + t.Log("waiting for the migration job to observe a mutation") + <-migrationWaitCh + } + + t.Log("resuming the schema change job") + // If configured so, mark the schema-change job to cancel. + if test.cancelSchemaJob { + cancelJob(t, ctx, s, schemaEvent.orig.ID) + } + // Resume the schema-change job and all other jobs. + schemaEvent.errChan <- nil + + // If canceled the job, wait for the job to finish. + if test.cancelSchemaJob { + t.Log("waiting for the schema job to reach the cancel status") + waitUntilState(t, tdb, schemaEvent.orig.ID, jobs.StatusCanceled) + } + + // Wait for the migration to complete, expecting success. + t.Logf("waiting for the new migration job to complete.") + testutils.SucceedsSoon(t, func() error { + select { + case <-finishChan: + return nil + default: + } + return errors.Errorf("waiting for the migration job to finish.") + }) + if test.waitForMigrationRestart { + // Ensure that we have observed the expected number of ignored schema change jobs. + require.Equal(t, int32(test.expectedSkipped), atomic.LoadInt32(&skippedCnt)) + } + + // Validate that the jobs table has new schema. + validateSchemaExists(t, ctx, s, sqlDB, true) + done <- struct{}{} + validateJobRetries(t, tdb, updateEventChan) + }) + } +} + +// cancelJob marks the given job as cancel-requested, leading the job to be +// canceled. +func cancelJob( + t *testing.T, ctx context.Context, s serverutils.TestServerInterface, jobID jobspb.JobID, +) { + err := s.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + // Using this way of canceling because the migration job us non-cancelable. + // Canceling in this way skips the check. + return s.JobRegistry().(*jobs.Registry).UpdateJobWithTxn( + ctx, jobID, txn, false /* useReadLock */, func(txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater, + ) error { + ju.UpdateStatus(jobs.StatusCancelRequested) + return nil + }) + }) + assert.NoError(t, err) +} + +// waitUntilState waits until the specified job reaches to given state. +func waitUntilState( + t *testing.T, tdb *sqlutils.SQLRunner, jobID jobspb.JobID, expectedStatus jobs.Status, +) { + testutils.SucceedsSoon(t, func() error { + var status jobs.Status + tdb.QueryRow(t, + "SELECT status FROM system.jobs WHERE id = $1", jobID, + ).Scan(&status) + if status == expectedStatus { + return nil + } + return errors.Errorf( + "waiting for job %v to reach status %v, current status is %v", + jobID, expectedStatus, status) + }) +} + +// migrate runs cluster migration by changing the 'version' cluster setting. +func migrate(t *testing.T, sqlDB *gosql.DB, expectError bool, done chan struct{}) { + defer func() { + if done != nil { + done <- struct{}{} + } + }() + _, err := sqlDB.Exec(`SET CLUSTER SETTING version = $1`, + clusterversion.ByKey(clusterversion.RetryJobsWithExponentialBackoff).String()) + if expectError { + assert.Error(t, err) + return + } + assert.NoError(t, err) +} + +// injectLegacyTable overwrites the existing table descriptor with the previous +// table descriptor. +func injectLegacyTable(t *testing.T, ctx context.Context, s serverutils.TestServerInterface) { + err := s.CollectionFactory().(*descs.CollectionFactory).Txn( + ctx, + s.InternalExecutor().(sqlutil.InternalExecutor), + s.DB(), + func(ctx context.Context, txn *kv.Txn, descriptors *descs.Collection) error { + id := systemschema.JobsTable.GetID() + tab, err := descriptors.GetMutableTableByID(ctx, txn, id, tree.ObjectLookupFlagsWithRequired()) + if err != nil { + return err + } + builder := tabledesc.NewBuilder(deprecatedDescriptor()) + require.NoError(t, builder.RunPostDeserializationChanges(ctx, nil)) + tab.TableDescriptor = builder.BuildCreatedMutableTable().TableDescriptor + tab.Version = tab.ClusterVersion.Version + 1 + return descriptors.WriteDesc(ctx, false, tab, txn) + }) + require.NoError(t, err) +} + +// validateSchemaExists validates whether the schema-changes of the system.jobs +// table exist or not. +func validateSchemaExists( + t *testing.T, + ctx context.Context, + s serverutils.TestServerInterface, + sqlDB *gosql.DB, + expectExists bool, +) { + // First validate by reading the columns and the index. + for _, stmt := range []string{ + "SELECT last_run, num_runs FROM system.jobs LIMIT 0", + "SELECT num_runs, last_run, claim_instance_id from system.jobs@jobs_run_stats_idx LIMIT 0", + } { + _, err := sqlDB.Exec(stmt) + if expectExists { + require.NoError( + t, err, "expected schema to exist, but unable to query it, using statement: %s", stmt, + ) + } else { + require.Error( + t, err, "expected schema to not exist, but queried it successfully, using statement: %s", stmt, + ) + } + } + + // Manually verify the table descriptor. + storedTable := getJobsTable(t, ctx, s) + jTable := systemschema.JobsTable + str := "not have" + if expectExists { + str = "have" + } + for _, schema := range [...]struct { + name string + validationFn func(catalog.TableDescriptor, catalog.TableDescriptor, string) (bool, error) + }{ + {"num_runs", migrations.HasBackoffCols}, + {"last_run", migrations.HasBackoffCols}, + {"jobs_run_stats_idx", migrations.HasBackoffIndex}, + } { + updated, err := schema.validationFn(storedTable, jTable, schema.name) + require.NoError(t, err) + require.Equal(t, expectExists, updated, + "expected jobs table to %s %s", str, schema) + } +} + +// getJobsTable returns system.jobs table descriptor, reading it from storage. +func getJobsTable( + t *testing.T, ctx context.Context, s serverutils.TestServerInterface, +) catalog.TableDescriptor { + var table catalog.TableDescriptor + // Retrieve the jobs table. + err := s.CollectionFactory().(*descs.CollectionFactory).Txn( + ctx, + s.InternalExecutor().(sqlutil.InternalExecutor), + s.DB(), + func(ctx context.Context, txn *kv.Txn, descriptors *descs.Collection) (err error) { + table, err = descriptors.GetImmutableTableByID(ctx, txn, keys.JobsTableID, tree.ObjectLookupFlags{ + CommonLookupFlags: tree.CommonLookupFlags{ + AvoidCached: true, + Required: true, + }, + }) + return err + }) + require.NoError(t, err) + return table +} + +// runGcJob creates and alters a dummy table to trigger jobs machinery, +// which validates its working. +func runGcJob(t *testing.T, tdb *sqlutils.SQLRunner) { + tdb.Exec(t, "CREATE TABLE foo (i INT PRIMARY KEY)") + tdb.Exec(t, "ALTER TABLE foo CONFIGURE ZONE USING gc.ttlseconds = 1;") + tdb.Exec(t, "DROP TABLE foo CASCADE;") + var jobID int64 + tdb.QueryRow(t, ` +SELECT job_id + FROM [SHOW JOBS] + WHERE job_type = 'SCHEMA CHANGE GC' AND description LIKE '%foo%';`, + ).Scan(&jobID) + var status jobs.Status + tdb.QueryRow(t, + "SELECT status FROM [SHOW JOB WHEN COMPLETE $1]", jobID, + ).Scan(&status) + require.Equal(t, jobs.StatusSucceeded, status) +} + +func validateJobRetries(t *testing.T, tdb *sqlutils.SQLRunner, eventCh chan updateEvent) { + tdb.Exec(t, "SET CLUSTER SETTING jobs.registry.retry.initial_delay = '2ms'") + tdb.Exec(t, "SET CLUSTER SETTING jobs.registry.retry.max_delay = '10ms'") + done := make(chan struct{}) + go func() { + // Fail a GC job once and then let it succeed. + var failed atomic.Value + failed.Store(false) + var ev updateEvent + for { + // eventCh receives events in the BeforeUpdate hook. + select { + case ev = <-eventCh: + case <-done: + return + } + // If not a schema-change GC job, let it run. + if ev.orig.Payload.Type() != jobspb.TypeSchemaChangeGC { + ev.errChan <- nil + continue + } + if ev.updated.Status == jobs.StatusSucceeded { + // If the job is succeeding, it must have been retried once, and the + // the number of retries is populated from the jobs table. + assert.Equal(t, 1, ev.orig.RunStats.NumRuns) + } + if failed.Load().(bool) || + ev.updated.Status != jobs.StatusRunning { + ev.errChan <- nil + continue + } + failed.Store(true) + ev.errChan <- jobs.NewRetryJobError("failing job to retry") + } + }() + runGcJob(t, tdb) + done <- struct{}{} +} + +func removeSpaces(stmt string) string { + stmt = strings.TrimSpace(regexp.MustCompile(`(\s+|;+)`).ReplaceAllString(stmt, " ")) + stmt = strings.ReplaceAll(stmt, "( ", "(") + stmt = strings.ReplaceAll(stmt, " )", ")") + return stmt +} + +// deprecatedDescriptor returns the system.jobs table descriptor that was being used +// before adding two new columns and an index in the current version. +func deprecatedDescriptor() *descpb.TableDescriptor { + uniqueRowIDString := "unique_rowid()" + nowString := "now():::TIMESTAMP" + pk := func(name string) descpb.IndexDescriptor { + return descpb.IndexDescriptor{ + Name: tabledesc.PrimaryKeyIndexName, + ID: 1, + Unique: true, + KeyColumnNames: []string{name}, + KeyColumnDirections: []descpb.IndexDescriptor_Direction{descpb.IndexDescriptor_ASC}, + KeyColumnIDs: []descpb.ColumnID{1}, + } + } + + return &descpb.TableDescriptor{ + Name: "jobs", + ID: keys.JobsTableID, + ParentID: keys.SystemDatabaseID, + UnexposedParentSchemaID: keys.PublicSchemaID, + Version: 1, + Columns: []descpb.ColumnDescriptor{ + {Name: "id", ID: 1, Type: types.Int, DefaultExpr: &uniqueRowIDString}, + {Name: "status", ID: 2, Type: types.String}, + {Name: "created", ID: 3, Type: types.Timestamp, DefaultExpr: &nowString}, + {Name: "payload", ID: 4, Type: types.Bytes}, + {Name: "progress", ID: 5, Type: types.Bytes, Nullable: true}, + {Name: "created_by_type", ID: 6, Type: types.String, Nullable: true}, + {Name: "created_by_id", ID: 7, Type: types.Int, Nullable: true}, + {Name: "claim_session_id", ID: 8, Type: types.Bytes, Nullable: true}, + {Name: "claim_instance_id", ID: 9, Type: types.Int, Nullable: true}, + }, + NextColumnID: 10, + Families: []descpb.ColumnFamilyDescriptor{ + { + Name: "fam_0_id_status_created_payload", + ID: 0, + ColumnNames: []string{"id", "status", "created", "payload", "created_by_type", "created_by_id"}, + ColumnIDs: []descpb.ColumnID{1, 2, 3, 4, 6, 7}, + }, + { + Name: "progress", + ID: 1, + ColumnNames: []string{"progress"}, + ColumnIDs: []descpb.ColumnID{5}, + DefaultColumnID: 5, + }, + { + Name: "claim", + ID: 2, + ColumnNames: []string{"claim_session_id", "claim_instance_id"}, + ColumnIDs: []descpb.ColumnID{8, 9}, + }, + }, + NextFamilyID: 3, + PrimaryIndex: pk("id"), + Indexes: []descpb.IndexDescriptor{ + { + Name: "jobs_status_created_idx", + ID: 2, + Unique: false, + KeyColumnNames: []string{"status", "created"}, + KeyColumnDirections: []descpb.IndexDescriptor_Direction{descpb.IndexDescriptor_ASC, descpb.IndexDescriptor_ASC}, + KeyColumnIDs: []descpb.ColumnID{2, 3}, + KeySuffixColumnIDs: []descpb.ColumnID{1}, + Version: descpb.StrictIndexColumnIDGuaranteesVersion, + }, + { + Name: "jobs_created_by_type_created_by_id_idx", + ID: 3, + Unique: false, + KeyColumnNames: []string{"created_by_type", "created_by_id"}, + KeyColumnDirections: []descpb.IndexDescriptor_Direction{descpb.IndexDescriptor_ASC, descpb.IndexDescriptor_ASC}, + KeyColumnIDs: []descpb.ColumnID{6, 7}, + StoreColumnIDs: []descpb.ColumnID{2}, + StoreColumnNames: []string{"status"}, + KeySuffixColumnIDs: []descpb.ColumnID{1}, + Version: descpb.StrictIndexColumnIDGuaranteesVersion, + }, + }, + NextIndexID: 4, + Privileges: descpb.NewCustomSuperuserPrivilegeDescriptor( + descpb.SystemAllowedPrivileges[keys.JobsTableID], security.NodeUserName()), + FormatVersion: descpb.InterleavedFormatVersion, + NextMutationID: 1, + } +} diff --git a/pkg/migration/tenant_migration.go b/pkg/migration/tenant_migration.go index 89611b21863b..5f4b4e3f43de 100644 --- a/pkg/migration/tenant_migration.go +++ b/pkg/migration/tenant_migration.go @@ -33,6 +33,7 @@ type TenantDeps struct { CollectionFactory *descs.CollectionFactory LeaseManager *lease.Manager InternalExecutor sqlutil.InternalExecutor + TestingKnobs *TestingKnobs } // TenantMigrationFunc is used to perform sql-level migrations. It may be run from diff --git a/pkg/migration/migrationmanager/testing_knobs.go b/pkg/migration/testing_knobs.go similarity index 50% rename from pkg/migration/migrationmanager/testing_knobs.go rename to pkg/migration/testing_knobs.go index 669aec7b1123..bfe6566b01ab 100644 --- a/pkg/migration/migrationmanager/testing_knobs.go +++ b/pkg/migration/testing_knobs.go @@ -8,12 +8,12 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package migrationmanager +package migration import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/migration" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" ) // TestingKnobs are knobs to inject behavior into the migration manager which @@ -26,7 +26,24 @@ type TestingKnobs struct { ListBetweenOverride func(from, to clusterversion.ClusterVersion) []clusterversion.ClusterVersion // RegistryOverride is used to inject migrations for specific cluster versions. - RegistryOverride func(cv clusterversion.ClusterVersion) (migration.Migration, bool) + RegistryOverride func(cv clusterversion.ClusterVersion) (Migration, bool) + + // BeforeWaitInRetryJobsWithExponentialBackoffMigration is called before + // waiting for a mutation job to complete in retryJobsWithExponentialBackoff + // migration. + // TODO(sajjad): Remove this knob when the related migration code is removed. + // This knob is used only in exponential backoff migration and related tests. See + // pkg/migration/retry_jobs_with_exponential_backoff.go and + // pkg/migration/retry_jobs_with_exponential_backoff_external_test.go + BeforeWaitInRetryJobsWithExponentialBackoffMigration func(jobspb.JobID) + + // SkippedMutation is called if a mutation job is skipped as part of the + // retryJobsWithExponentialBackoff migration. + // TODO(sajjad): Remove this knob when the related migration code is removed. + // This knob is used only in exponential backoff migration and related tests. See + // pkg/migration/retry_jobs_with_exponential_backoff.go and + // pkg/migration/retry_jobs_with_exponential_backoff_external_test.go + SkippedMutation func() } // ModuleTestingKnobs makes TestingKnobs a base.ModuleTestingKnobs. diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 925f9fa7741c..47a36724dfe9 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -313,7 +313,6 @@ go_test( "//pkg/kv/kvserver/liveness", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/migration", - "//pkg/migration/migrationmanager", "//pkg/migration/migrations", "//pkg/roachpb:with-mocks", "//pkg/rpc", diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 1baae5799ef7..d5490ef020f6 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -801,13 +801,14 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { } } - knobs, _ := cfg.TestingKnobs.MigrationManager.(*migrationmanager.TestingKnobs) + knobs, _ := cfg.TestingKnobs.MigrationManager.(*migration.TestingKnobs) migrationMgr := migrationmanager.NewManager( systemDeps, leaseMgr, cfg.circularInternalExecutor, jobRegistry, codec, cfg.Settings, knobs, ) execCfg.MigrationJobDeps = migrationMgr execCfg.VersionUpgradeHook = migrationMgr.Migrate + execCfg.MigrationTestingKnobs = knobs } temporaryObjectCleaner := sql.NewTemporaryObjectCleaner( diff --git a/pkg/server/version_cluster_test.go b/pkg/server/version_cluster_test.go index caf696868d07..265579c48c93 100644 --- a/pkg/server/version_cluster_test.go +++ b/pkg/server/version_cluster_test.go @@ -23,7 +23,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/migration" - "github.com/cockroachdb/cockroach/pkg/migration/migrationmanager" "github.com/cockroachdb/cockroach/pkg/migration/migrations" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" @@ -420,7 +419,7 @@ func TestClusterVersionMixedVersionTooOld(t *testing.T) { }, // Inject a migration which would run to upgrade the cluster. // We'll validate that we never create a job for this migration. - MigrationManager: &migrationmanager.TestingKnobs{ + MigrationManager: &migration.TestingKnobs{ ListBetweenOverride: func(from, to clusterversion.ClusterVersion) []clusterversion.ClusterVersion { return []clusterversion.ClusterVersion{to} }, diff --git a/pkg/sql/catalog/systemschema/system.go b/pkg/sql/catalog/systemschema/system.go index 0e6d1fcb4737..8e10365412ac 100644 --- a/pkg/sql/catalog/systemschema/system.go +++ b/pkg/sql/catalog/systemschema/system.go @@ -166,6 +166,9 @@ CREATE TABLE system.ui ( "lastUpdated" TIMESTAMP NOT NULL );` + // JobsRunStatsIdxPredicate is the predicate in jobs_run_stats_idx in JobsTable. + JobsRunStatsIdxPredicate = `status IN ('running':::STRING, 'reverting':::STRING, 'pending':::STRING, 'pause-requested':::STRING, 'cancel-requested':::STRING)` + // Note: this schema is changed in a migration (a progress column is added in // a separate family). // NB: main column family uses old, pre created_by_type/created_by_id columns, named. @@ -181,12 +184,19 @@ CREATE TABLE system.jobs ( created_by_id INT, claim_session_id BYTES, claim_instance_id INT8, + num_runs INT8, + last_run TIMESTAMP, INDEX (status, created), INDEX (created_by_type, created_by_id) STORING (status), - + INDEX jobs_run_stats_idx ( + claim_session_id, + status, + created + ) STORING(last_run, num_runs, claim_instance_id) + WHERE ` + JobsRunStatsIdxPredicate + `, FAMILY fam_0_id_status_created_payload (id, status, created, payload, created_by_type, created_by_id), FAMILY progress (progress), - FAMILY claim (claim_session_id, claim_instance_id) + FAMILY claim (claim_session_id, claim_instance_id, num_runs, last_run) );` // web_sessions are used to track authenticated user actions over stateless @@ -967,8 +977,7 @@ var ( NextMutationID: 1, }) - nowString = "now():::TIMESTAMP" - nowTZString = "now():::TIMESTAMPTZ" + nowString = "now():::TIMESTAMP" // JobsTable is the descriptor for the jobs table. JobsTable = makeTable(descpb.TableDescriptor{ @@ -987,8 +996,10 @@ var ( {Name: "created_by_id", ID: 7, Type: types.Int, Nullable: true}, {Name: "claim_session_id", ID: 8, Type: types.Bytes, Nullable: true}, {Name: "claim_instance_id", ID: 9, Type: types.Int, Nullable: true}, + {Name: "num_runs", ID: 10, Type: types.Int, Nullable: true}, + {Name: "last_run", ID: 11, Type: types.Timestamp, Nullable: true}, }, - NextColumnID: 10, + NextColumnID: 12, Families: []descpb.ColumnFamilyDescriptor{ { // NB: We are using family name that existed prior to adding created_by_type and @@ -1009,8 +1020,8 @@ var ( { Name: "claim", ID: 2, - ColumnNames: []string{"claim_session_id", "claim_instance_id"}, - ColumnIDs: []descpb.ColumnID{8, 9}, + ColumnNames: []string{"claim_session_id", "claim_instance_id", "num_runs", "last_run"}, + ColumnIDs: []descpb.ColumnID{8, 9, 10, 11}, }, }, NextFamilyID: 3, @@ -1038,8 +1049,21 @@ var ( KeySuffixColumnIDs: []descpb.ColumnID{1}, Version: descpb.StrictIndexColumnIDGuaranteesVersion, }, + { + Name: "jobs_run_stats_idx", + ID: 4, + Unique: false, + KeyColumnNames: []string{"claim_session_id", "status", "created"}, + KeyColumnDirections: []descpb.IndexDescriptor_Direction{descpb.IndexDescriptor_ASC, descpb.IndexDescriptor_ASC, descpb.IndexDescriptor_ASC}, + KeyColumnIDs: []descpb.ColumnID{8, 2, 3}, + StoreColumnNames: []string{"last_run", "num_runs", "claim_instance_id"}, + StoreColumnIDs: []descpb.ColumnID{11, 10, 9}, + KeySuffixColumnIDs: []descpb.ColumnID{1}, + Version: descpb.StrictIndexColumnIDGuaranteesVersion, + Predicate: JobsRunStatsIdxPredicate, + }, }, - NextIndexID: 4, + NextIndexID: 5, Privileges: descpb.NewCustomSuperuserPrivilegeDescriptor( descpb.SystemAllowedPrivileges[keys.JobsTableID], security.NodeUserName()), FormatVersion: descpb.InterleavedFormatVersion, @@ -1754,6 +1778,8 @@ var ( NextMutationID: 1, }) + nowTZString = "now():::TIMESTAMPTZ" + // ScheduledJobsTable is the descriptor for the scheduled jobs table. ScheduledJobsTable = makeTable(descpb.TableDescriptor{ Name: "scheduled_jobs", diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index 794dc047c97a..98408c7f089d 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -1383,34 +1383,48 @@ func NewColOperator( opName := opNamePrefix + strings.ToLower(wf.Func.AggregateFunc.String()) result.finishBufferedWindowerArgs( ctx, flowCtx, windowArgs, opName, spec.ProcessorID, factory, true /* needsBuffer */) - aggArgs := colexecagg.NewAggregatorArgs{ - Allocator: windowArgs.MainAllocator, - InputTypes: argTypes, - EvalCtx: evalCtx, - } - // The aggregate function will be presented with a ColVec slice - // containing only the argument columns. - colIdx := make([]uint32, len(argTypes)) - for i := range argIdxs { - colIdx[i] = uint32(i) - } - aggregations := []execinfrapb.AggregatorSpec_Aggregation{{ - Func: *wf.Func.AggregateFunc, - ColIdx: colIdx, - }} - semaCtx := flowCtx.TypeResolverFactory.NewSemaContext(evalCtx.Txn) - aggArgs.Constructors, aggArgs.ConstArguments, aggArgs.OutputTypes, err = - colexecagg.ProcessAggregations(evalCtx, semaCtx, aggregations, argTypes) - // Window functions reuse the hash aggregate function implementation. - aggFnsAlloc, _, toClose, err := colexecagg.NewAggregateFuncsAlloc( - &aggArgs, aggregations, 1 /* allocSize */, colexecagg.WindowAggKind, - ) - if err != nil { - colexecerror.InternalError(err) + aggType := *wf.Func.AggregateFunc + switch *wf.Func.AggregateFunc { + case execinfrapb.CountRows: + // count_rows has a specialized implementation. + result.Root = colexecwindow.NewCountRowsOperator(windowArgs, wf.Frame, &wf.Ordering) + default: + aggArgs := colexecagg.NewAggregatorArgs{ + Allocator: windowArgs.MainAllocator, + InputTypes: argTypes, + EvalCtx: evalCtx, + } + // The aggregate function will be presented with a ColVec slice + // containing only the argument columns. + colIdx := make([]uint32, len(argTypes)) + for i := range argIdxs { + colIdx[i] = uint32(i) + } + aggregations := []execinfrapb.AggregatorSpec_Aggregation{{ + Func: aggType, + ColIdx: colIdx, + }} + semaCtx := flowCtx.TypeResolverFactory.NewSemaContext(evalCtx.Txn) + aggArgs.Constructors, aggArgs.ConstArguments, aggArgs.OutputTypes, err = + colexecagg.ProcessAggregations(evalCtx, semaCtx, aggregations, argTypes) + var toClose colexecop.Closers + var aggFnsAlloc *colexecagg.AggregateFuncsAlloc + if (aggType != execinfrapb.Min && aggType != execinfrapb.Max) || + wf.Frame.Exclusion != execinfrapb.WindowerSpec_Frame_NO_EXCLUSION || + !colexecwindow.WindowFrameCanShrink(wf.Frame, &wf.Ordering) { + // Min and max window functions have specialized implementations + // when the frame can shrink and has a default exclusion clause. + aggFnsAlloc, _, toClose, err = colexecagg.NewAggregateFuncsAlloc( + &aggArgs, aggregations, 1 /* allocSize */, colexecagg.WindowAggKind, + ) + if err != nil { + colexecerror.InternalError(err) + } + } + result.Root = colexecwindow.NewWindowAggregatorOperator( + windowArgs, aggType, wf.Frame, &wf.Ordering, argIdxs, + aggArgs.OutputTypes[0], aggFnsAlloc, toClose) } - result.Root = colexecwindow.NewWindowAggregatorOperator( - windowArgs, wf.Frame, &wf.Ordering, argIdxs, - aggArgs.OutputTypes[0], aggFnsAlloc, toClose) } else { colexecerror.InternalError(errors.AssertionFailedf("window function spec is nil")) } diff --git a/pkg/sql/colexec/colexecagg/avg_agg_tmpl.go b/pkg/sql/colexec/colexecagg/avg_agg_tmpl.go index ffa48081bfc6..00e8173a4fc3 100644 --- a/pkg/sql/colexec/colexecagg/avg_agg_tmpl.go +++ b/pkg/sql/colexec/colexecagg/avg_agg_tmpl.go @@ -57,6 +57,12 @@ func _ASSIGN_ADD(_, _, _, _, _, _ string) { colexecerror.InternalError(errors.AssertionFailedf("")) } +// _ASSIGN_SUBTRACT is the template subtraction function for assigning the first +// input to the result of the second input - the third input. +func _ASSIGN_SUBTRACT(_, _, _, _, _, _ string) { + colexecerror.InternalError(errors.AssertionFailedf("")) +} + // */}} func newAvg_AGGKINDAggAlloc( @@ -93,14 +99,11 @@ type avg_TYPE_AGGKINDAgg struct { // so we can index into the slice once per group, instead of on each // iteration. curSum _RET_GOTYPE - // curCount keeps track of the number of elements that we've seen + // curCount keeps track of the number of non-null elements that we've seen // belonging to the current group. curCount int64 // col points to the statically-typed output vector. col []_RET_GOTYPE - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool // {{if .NeedsHelper}} // {{/* // overloadHelper is used only when we perform the summation of integers @@ -209,7 +212,7 @@ func (a *avg_TYPE_AGGKINDAgg) Flush(outputIdx int) { outputIdx = a.curIdx a.curIdx++ // {{end}} - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(outputIdx) } else { _ASSIGN_DIV_INT64(a.col[outputIdx], a.curSum, a.curCount, a.col, _, _) @@ -222,7 +225,6 @@ func (a *avg_TYPE_AGGKINDAgg) Reset() { // {{end}} a.curSum = zero_RET_TYPEValue a.curCount = 0 - a.foundNonNullForCurrentGroup = false } type avg_TYPE_AGGKINDAggAlloc struct { @@ -246,6 +248,36 @@ func (a *avg_TYPE_AGGKINDAggAlloc) newAggFunc() AggregateFunc { return f } +// {{if eq "_AGGKIND" "Window"}} + +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). +func (a *avg_TYPE_AGGKINDAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + // {{if .NeedsHelper}} + // In order to inline the templated code of overloads, we need to have a + // "_overloadHelper" local variable of type "overloadHelper". + _overloadHelper := a.overloadHelper + // {{end}} + execgen.SETVARIABLESIZE(oldCurSumSize, a.curSum) + vec := vecs[inputIdxs[0]] + col, nulls := vec.TemplateType(), vec.Nulls() + _, _ = col.Get(endIdx-1), col.Get(startIdx) + if nulls.MaybeHasNulls() { + for i := startIdx; i < endIdx; i++ { + _REMOVE_ROW(a, nulls, i, true) + } + } else { + for i := startIdx; i < endIdx; i++ { + _REMOVE_ROW(a, nulls, i, false) + } + } + execgen.SETVARIABLESIZE(newCurSumSize, a.curSum) + if newCurSumSize != oldCurSumSize { + a.allocator.AdjustMemoryUsage(int64(newCurSumSize - oldCurSumSize)) + } +} + +// {{end}} // {{end}} // {{end}} // {{end}} @@ -268,7 +300,7 @@ func _ACCUMULATE_AVG( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { // {{with .Global}} @@ -280,14 +312,6 @@ func _ACCUMULATE_AVG( a.curSum = zero_RET_TYPEValue // {{end}} a.curCount = 0 - - // {{/* - // We only need to reset this flag if there are nulls. If there are no - // nulls, this will be updated unconditionally below. - // */}} - // {{if .HasNulls}} - a.foundNonNullForCurrentGroup = false - // {{end}} } a.isFirstGroup = false } @@ -306,9 +330,29 @@ func _ACCUMULATE_AVG( v := col.Get(i) _ASSIGN_ADD(a.curSum, a.curSum, v, _, _, col) a.curCount++ - a.foundNonNullForCurrentGroup = true } // {{end}} // {{/* } // */}} + +// {{/* +// _REMOVE_ROW removes the value of the ith row from the output for the +// current aggregation. +func _REMOVE_ROW(a *_AGG_TYPE_AGGKINDAgg, nulls *coldata.Nulls, i int, _HAS_NULLS bool) { // */}} + // {{define "removeRow"}} + var isNull bool + // {{if .HasNulls}} + isNull = nulls.NullAt(i) + // {{else}} + isNull = false + // {{end}} + if !isNull { + //gcassert:bce + v := col.Get(i) + _ASSIGN_SUBTRACT(a.curSum, a.curSum, v, _, _, col) + a.curCount-- + } + // {{end}} + // {{/* +} // */}} diff --git a/pkg/sql/colexec/colexecagg/count_agg_tmpl.go b/pkg/sql/colexec/colexecagg/count_agg_tmpl.go index a940c4235617..582117f01deb 100644 --- a/pkg/sql/colexec/colexecagg/count_agg_tmpl.go +++ b/pkg/sql/colexec/colexecagg/count_agg_tmpl.go @@ -23,7 +23,6 @@ import ( "unsafe" "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" "github.com/cockroachdb/cockroach/pkg/sql/colmem" ) @@ -63,7 +62,6 @@ func (a *count_COUNTKIND_AGGKINDAgg) SetOutput(vec coldata.Vec) { func (a *count_COUNTKIND_AGGKINDAgg) Compute( vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int, sel []int, ) { - execgen.SETVARIABLESIZE(oldCurAggSize, a.curAgg) // {{if not (eq .CountKind "Rows")}} // If this is a COUNT(col) aggregator and there are nulls in this batch, // we must check each value for nullity. Note that it is only legal to do a @@ -138,10 +136,6 @@ func (a *count_COUNTKIND_AGGKINDAgg) Compute( } } // {{end}} - execgen.SETVARIABLESIZE(newCurAggSize, a.curAgg) - if newCurAggSize != oldCurAggSize { - a.allocator.AdjustMemoryUsage(int64(newCurAggSize - oldCurAggSize)) - } } func (a *count_COUNTKIND_AGGKINDAgg) Flush(outputIdx int) { @@ -170,6 +164,27 @@ func (a *count_COUNTKIND_AGGKINDAgg) Reset() { a.curAgg = 0 } +// {{if and (eq "_AGGKIND" "Window") (not (eq .CountKind "Rows"))}} + +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). +func (a *count_COUNTKIND_AGGKINDAgg) Remove( + vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int, +) { + nulls := vecs[inputIdxs[0]].Nulls() + if nulls.MaybeHasNulls() { + for i := startIdx; i < endIdx; i++ { + _REMOVE_ROW(a, nulls, i, true) + } + } else { + for i := startIdx; i < endIdx; i++ { + _REMOVE_ROW(a, nulls, i, false) + } + } +} + +// {{end}} + type count_COUNTKIND_AGGKINDAggAlloc struct { aggAllocBase aggFuncs []count_COUNTKIND_AGGKINDAgg @@ -229,3 +244,22 @@ func _ACCUMULATE_COUNT( // {{/* } // */}} + +// {{/* +// _REMOVE_ROW removes the value of the ith row from the output for the +// current aggregation. +func _REMOVE_ROW(a *countAgg, nulls *coldata.Nulls, i int, _COL_WITH_NULLS bool) { // */}} + // {{define "removeRow"}} + var y int64 + // {{if .ColWithNulls}} + y = int64(0) + if !nulls.NullAt(i) { + y = 1 + } + // {{else}} + y = int64(1) + // {{end}} + a.curAgg -= y + // {{end}} + // {{/* +} // */}} diff --git a/pkg/sql/colexec/colexecagg/hash_avg_agg.eg.go b/pkg/sql/colexec/colexecagg/hash_avg_agg.eg.go index 979fea03d489..cc5528c75381 100644 --- a/pkg/sql/colexec/colexecagg/hash_avg_agg.eg.go +++ b/pkg/sql/colexec/colexecagg/hash_avg_agg.eg.go @@ -74,15 +74,12 @@ type avgInt16HashAgg struct { // so we can index into the slice once per group, instead of on each // iteration. curSum apd.Decimal - // curCount keeps track of the number of elements that we've seen + // curCount keeps track of the number of non-null elements that we've seen // belonging to the current group. curCount int64 // col points to the statically-typed output vector. - col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool - overloadHelper execgen.OverloadHelper + col []apd.Decimal + overloadHelper execgen.OverloadHelper } var _ AggregateFunc = &avgInt16HashAgg{} @@ -122,7 +119,6 @@ func (a *avgInt16HashAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -143,7 +139,6 @@ func (a *avgInt16HashAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -160,7 +155,7 @@ func (a *avgInt16HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // NULL. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(outputIdx) } else { @@ -174,7 +169,6 @@ func (a *avgInt16HashAgg) Flush(outputIdx int) { func (a *avgInt16HashAgg) Reset() { a.curSum = zeroDecimalValue a.curCount = 0 - a.foundNonNullForCurrentGroup = false } type avgInt16HashAggAlloc struct { @@ -204,15 +198,12 @@ type avgInt32HashAgg struct { // so we can index into the slice once per group, instead of on each // iteration. curSum apd.Decimal - // curCount keeps track of the number of elements that we've seen + // curCount keeps track of the number of non-null elements that we've seen // belonging to the current group. curCount int64 // col points to the statically-typed output vector. - col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool - overloadHelper execgen.OverloadHelper + col []apd.Decimal + overloadHelper execgen.OverloadHelper } var _ AggregateFunc = &avgInt32HashAgg{} @@ -252,7 +243,6 @@ func (a *avgInt32HashAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -273,7 +263,6 @@ func (a *avgInt32HashAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -290,7 +279,7 @@ func (a *avgInt32HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // NULL. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(outputIdx) } else { @@ -304,7 +293,6 @@ func (a *avgInt32HashAgg) Flush(outputIdx int) { func (a *avgInt32HashAgg) Reset() { a.curSum = zeroDecimalValue a.curCount = 0 - a.foundNonNullForCurrentGroup = false } type avgInt32HashAggAlloc struct { @@ -334,15 +322,12 @@ type avgInt64HashAgg struct { // so we can index into the slice once per group, instead of on each // iteration. curSum apd.Decimal - // curCount keeps track of the number of elements that we've seen + // curCount keeps track of the number of non-null elements that we've seen // belonging to the current group. curCount int64 // col points to the statically-typed output vector. - col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool - overloadHelper execgen.OverloadHelper + col []apd.Decimal + overloadHelper execgen.OverloadHelper } var _ AggregateFunc = &avgInt64HashAgg{} @@ -382,7 +367,6 @@ func (a *avgInt64HashAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -403,7 +387,6 @@ func (a *avgInt64HashAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -420,7 +403,7 @@ func (a *avgInt64HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // NULL. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(outputIdx) } else { @@ -434,7 +417,6 @@ func (a *avgInt64HashAgg) Flush(outputIdx int) { func (a *avgInt64HashAgg) Reset() { a.curSum = zeroDecimalValue a.curCount = 0 - a.foundNonNullForCurrentGroup = false } type avgInt64HashAggAlloc struct { @@ -464,14 +446,11 @@ type avgDecimalHashAgg struct { // so we can index into the slice once per group, instead of on each // iteration. curSum apd.Decimal - // curCount keeps track of the number of elements that we've seen + // curCount keeps track of the number of non-null elements that we've seen // belonging to the current group. curCount int64 // col points to the statically-typed output vector. col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool } var _ AggregateFunc = &avgDecimalHashAgg{} @@ -507,7 +486,6 @@ func (a *avgDecimalHashAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -527,7 +505,6 @@ func (a *avgDecimalHashAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -544,7 +521,7 @@ func (a *avgDecimalHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // NULL. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(outputIdx) } else { @@ -558,7 +535,6 @@ func (a *avgDecimalHashAgg) Flush(outputIdx int) { func (a *avgDecimalHashAgg) Reset() { a.curSum = zeroDecimalValue a.curCount = 0 - a.foundNonNullForCurrentGroup = false } type avgDecimalHashAggAlloc struct { @@ -588,14 +564,11 @@ type avgFloat64HashAgg struct { // so we can index into the slice once per group, instead of on each // iteration. curSum float64 - // curCount keeps track of the number of elements that we've seen + // curCount keeps track of the number of non-null elements that we've seen // belonging to the current group. curCount int64 // col points to the statically-typed output vector. col []float64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool } var _ AggregateFunc = &avgFloat64HashAgg{} @@ -628,7 +601,6 @@ func (a *avgFloat64HashAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -645,7 +617,6 @@ func (a *avgFloat64HashAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -662,7 +633,7 @@ func (a *avgFloat64HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // NULL. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curSum / float64(a.curCount) @@ -672,7 +643,6 @@ func (a *avgFloat64HashAgg) Flush(outputIdx int) { func (a *avgFloat64HashAgg) Reset() { a.curSum = zeroFloat64Value a.curCount = 0 - a.foundNonNullForCurrentGroup = false } type avgFloat64HashAggAlloc struct { @@ -702,14 +672,11 @@ type avgIntervalHashAgg struct { // so we can index into the slice once per group, instead of on each // iteration. curSum duration.Duration - // curCount keeps track of the number of elements that we've seen + // curCount keeps track of the number of non-null elements that we've seen // belonging to the current group. curCount int64 // col points to the statically-typed output vector. col []duration.Duration - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool } var _ AggregateFunc = &avgIntervalHashAgg{} @@ -737,7 +704,6 @@ func (a *avgIntervalHashAgg) Compute( v := col.Get(i) a.curSum = a.curSum.Add(v) a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -749,7 +715,6 @@ func (a *avgIntervalHashAgg) Compute( v := col.Get(i) a.curSum = a.curSum.Add(v) a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -766,7 +731,7 @@ func (a *avgIntervalHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // NULL. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curSum.Div(int64(a.curCount)) @@ -776,7 +741,6 @@ func (a *avgIntervalHashAgg) Flush(outputIdx int) { func (a *avgIntervalHashAgg) Reset() { a.curSum = zeroIntervalValue a.curCount = 0 - a.foundNonNullForCurrentGroup = false } type avgIntervalHashAggAlloc struct { diff --git a/pkg/sql/colexec/colexecagg/hash_count_agg.eg.go b/pkg/sql/colexec/colexecagg/hash_count_agg.eg.go index 82543eaa6bac..034ac2915ce5 100644 --- a/pkg/sql/colexec/colexecagg/hash_count_agg.eg.go +++ b/pkg/sql/colexec/colexecagg/hash_count_agg.eg.go @@ -42,7 +42,6 @@ func (a *countRowsHashAgg) SetOutput(vec coldata.Vec) { func (a *countRowsHashAgg) Compute( vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int, sel []int, ) { - var oldCurAggSize uintptr a.allocator.PerformOperation([]coldata.Vec{a.vec}, func() { { { @@ -55,10 +54,6 @@ func (a *countRowsHashAgg) Compute( } }, ) - var newCurAggSize uintptr - if newCurAggSize != oldCurAggSize { - a.allocator.AdjustMemoryUsage(int64(newCurAggSize - oldCurAggSize)) - } } func (a *countRowsHashAgg) Flush(outputIdx int) { @@ -116,7 +111,6 @@ func (a *countHashAgg) SetOutput(vec coldata.Vec) { func (a *countHashAgg) Compute( vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int, sel []int, ) { - var oldCurAggSize uintptr // If this is a COUNT(col) aggregator and there are nulls in this batch, // we must check each value for nullity. Note that it is only legal to do a // COUNT aggregate on a single column. @@ -143,10 +137,6 @@ func (a *countHashAgg) Compute( } }, ) - var newCurAggSize uintptr - if newCurAggSize != oldCurAggSize { - a.allocator.AdjustMemoryUsage(int64(newCurAggSize - oldCurAggSize)) - } } func (a *countHashAgg) Flush(outputIdx int) { diff --git a/pkg/sql/colexec/colexecagg/hash_min_max_agg.eg.go b/pkg/sql/colexec/colexecagg/hash_min_max_agg.eg.go index 2965a211e7fc..d02bb69cd302 100644 --- a/pkg/sql/colexec/colexecagg/hash_min_max_agg.eg.go +++ b/pkg/sql/colexec/colexecagg/hash_min_max_agg.eg.go @@ -116,11 +116,11 @@ type minBoolHashAgg struct { col coldata.Bools // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg bool - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minBoolHashAgg{} @@ -145,10 +145,9 @@ func (a *minBoolHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -171,6 +170,7 @@ func (a *minBoolHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -179,10 +179,9 @@ func (a *minBoolHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -205,6 +204,7 @@ func (a *minBoolHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -221,7 +221,7 @@ func (a *minBoolHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -229,7 +229,7 @@ func (a *minBoolHashAgg) Flush(outputIdx int) { } func (a *minBoolHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minBoolHashAggAlloc struct { @@ -259,11 +259,11 @@ type minBytesHashAgg struct { col *coldata.Bytes // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg []byte - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minBytesHashAgg{} @@ -288,10 +288,9 @@ func (a *minBytesHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -306,6 +305,7 @@ func (a *minBytesHashAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } else { @@ -314,10 +314,9 @@ func (a *minBytesHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -332,6 +331,7 @@ func (a *minBytesHashAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } @@ -348,19 +348,24 @@ func (a *minBytesHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) } oldCurAggSize := len(a.curAgg) - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *minBytesHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + oldCurAggSize := len(a.curAgg) + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type minBytesHashAggAlloc struct { @@ -390,11 +395,11 @@ type minDecimalHashAgg struct { col coldata.Decimals // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minDecimalHashAgg{} @@ -419,10 +424,9 @@ func (a *minDecimalHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -437,6 +441,7 @@ func (a *minDecimalHashAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } else { @@ -445,10 +450,9 @@ func (a *minDecimalHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -463,6 +467,7 @@ func (a *minDecimalHashAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } @@ -479,7 +484,7 @@ func (a *minDecimalHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -487,7 +492,7 @@ func (a *minDecimalHashAgg) Flush(outputIdx int) { } func (a *minDecimalHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minDecimalHashAggAlloc struct { @@ -517,11 +522,11 @@ type minInt16HashAgg struct { col coldata.Int16s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int16 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minInt16HashAgg{} @@ -546,10 +551,9 @@ func (a *minInt16HashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -575,6 +579,7 @@ func (a *minInt16HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -583,10 +588,9 @@ func (a *minInt16HashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -612,6 +616,7 @@ func (a *minInt16HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -628,7 +633,7 @@ func (a *minInt16HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -636,7 +641,7 @@ func (a *minInt16HashAgg) Flush(outputIdx int) { } func (a *minInt16HashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minInt16HashAggAlloc struct { @@ -666,11 +671,11 @@ type minInt32HashAgg struct { col coldata.Int32s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int32 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minInt32HashAgg{} @@ -695,10 +700,9 @@ func (a *minInt32HashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -724,6 +728,7 @@ func (a *minInt32HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -732,10 +737,9 @@ func (a *minInt32HashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -761,6 +765,7 @@ func (a *minInt32HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -777,7 +782,7 @@ func (a *minInt32HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -785,7 +790,7 @@ func (a *minInt32HashAgg) Flush(outputIdx int) { } func (a *minInt32HashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minInt32HashAggAlloc struct { @@ -815,11 +820,11 @@ type minInt64HashAgg struct { col coldata.Int64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minInt64HashAgg{} @@ -844,10 +849,9 @@ func (a *minInt64HashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -873,6 +877,7 @@ func (a *minInt64HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -881,10 +886,9 @@ func (a *minInt64HashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -910,6 +914,7 @@ func (a *minInt64HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -926,7 +931,7 @@ func (a *minInt64HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -934,7 +939,7 @@ func (a *minInt64HashAgg) Flush(outputIdx int) { } func (a *minInt64HashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minInt64HashAggAlloc struct { @@ -964,11 +969,11 @@ type minFloat64HashAgg struct { col coldata.Float64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg float64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minFloat64HashAgg{} @@ -993,10 +998,9 @@ func (a *minFloat64HashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1030,6 +1034,7 @@ func (a *minFloat64HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1038,10 +1043,9 @@ func (a *minFloat64HashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1075,6 +1079,7 @@ func (a *minFloat64HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1091,7 +1096,7 @@ func (a *minFloat64HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -1099,7 +1104,7 @@ func (a *minFloat64HashAgg) Flush(outputIdx int) { } func (a *minFloat64HashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minFloat64HashAggAlloc struct { @@ -1129,11 +1134,11 @@ type minTimestampHashAgg struct { col coldata.Times // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg time.Time - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minTimestampHashAgg{} @@ -1158,10 +1163,9 @@ func (a *minTimestampHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1183,6 +1187,7 @@ func (a *minTimestampHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1191,10 +1196,9 @@ func (a *minTimestampHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1216,6 +1220,7 @@ func (a *minTimestampHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1232,7 +1237,7 @@ func (a *minTimestampHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -1240,7 +1245,7 @@ func (a *minTimestampHashAgg) Flush(outputIdx int) { } func (a *minTimestampHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minTimestampHashAggAlloc struct { @@ -1270,11 +1275,11 @@ type minIntervalHashAgg struct { col coldata.Durations // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg duration.Duration - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minIntervalHashAgg{} @@ -1299,10 +1304,9 @@ func (a *minIntervalHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1317,6 +1321,7 @@ func (a *minIntervalHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1325,10 +1330,9 @@ func (a *minIntervalHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1343,6 +1347,7 @@ func (a *minIntervalHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1359,7 +1364,7 @@ func (a *minIntervalHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -1367,7 +1372,7 @@ func (a *minIntervalHashAgg) Flush(outputIdx int) { } func (a *minIntervalHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minIntervalHashAggAlloc struct { @@ -1397,11 +1402,11 @@ type minJSONHashAgg struct { col *coldata.JSONs // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg json.JSON - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minJSONHashAgg{} @@ -1429,7 +1434,7 @@ func (a *minJSONHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -1443,7 +1448,6 @@ func (a *minJSONHashAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1475,6 +1479,7 @@ func (a *minJSONHashAgg) Compute( } } + a.numNonNull++ } } } else { @@ -1483,7 +1488,7 @@ func (a *minJSONHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -1497,7 +1502,6 @@ func (a *minJSONHashAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1529,6 +1533,7 @@ func (a *minJSONHashAgg) Compute( } } + a.numNonNull++ } } } @@ -1548,7 +1553,7 @@ func (a *minJSONHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -1557,13 +1562,21 @@ func (a *minJSONHashAgg) Flush(outputIdx int) { if a.curAgg != nil { oldCurAggSize = a.curAgg.Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *minJSONHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + var oldCurAggSize uintptr + if a.curAgg != nil { + oldCurAggSize = a.curAgg.Size() + } + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type minJSONHashAggAlloc struct { @@ -1593,11 +1606,11 @@ type minDatumHashAgg struct { col coldata.DatumVec // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg interface{} - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minDatumHashAgg{} @@ -1626,10 +1639,9 @@ func (a *minDatumHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1646,6 +1658,7 @@ func (a *minDatumHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1654,10 +1667,9 @@ func (a *minDatumHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1674,6 +1686,7 @@ func (a *minDatumHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1694,7 +1707,7 @@ func (a *minDatumHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -1704,13 +1717,22 @@ func (a *minDatumHashAgg) Flush(outputIdx int) { if a.curAgg != nil { oldCurAggSize = a.curAgg.(tree.Datum).Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *minDatumHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + + var oldCurAggSize uintptr + if a.curAgg != nil { + oldCurAggSize = a.curAgg.(tree.Datum).Size() + } + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type minDatumHashAggAlloc struct { @@ -1809,11 +1831,11 @@ type maxBoolHashAgg struct { col coldata.Bools // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg bool - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxBoolHashAgg{} @@ -1838,10 +1860,9 @@ func (a *maxBoolHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1864,6 +1885,7 @@ func (a *maxBoolHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1872,10 +1894,9 @@ func (a *maxBoolHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1898,6 +1919,7 @@ func (a *maxBoolHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1914,7 +1936,7 @@ func (a *maxBoolHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -1922,7 +1944,7 @@ func (a *maxBoolHashAgg) Flush(outputIdx int) { } func (a *maxBoolHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxBoolHashAggAlloc struct { @@ -1952,11 +1974,11 @@ type maxBytesHashAgg struct { col *coldata.Bytes // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg []byte - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxBytesHashAgg{} @@ -1981,10 +2003,9 @@ func (a *maxBytesHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1999,6 +2020,7 @@ func (a *maxBytesHashAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } else { @@ -2007,10 +2029,9 @@ func (a *maxBytesHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2025,6 +2046,7 @@ func (a *maxBytesHashAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } @@ -2041,19 +2063,24 @@ func (a *maxBytesHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) } oldCurAggSize := len(a.curAgg) - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *maxBytesHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + oldCurAggSize := len(a.curAgg) + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type maxBytesHashAggAlloc struct { @@ -2083,11 +2110,11 @@ type maxDecimalHashAgg struct { col coldata.Decimals // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxDecimalHashAgg{} @@ -2112,10 +2139,9 @@ func (a *maxDecimalHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2130,6 +2156,7 @@ func (a *maxDecimalHashAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } else { @@ -2138,10 +2165,9 @@ func (a *maxDecimalHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2156,6 +2182,7 @@ func (a *maxDecimalHashAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } @@ -2172,7 +2199,7 @@ func (a *maxDecimalHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -2180,7 +2207,7 @@ func (a *maxDecimalHashAgg) Flush(outputIdx int) { } func (a *maxDecimalHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxDecimalHashAggAlloc struct { @@ -2210,11 +2237,11 @@ type maxInt16HashAgg struct { col coldata.Int16s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int16 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxInt16HashAgg{} @@ -2239,10 +2266,9 @@ func (a *maxInt16HashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2268,6 +2294,7 @@ func (a *maxInt16HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2276,10 +2303,9 @@ func (a *maxInt16HashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2305,6 +2331,7 @@ func (a *maxInt16HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2321,7 +2348,7 @@ func (a *maxInt16HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -2329,7 +2356,7 @@ func (a *maxInt16HashAgg) Flush(outputIdx int) { } func (a *maxInt16HashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxInt16HashAggAlloc struct { @@ -2359,11 +2386,11 @@ type maxInt32HashAgg struct { col coldata.Int32s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int32 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxInt32HashAgg{} @@ -2388,10 +2415,9 @@ func (a *maxInt32HashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2417,6 +2443,7 @@ func (a *maxInt32HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2425,10 +2452,9 @@ func (a *maxInt32HashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2454,6 +2480,7 @@ func (a *maxInt32HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2470,7 +2497,7 @@ func (a *maxInt32HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -2478,7 +2505,7 @@ func (a *maxInt32HashAgg) Flush(outputIdx int) { } func (a *maxInt32HashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxInt32HashAggAlloc struct { @@ -2508,11 +2535,11 @@ type maxInt64HashAgg struct { col coldata.Int64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxInt64HashAgg{} @@ -2537,10 +2564,9 @@ func (a *maxInt64HashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2566,6 +2592,7 @@ func (a *maxInt64HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2574,10 +2601,9 @@ func (a *maxInt64HashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2603,6 +2629,7 @@ func (a *maxInt64HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2619,7 +2646,7 @@ func (a *maxInt64HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -2627,7 +2654,7 @@ func (a *maxInt64HashAgg) Flush(outputIdx int) { } func (a *maxInt64HashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxInt64HashAggAlloc struct { @@ -2657,11 +2684,11 @@ type maxFloat64HashAgg struct { col coldata.Float64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg float64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxFloat64HashAgg{} @@ -2686,10 +2713,9 @@ func (a *maxFloat64HashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2723,6 +2749,7 @@ func (a *maxFloat64HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2731,10 +2758,9 @@ func (a *maxFloat64HashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2768,6 +2794,7 @@ func (a *maxFloat64HashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2784,7 +2811,7 @@ func (a *maxFloat64HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -2792,7 +2819,7 @@ func (a *maxFloat64HashAgg) Flush(outputIdx int) { } func (a *maxFloat64HashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxFloat64HashAggAlloc struct { @@ -2822,11 +2849,11 @@ type maxTimestampHashAgg struct { col coldata.Times // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg time.Time - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxTimestampHashAgg{} @@ -2851,10 +2878,9 @@ func (a *maxTimestampHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2876,6 +2902,7 @@ func (a *maxTimestampHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2884,10 +2911,9 @@ func (a *maxTimestampHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2909,6 +2935,7 @@ func (a *maxTimestampHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2925,7 +2952,7 @@ func (a *maxTimestampHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -2933,7 +2960,7 @@ func (a *maxTimestampHashAgg) Flush(outputIdx int) { } func (a *maxTimestampHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxTimestampHashAggAlloc struct { @@ -2963,11 +2990,11 @@ type maxIntervalHashAgg struct { col coldata.Durations // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg duration.Duration - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxIntervalHashAgg{} @@ -2992,10 +3019,9 @@ func (a *maxIntervalHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3010,6 +3036,7 @@ func (a *maxIntervalHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -3018,10 +3045,9 @@ func (a *maxIntervalHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3036,6 +3062,7 @@ func (a *maxIntervalHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -3052,7 +3079,7 @@ func (a *maxIntervalHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -3060,7 +3087,7 @@ func (a *maxIntervalHashAgg) Flush(outputIdx int) { } func (a *maxIntervalHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxIntervalHashAggAlloc struct { @@ -3090,11 +3117,11 @@ type maxJSONHashAgg struct { col *coldata.JSONs // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg json.JSON - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxJSONHashAgg{} @@ -3122,7 +3149,7 @@ func (a *maxJSONHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -3136,7 +3163,6 @@ func (a *maxJSONHashAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3168,6 +3194,7 @@ func (a *maxJSONHashAgg) Compute( } } + a.numNonNull++ } } } else { @@ -3176,7 +3203,7 @@ func (a *maxJSONHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -3190,7 +3217,6 @@ func (a *maxJSONHashAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3222,6 +3248,7 @@ func (a *maxJSONHashAgg) Compute( } } + a.numNonNull++ } } } @@ -3241,7 +3268,7 @@ func (a *maxJSONHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -3250,13 +3277,21 @@ func (a *maxJSONHashAgg) Flush(outputIdx int) { if a.curAgg != nil { oldCurAggSize = a.curAgg.Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *maxJSONHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + var oldCurAggSize uintptr + if a.curAgg != nil { + oldCurAggSize = a.curAgg.Size() + } + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type maxJSONHashAggAlloc struct { @@ -3286,11 +3321,11 @@ type maxDatumHashAgg struct { col coldata.DatumVec // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg interface{} - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxDatumHashAgg{} @@ -3319,10 +3354,9 @@ func (a *maxDatumHashAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3339,6 +3373,7 @@ func (a *maxDatumHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -3347,10 +3382,9 @@ func (a *maxDatumHashAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3367,6 +3401,7 @@ func (a *maxDatumHashAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -3387,7 +3422,7 @@ func (a *maxDatumHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -3397,13 +3432,22 @@ func (a *maxDatumHashAgg) Flush(outputIdx int) { if a.curAgg != nil { oldCurAggSize = a.curAgg.(tree.Datum).Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *maxDatumHashAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + + var oldCurAggSize uintptr + if a.curAgg != nil { + oldCurAggSize = a.curAgg.(tree.Datum).Size() + } + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type maxDatumHashAggAlloc struct { diff --git a/pkg/sql/colexec/colexecagg/hash_sum_agg.eg.go b/pkg/sql/colexec/colexecagg/hash_sum_agg.eg.go index e6b789f2f7a2..60f7346079ed 100644 --- a/pkg/sql/colexec/colexecagg/hash_sum_agg.eg.go +++ b/pkg/sql/colexec/colexecagg/hash_sum_agg.eg.go @@ -75,10 +75,10 @@ type sumInt16HashAgg struct { curAgg apd.Decimal // col points to the output vector we are updating. col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool - overloadHelper execgen.OverloadHelper + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 + overloadHelper execgen.OverloadHelper } var _ AggregateFunc = &sumInt16HashAgg{} @@ -117,7 +117,7 @@ func (a *sumInt16HashAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -137,7 +137,7 @@ func (a *sumInt16HashAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -154,7 +154,7 @@ func (a *sumInt16HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curAgg @@ -163,7 +163,7 @@ func (a *sumInt16HashAgg) Flush(outputIdx int) { func (a *sumInt16HashAgg) Reset() { a.curAgg = zeroDecimalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumInt16HashAggAlloc struct { @@ -194,10 +194,10 @@ type sumInt32HashAgg struct { curAgg apd.Decimal // col points to the output vector we are updating. col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool - overloadHelper execgen.OverloadHelper + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 + overloadHelper execgen.OverloadHelper } var _ AggregateFunc = &sumInt32HashAgg{} @@ -236,7 +236,7 @@ func (a *sumInt32HashAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -256,7 +256,7 @@ func (a *sumInt32HashAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -273,7 +273,7 @@ func (a *sumInt32HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curAgg @@ -282,7 +282,7 @@ func (a *sumInt32HashAgg) Flush(outputIdx int) { func (a *sumInt32HashAgg) Reset() { a.curAgg = zeroDecimalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumInt32HashAggAlloc struct { @@ -313,10 +313,10 @@ type sumInt64HashAgg struct { curAgg apd.Decimal // col points to the output vector we are updating. col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool - overloadHelper execgen.OverloadHelper + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 + overloadHelper execgen.OverloadHelper } var _ AggregateFunc = &sumInt64HashAgg{} @@ -355,7 +355,7 @@ func (a *sumInt64HashAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -375,7 +375,7 @@ func (a *sumInt64HashAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -392,7 +392,7 @@ func (a *sumInt64HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curAgg @@ -401,7 +401,7 @@ func (a *sumInt64HashAgg) Flush(outputIdx int) { func (a *sumInt64HashAgg) Reset() { a.curAgg = zeroDecimalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumInt64HashAggAlloc struct { @@ -432,9 +432,9 @@ type sumDecimalHashAgg struct { curAgg apd.Decimal // col points to the output vector we are updating. col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &sumDecimalHashAgg{} @@ -469,7 +469,7 @@ func (a *sumDecimalHashAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -488,7 +488,7 @@ func (a *sumDecimalHashAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -505,7 +505,7 @@ func (a *sumDecimalHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curAgg @@ -514,7 +514,7 @@ func (a *sumDecimalHashAgg) Flush(outputIdx int) { func (a *sumDecimalHashAgg) Reset() { a.curAgg = zeroDecimalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumDecimalHashAggAlloc struct { @@ -545,9 +545,9 @@ type sumFloat64HashAgg struct { curAgg float64 // col points to the output vector we are updating. col []float64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &sumFloat64HashAgg{} @@ -579,7 +579,7 @@ func (a *sumFloat64HashAgg) Compute( a.curAgg = float64(a.curAgg) + float64(v) } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -595,7 +595,7 @@ func (a *sumFloat64HashAgg) Compute( a.curAgg = float64(a.curAgg) + float64(v) } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -612,7 +612,7 @@ func (a *sumFloat64HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curAgg @@ -621,7 +621,7 @@ func (a *sumFloat64HashAgg) Flush(outputIdx int) { func (a *sumFloat64HashAgg) Reset() { a.curAgg = zeroFloat64Value - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumFloat64HashAggAlloc struct { @@ -652,9 +652,9 @@ type sumIntervalHashAgg struct { curAgg duration.Duration // col points to the output vector we are updating. col []duration.Duration - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &sumIntervalHashAgg{} @@ -681,7 +681,7 @@ func (a *sumIntervalHashAgg) Compute( if !isNull { v := col.Get(i) a.curAgg = a.curAgg.Add(v) - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -692,7 +692,7 @@ func (a *sumIntervalHashAgg) Compute( if !isNull { v := col.Get(i) a.curAgg = a.curAgg.Add(v) - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -709,7 +709,7 @@ func (a *sumIntervalHashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curAgg @@ -718,7 +718,7 @@ func (a *sumIntervalHashAgg) Flush(outputIdx int) { func (a *sumIntervalHashAgg) Reset() { a.curAgg = zeroIntervalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumIntervalHashAggAlloc struct { diff --git a/pkg/sql/colexec/colexecagg/hash_sum_int_agg.eg.go b/pkg/sql/colexec/colexecagg/hash_sum_int_agg.eg.go index 9a9404cac9b4..ba9c01815e8a 100644 --- a/pkg/sql/colexec/colexecagg/hash_sum_int_agg.eg.go +++ b/pkg/sql/colexec/colexecagg/hash_sum_int_agg.eg.go @@ -56,9 +56,9 @@ type sumIntInt16HashAgg struct { curAgg int64 // col points to the output vector we are updating. col []int64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &sumIntInt16HashAgg{} @@ -93,7 +93,7 @@ func (a *sumIntInt16HashAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -112,7 +112,7 @@ func (a *sumIntInt16HashAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -129,7 +129,7 @@ func (a *sumIntInt16HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curAgg @@ -138,7 +138,7 @@ func (a *sumIntInt16HashAgg) Flush(outputIdx int) { func (a *sumIntInt16HashAgg) Reset() { a.curAgg = zeroInt64Value - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumIntInt16HashAggAlloc struct { @@ -169,9 +169,9 @@ type sumIntInt32HashAgg struct { curAgg int64 // col points to the output vector we are updating. col []int64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &sumIntInt32HashAgg{} @@ -206,7 +206,7 @@ func (a *sumIntInt32HashAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -225,7 +225,7 @@ func (a *sumIntInt32HashAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -242,7 +242,7 @@ func (a *sumIntInt32HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curAgg @@ -251,7 +251,7 @@ func (a *sumIntInt32HashAgg) Flush(outputIdx int) { func (a *sumIntInt32HashAgg) Reset() { a.curAgg = zeroInt64Value - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumIntInt32HashAggAlloc struct { @@ -282,9 +282,9 @@ type sumIntInt64HashAgg struct { curAgg int64 // col points to the output vector we are updating. col []int64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &sumIntInt64HashAgg{} @@ -319,7 +319,7 @@ func (a *sumIntInt64HashAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -338,7 +338,7 @@ func (a *sumIntInt64HashAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -355,7 +355,7 @@ func (a *sumIntInt64HashAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curAgg @@ -364,7 +364,7 @@ func (a *sumIntInt64HashAgg) Flush(outputIdx int) { func (a *sumIntInt64HashAgg) Reset() { a.curAgg = zeroInt64Value - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumIntInt64HashAggAlloc struct { diff --git a/pkg/sql/colexec/colexecagg/min_max_agg_tmpl.go b/pkg/sql/colexec/colexecagg/min_max_agg_tmpl.go index 23919be3330b..749febcaca1d 100644 --- a/pkg/sql/colexec/colexecagg/min_max_agg_tmpl.go +++ b/pkg/sql/colexec/colexecagg/min_max_agg_tmpl.go @@ -97,11 +97,11 @@ type _AGG_TYPE_AGGKINDAgg struct { col _GOTYPESLICE // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg _GOTYPE - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &_AGG_TYPE_AGGKINDAgg{} @@ -192,14 +192,20 @@ func (a *_AGG_TYPE_AGGKINDAgg) Flush(outputIdx int) { outputIdx = a.curIdx a.curIdx++ // {{end}} - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // {{if eq "_AGGKIND" "Window"}} + // We need to copy the value because window functions reuse the aggregation + // between rows. + execgen.COPYVAL(a.curAgg, a.curAgg) + // {{end}} a.col.Set(outputIdx, a.curAgg) } - // {{if or (.IsBytesLike) (eq .VecMethod "Datum")}} + // {{if and (not (eq "_AGGKIND" "Window")) (or (.IsBytesLike) (eq .VecMethod "Datum"))}} execgen.SETVARIABLESIZE(oldCurAggSize, a.curAgg) - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil // {{end}} @@ -209,7 +215,13 @@ func (a *_AGG_TYPE_AGGKINDAgg) Reset() { // {{if eq "_AGGKIND" "Ordered"}} a.orderedAggregateFuncBase.Reset() // {{end}} - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + // {{if or (.IsBytesLike) (eq .VecMethod "Datum")}} + execgen.SETVARIABLESIZE(oldCurAggSize, a.curAgg) + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil + // {{end}} } type _AGG_TYPE_AGGKINDAggAlloc struct { @@ -233,6 +245,17 @@ func (a *_AGG_TYPE_AGGKINDAggAlloc) newAggFunc() AggregateFunc { return f } +// {{if eq "_AGGKIND" "Window"}} + +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*_AGG_TYPE_AGGKINDAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on _AGG_TYPE_AGGKINDAgg")) +} + +// {{end}} // {{end}} // {{end}} // {{end}} @@ -255,13 +278,13 @@ func _ACCUMULATE_MINMAX( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -274,7 +297,7 @@ func _ACCUMULATE_MINMAX( isNull = false // {{end}} if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { // {{if and (.Sliceable) (not .HasSel)}} //gcassert:bce // {{end}} @@ -282,7 +305,6 @@ func _ACCUMULATE_MINMAX( // {{with .Global}} execgen.COPYVAL(a.curAgg, val) // {{end}} - a.foundNonNullForCurrentGroup = true } else { var cmp bool // {{if and (.Sliceable) (not .HasSel)}} @@ -296,6 +318,7 @@ func _ACCUMULATE_MINMAX( } // {{end}} } + a.numNonNull++ } // {{end}} diff --git a/pkg/sql/colexec/colexecagg/ordered_avg_agg.eg.go b/pkg/sql/colexec/colexecagg/ordered_avg_agg.eg.go index 61eaa6196bce..c5645b9b437c 100644 --- a/pkg/sql/colexec/colexecagg/ordered_avg_agg.eg.go +++ b/pkg/sql/colexec/colexecagg/ordered_avg_agg.eg.go @@ -74,15 +74,12 @@ type avgInt16OrderedAgg struct { // so we can index into the slice once per group, instead of on each // iteration. curSum apd.Decimal - // curCount keeps track of the number of elements that we've seen + // curCount keeps track of the number of non-null elements that we've seen // belonging to the current group. curCount int64 // col points to the statically-typed output vector. - col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool - overloadHelper execgen.OverloadHelper + col []apd.Decimal + overloadHelper execgen.OverloadHelper } var _ AggregateFunc = &avgInt16OrderedAgg{} @@ -117,7 +114,7 @@ func (a *avgInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { @@ -129,8 +126,6 @@ func (a *avgInt16OrderedAgg) Compute( a.curIdx++ a.curSum = zeroDecimalValue a.curCount = 0 - - a.foundNonNullForCurrentGroup = false } a.isFirstGroup = false } @@ -151,7 +146,6 @@ func (a *avgInt16OrderedAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -162,7 +156,7 @@ func (a *avgInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { @@ -174,7 +168,6 @@ func (a *avgInt16OrderedAgg) Compute( a.curIdx++ a.curSum = zeroDecimalValue a.curCount = 0 - } a.isFirstGroup = false } @@ -195,7 +188,6 @@ func (a *avgInt16OrderedAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -208,7 +200,7 @@ func (a *avgInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { @@ -220,8 +212,6 @@ func (a *avgInt16OrderedAgg) Compute( a.curIdx++ a.curSum = zeroDecimalValue a.curCount = 0 - - a.foundNonNullForCurrentGroup = false } a.isFirstGroup = false } @@ -241,7 +231,6 @@ func (a *avgInt16OrderedAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -251,7 +240,7 @@ func (a *avgInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { @@ -263,7 +252,6 @@ func (a *avgInt16OrderedAgg) Compute( a.curIdx++ a.curSum = zeroDecimalValue a.curCount = 0 - } a.isFirstGroup = false } @@ -283,7 +271,6 @@ func (a *avgInt16OrderedAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -304,7 +291,7 @@ func (a *avgInt16OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(outputIdx) } else { @@ -319,7 +306,6 @@ func (a *avgInt16OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() a.curSum = zeroDecimalValue a.curCount = 0 - a.foundNonNullForCurrentGroup = false } type avgInt16OrderedAggAlloc struct { @@ -349,15 +335,12 @@ type avgInt32OrderedAgg struct { // so we can index into the slice once per group, instead of on each // iteration. curSum apd.Decimal - // curCount keeps track of the number of elements that we've seen + // curCount keeps track of the number of non-null elements that we've seen // belonging to the current group. curCount int64 // col points to the statically-typed output vector. - col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool - overloadHelper execgen.OverloadHelper + col []apd.Decimal + overloadHelper execgen.OverloadHelper } var _ AggregateFunc = &avgInt32OrderedAgg{} @@ -392,7 +375,7 @@ func (a *avgInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { @@ -404,8 +387,6 @@ func (a *avgInt32OrderedAgg) Compute( a.curIdx++ a.curSum = zeroDecimalValue a.curCount = 0 - - a.foundNonNullForCurrentGroup = false } a.isFirstGroup = false } @@ -426,7 +407,6 @@ func (a *avgInt32OrderedAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -437,7 +417,7 @@ func (a *avgInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { @@ -449,7 +429,6 @@ func (a *avgInt32OrderedAgg) Compute( a.curIdx++ a.curSum = zeroDecimalValue a.curCount = 0 - } a.isFirstGroup = false } @@ -470,7 +449,6 @@ func (a *avgInt32OrderedAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -483,7 +461,7 @@ func (a *avgInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { @@ -495,8 +473,6 @@ func (a *avgInt32OrderedAgg) Compute( a.curIdx++ a.curSum = zeroDecimalValue a.curCount = 0 - - a.foundNonNullForCurrentGroup = false } a.isFirstGroup = false } @@ -516,7 +492,6 @@ func (a *avgInt32OrderedAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -526,7 +501,7 @@ func (a *avgInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { @@ -538,7 +513,6 @@ func (a *avgInt32OrderedAgg) Compute( a.curIdx++ a.curSum = zeroDecimalValue a.curCount = 0 - } a.isFirstGroup = false } @@ -558,7 +532,6 @@ func (a *avgInt32OrderedAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -579,7 +552,7 @@ func (a *avgInt32OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(outputIdx) } else { @@ -594,7 +567,6 @@ func (a *avgInt32OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() a.curSum = zeroDecimalValue a.curCount = 0 - a.foundNonNullForCurrentGroup = false } type avgInt32OrderedAggAlloc struct { @@ -624,15 +596,12 @@ type avgInt64OrderedAgg struct { // so we can index into the slice once per group, instead of on each // iteration. curSum apd.Decimal - // curCount keeps track of the number of elements that we've seen + // curCount keeps track of the number of non-null elements that we've seen // belonging to the current group. curCount int64 // col points to the statically-typed output vector. - col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool - overloadHelper execgen.OverloadHelper + col []apd.Decimal + overloadHelper execgen.OverloadHelper } var _ AggregateFunc = &avgInt64OrderedAgg{} @@ -667,7 +636,7 @@ func (a *avgInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { @@ -679,8 +648,6 @@ func (a *avgInt64OrderedAgg) Compute( a.curIdx++ a.curSum = zeroDecimalValue a.curCount = 0 - - a.foundNonNullForCurrentGroup = false } a.isFirstGroup = false } @@ -701,7 +668,6 @@ func (a *avgInt64OrderedAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -712,7 +678,7 @@ func (a *avgInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { @@ -724,7 +690,6 @@ func (a *avgInt64OrderedAgg) Compute( a.curIdx++ a.curSum = zeroDecimalValue a.curCount = 0 - } a.isFirstGroup = false } @@ -745,7 +710,6 @@ func (a *avgInt64OrderedAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -758,7 +722,7 @@ func (a *avgInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { @@ -770,8 +734,6 @@ func (a *avgInt64OrderedAgg) Compute( a.curIdx++ a.curSum = zeroDecimalValue a.curCount = 0 - - a.foundNonNullForCurrentGroup = false } a.isFirstGroup = false } @@ -791,7 +753,6 @@ func (a *avgInt64OrderedAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -801,7 +762,7 @@ func (a *avgInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { @@ -813,7 +774,6 @@ func (a *avgInt64OrderedAgg) Compute( a.curIdx++ a.curSum = zeroDecimalValue a.curCount = 0 - } a.isFirstGroup = false } @@ -833,7 +793,6 @@ func (a *avgInt64OrderedAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -854,7 +813,7 @@ func (a *avgInt64OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(outputIdx) } else { @@ -869,7 +828,6 @@ func (a *avgInt64OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() a.curSum = zeroDecimalValue a.curCount = 0 - a.foundNonNullForCurrentGroup = false } type avgInt64OrderedAggAlloc struct { @@ -899,14 +857,11 @@ type avgDecimalOrderedAgg struct { // so we can index into the slice once per group, instead of on each // iteration. curSum apd.Decimal - // curCount keeps track of the number of elements that we've seen + // curCount keeps track of the number of non-null elements that we've seen // belonging to the current group. curCount int64 // col points to the statically-typed output vector. col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool } var _ AggregateFunc = &avgDecimalOrderedAgg{} @@ -938,7 +893,7 @@ func (a *avgDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { @@ -950,8 +905,6 @@ func (a *avgDecimalOrderedAgg) Compute( a.curIdx++ a.curSum = zeroDecimalValue a.curCount = 0 - - a.foundNonNullForCurrentGroup = false } a.isFirstGroup = false } @@ -971,7 +924,6 @@ func (a *avgDecimalOrderedAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -982,7 +934,7 @@ func (a *avgDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { @@ -994,7 +946,6 @@ func (a *avgDecimalOrderedAgg) Compute( a.curIdx++ a.curSum = zeroDecimalValue a.curCount = 0 - } a.isFirstGroup = false } @@ -1014,7 +965,6 @@ func (a *avgDecimalOrderedAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -1027,7 +977,7 @@ func (a *avgDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { @@ -1039,8 +989,6 @@ func (a *avgDecimalOrderedAgg) Compute( a.curIdx++ a.curSum = zeroDecimalValue a.curCount = 0 - - a.foundNonNullForCurrentGroup = false } a.isFirstGroup = false } @@ -1059,7 +1007,6 @@ func (a *avgDecimalOrderedAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -1069,7 +1016,7 @@ func (a *avgDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { @@ -1081,7 +1028,6 @@ func (a *avgDecimalOrderedAgg) Compute( a.curIdx++ a.curSum = zeroDecimalValue a.curCount = 0 - } a.isFirstGroup = false } @@ -1100,7 +1046,6 @@ func (a *avgDecimalOrderedAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -1121,7 +1066,7 @@ func (a *avgDecimalOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(outputIdx) } else { @@ -1136,7 +1081,6 @@ func (a *avgDecimalOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() a.curSum = zeroDecimalValue a.curCount = 0 - a.foundNonNullForCurrentGroup = false } type avgDecimalOrderedAggAlloc struct { @@ -1166,14 +1110,11 @@ type avgFloat64OrderedAgg struct { // so we can index into the slice once per group, instead of on each // iteration. curSum float64 - // curCount keeps track of the number of elements that we've seen + // curCount keeps track of the number of non-null elements that we've seen // belonging to the current group. curCount int64 // col points to the statically-typed output vector. col []float64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool } var _ AggregateFunc = &avgFloat64OrderedAgg{} @@ -1205,7 +1146,7 @@ func (a *avgFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curSum / float64(a.curCount) @@ -1213,8 +1154,6 @@ func (a *avgFloat64OrderedAgg) Compute( a.curIdx++ a.curSum = zeroFloat64Value a.curCount = 0 - - a.foundNonNullForCurrentGroup = false } a.isFirstGroup = false } @@ -1231,7 +1170,6 @@ func (a *avgFloat64OrderedAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -1242,7 +1180,7 @@ func (a *avgFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curSum / float64(a.curCount) @@ -1250,7 +1188,6 @@ func (a *avgFloat64OrderedAgg) Compute( a.curIdx++ a.curSum = zeroFloat64Value a.curCount = 0 - } a.isFirstGroup = false } @@ -1267,7 +1204,6 @@ func (a *avgFloat64OrderedAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -1280,7 +1216,7 @@ func (a *avgFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curSum / float64(a.curCount) @@ -1288,8 +1224,6 @@ func (a *avgFloat64OrderedAgg) Compute( a.curIdx++ a.curSum = zeroFloat64Value a.curCount = 0 - - a.foundNonNullForCurrentGroup = false } a.isFirstGroup = false } @@ -1305,7 +1239,6 @@ func (a *avgFloat64OrderedAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -1315,7 +1248,7 @@ func (a *avgFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curSum / float64(a.curCount) @@ -1323,7 +1256,6 @@ func (a *avgFloat64OrderedAgg) Compute( a.curIdx++ a.curSum = zeroFloat64Value a.curCount = 0 - } a.isFirstGroup = false } @@ -1339,7 +1271,6 @@ func (a *avgFloat64OrderedAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -1360,7 +1291,7 @@ func (a *avgFloat64OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curSum / float64(a.curCount) @@ -1371,7 +1302,6 @@ func (a *avgFloat64OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() a.curSum = zeroFloat64Value a.curCount = 0 - a.foundNonNullForCurrentGroup = false } type avgFloat64OrderedAggAlloc struct { @@ -1401,14 +1331,11 @@ type avgIntervalOrderedAgg struct { // so we can index into the slice once per group, instead of on each // iteration. curSum duration.Duration - // curCount keeps track of the number of elements that we've seen + // curCount keeps track of the number of non-null elements that we've seen // belonging to the current group. curCount int64 // col points to the statically-typed output vector. col []duration.Duration - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool } var _ AggregateFunc = &avgIntervalOrderedAgg{} @@ -1440,7 +1367,7 @@ func (a *avgIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curSum.Div(int64(a.curCount)) @@ -1448,8 +1375,6 @@ func (a *avgIntervalOrderedAgg) Compute( a.curIdx++ a.curSum = zeroIntervalValue a.curCount = 0 - - a.foundNonNullForCurrentGroup = false } a.isFirstGroup = false } @@ -1461,7 +1386,6 @@ func (a *avgIntervalOrderedAgg) Compute( v := col.Get(i) a.curSum = a.curSum.Add(v) a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -1472,7 +1396,7 @@ func (a *avgIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curSum.Div(int64(a.curCount)) @@ -1480,7 +1404,6 @@ func (a *avgIntervalOrderedAgg) Compute( a.curIdx++ a.curSum = zeroIntervalValue a.curCount = 0 - } a.isFirstGroup = false } @@ -1492,7 +1415,6 @@ func (a *avgIntervalOrderedAgg) Compute( v := col.Get(i) a.curSum = a.curSum.Add(v) a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -1505,7 +1427,7 @@ func (a *avgIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curSum.Div(int64(a.curCount)) @@ -1513,8 +1435,6 @@ func (a *avgIntervalOrderedAgg) Compute( a.curIdx++ a.curSum = zeroIntervalValue a.curCount = 0 - - a.foundNonNullForCurrentGroup = false } a.isFirstGroup = false } @@ -1525,7 +1445,6 @@ func (a *avgIntervalOrderedAgg) Compute( v := col.Get(i) a.curSum = a.curSum.Add(v) a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -1535,7 +1454,7 @@ func (a *avgIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curSum.Div(int64(a.curCount)) @@ -1543,7 +1462,6 @@ func (a *avgIntervalOrderedAgg) Compute( a.curIdx++ a.curSum = zeroIntervalValue a.curCount = 0 - } a.isFirstGroup = false } @@ -1554,7 +1472,6 @@ func (a *avgIntervalOrderedAgg) Compute( v := col.Get(i) a.curSum = a.curSum.Add(v) a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -1575,7 +1492,7 @@ func (a *avgIntervalOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curSum.Div(int64(a.curCount)) @@ -1586,7 +1503,6 @@ func (a *avgIntervalOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() a.curSum = zeroIntervalValue a.curCount = 0 - a.foundNonNullForCurrentGroup = false } type avgIntervalOrderedAggAlloc struct { diff --git a/pkg/sql/colexec/colexecagg/ordered_count_agg.eg.go b/pkg/sql/colexec/colexecagg/ordered_count_agg.eg.go index c8f7d3377f6f..b8afebb20536 100644 --- a/pkg/sql/colexec/colexecagg/ordered_count_agg.eg.go +++ b/pkg/sql/colexec/colexecagg/ordered_count_agg.eg.go @@ -42,7 +42,6 @@ func (a *countRowsOrderedAgg) SetOutput(vec coldata.Vec) { func (a *countRowsOrderedAgg) Compute( vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int, sel []int, ) { - var oldCurAggSize uintptr a.allocator.PerformOperation([]coldata.Vec{a.vec}, func() { // Capture groups to force bounds check to work. See // https://github.com/golang/go/issues/39756 @@ -86,10 +85,6 @@ func (a *countRowsOrderedAgg) Compute( } }, ) - var newCurAggSize uintptr - if newCurAggSize != oldCurAggSize { - a.allocator.AdjustMemoryUsage(int64(newCurAggSize - oldCurAggSize)) - } } func (a *countRowsOrderedAgg) Flush(outputIdx int) { @@ -158,7 +153,6 @@ func (a *countOrderedAgg) SetOutput(vec coldata.Vec) { func (a *countOrderedAgg) Compute( vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int, sel []int, ) { - var oldCurAggSize uintptr // If this is a COUNT(col) aggregator and there are nulls in this batch, // we must check each value for nullity. Note that it is only legal to do a // COUNT aggregate on a single column. @@ -243,10 +237,6 @@ func (a *countOrderedAgg) Compute( } }, ) - var newCurAggSize uintptr - if newCurAggSize != oldCurAggSize { - a.allocator.AdjustMemoryUsage(int64(newCurAggSize - oldCurAggSize)) - } } func (a *countOrderedAgg) Flush(outputIdx int) { diff --git a/pkg/sql/colexec/colexecagg/ordered_min_max_agg.eg.go b/pkg/sql/colexec/colexecagg/ordered_min_max_agg.eg.go index 5de5ff46bd41..2e8cec106e34 100644 --- a/pkg/sql/colexec/colexecagg/ordered_min_max_agg.eg.go +++ b/pkg/sql/colexec/colexecagg/ordered_min_max_agg.eg.go @@ -116,11 +116,11 @@ type minBoolOrderedAgg struct { col coldata.Bools // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg bool - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minBoolOrderedAgg{} @@ -152,13 +152,13 @@ func (a *minBoolOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -166,10 +166,9 @@ func (a *minBoolOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -192,6 +191,7 @@ func (a *minBoolOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -202,13 +202,13 @@ func (a *minBoolOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -216,10 +216,9 @@ func (a *minBoolOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -242,6 +241,7 @@ func (a *minBoolOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -254,13 +254,13 @@ func (a *minBoolOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -268,10 +268,9 @@ func (a *minBoolOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -294,6 +293,7 @@ func (a *minBoolOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -303,13 +303,13 @@ func (a *minBoolOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -317,10 +317,9 @@ func (a *minBoolOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -343,6 +342,7 @@ func (a *minBoolOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -363,7 +363,7 @@ func (a *minBoolOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -372,7 +372,7 @@ func (a *minBoolOrderedAgg) Flush(outputIdx int) { func (a *minBoolOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minBoolOrderedAggAlloc struct { @@ -402,11 +402,11 @@ type minBytesOrderedAgg struct { col *coldata.Bytes // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg []byte - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minBytesOrderedAgg{} @@ -438,13 +438,13 @@ func (a *minBytesOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -452,10 +452,9 @@ func (a *minBytesOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -470,6 +469,7 @@ func (a *minBytesOrderedAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } else { @@ -480,13 +480,13 @@ func (a *minBytesOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -494,10 +494,9 @@ func (a *minBytesOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -512,6 +511,7 @@ func (a *minBytesOrderedAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } @@ -524,13 +524,13 @@ func (a *minBytesOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -538,10 +538,9 @@ func (a *minBytesOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -556,6 +555,7 @@ func (a *minBytesOrderedAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } else { @@ -565,13 +565,13 @@ func (a *minBytesOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -579,10 +579,9 @@ func (a *minBytesOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -597,6 +596,7 @@ func (a *minBytesOrderedAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } @@ -617,20 +617,25 @@ func (a *minBytesOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) } oldCurAggSize := len(a.curAgg) - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *minBytesOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + oldCurAggSize := len(a.curAgg) + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type minBytesOrderedAggAlloc struct { @@ -660,11 +665,11 @@ type minDecimalOrderedAgg struct { col coldata.Decimals // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minDecimalOrderedAgg{} @@ -696,13 +701,13 @@ func (a *minDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -710,10 +715,9 @@ func (a *minDecimalOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -728,6 +732,7 @@ func (a *minDecimalOrderedAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } else { @@ -738,13 +743,13 @@ func (a *minDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -752,10 +757,9 @@ func (a *minDecimalOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -770,6 +774,7 @@ func (a *minDecimalOrderedAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } @@ -782,13 +787,13 @@ func (a *minDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -796,10 +801,9 @@ func (a *minDecimalOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -814,6 +818,7 @@ func (a *minDecimalOrderedAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } else { @@ -823,13 +828,13 @@ func (a *minDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -837,10 +842,9 @@ func (a *minDecimalOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -855,6 +859,7 @@ func (a *minDecimalOrderedAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } @@ -875,7 +880,7 @@ func (a *minDecimalOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -884,7 +889,7 @@ func (a *minDecimalOrderedAgg) Flush(outputIdx int) { func (a *minDecimalOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minDecimalOrderedAggAlloc struct { @@ -914,11 +919,11 @@ type minInt16OrderedAgg struct { col coldata.Int16s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int16 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minInt16OrderedAgg{} @@ -950,13 +955,13 @@ func (a *minInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -964,10 +969,9 @@ func (a *minInt16OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -993,6 +997,7 @@ func (a *minInt16OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1003,13 +1008,13 @@ func (a *minInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1017,10 +1022,9 @@ func (a *minInt16OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1046,6 +1050,7 @@ func (a *minInt16OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1058,13 +1063,13 @@ func (a *minInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1072,10 +1077,9 @@ func (a *minInt16OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1101,6 +1105,7 @@ func (a *minInt16OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1110,13 +1115,13 @@ func (a *minInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1124,10 +1129,9 @@ func (a *minInt16OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1153,6 +1157,7 @@ func (a *minInt16OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1173,7 +1178,7 @@ func (a *minInt16OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -1182,7 +1187,7 @@ func (a *minInt16OrderedAgg) Flush(outputIdx int) { func (a *minInt16OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minInt16OrderedAggAlloc struct { @@ -1212,11 +1217,11 @@ type minInt32OrderedAgg struct { col coldata.Int32s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int32 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minInt32OrderedAgg{} @@ -1248,13 +1253,13 @@ func (a *minInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1262,10 +1267,9 @@ func (a *minInt32OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1291,6 +1295,7 @@ func (a *minInt32OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1301,13 +1306,13 @@ func (a *minInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1315,10 +1320,9 @@ func (a *minInt32OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1344,6 +1348,7 @@ func (a *minInt32OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1356,13 +1361,13 @@ func (a *minInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1370,10 +1375,9 @@ func (a *minInt32OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1399,6 +1403,7 @@ func (a *minInt32OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1408,13 +1413,13 @@ func (a *minInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1422,10 +1427,9 @@ func (a *minInt32OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1451,6 +1455,7 @@ func (a *minInt32OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1471,7 +1476,7 @@ func (a *minInt32OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -1480,7 +1485,7 @@ func (a *minInt32OrderedAgg) Flush(outputIdx int) { func (a *minInt32OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minInt32OrderedAggAlloc struct { @@ -1510,11 +1515,11 @@ type minInt64OrderedAgg struct { col coldata.Int64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minInt64OrderedAgg{} @@ -1546,13 +1551,13 @@ func (a *minInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1560,10 +1565,9 @@ func (a *minInt64OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1589,6 +1593,7 @@ func (a *minInt64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1599,13 +1604,13 @@ func (a *minInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1613,10 +1618,9 @@ func (a *minInt64OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1642,6 +1646,7 @@ func (a *minInt64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1654,13 +1659,13 @@ func (a *minInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1668,10 +1673,9 @@ func (a *minInt64OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1697,6 +1701,7 @@ func (a *minInt64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1706,13 +1711,13 @@ func (a *minInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1720,10 +1725,9 @@ func (a *minInt64OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1749,6 +1753,7 @@ func (a *minInt64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1769,7 +1774,7 @@ func (a *minInt64OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -1778,7 +1783,7 @@ func (a *minInt64OrderedAgg) Flush(outputIdx int) { func (a *minInt64OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minInt64OrderedAggAlloc struct { @@ -1808,11 +1813,11 @@ type minFloat64OrderedAgg struct { col coldata.Float64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg float64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minFloat64OrderedAgg{} @@ -1844,13 +1849,13 @@ func (a *minFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1858,10 +1863,9 @@ func (a *minFloat64OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1895,6 +1899,7 @@ func (a *minFloat64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1905,13 +1910,13 @@ func (a *minFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1919,10 +1924,9 @@ func (a *minFloat64OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1956,6 +1960,7 @@ func (a *minFloat64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1968,13 +1973,13 @@ func (a *minFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1982,10 +1987,9 @@ func (a *minFloat64OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2019,6 +2023,7 @@ func (a *minFloat64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2028,13 +2033,13 @@ func (a *minFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2042,10 +2047,9 @@ func (a *minFloat64OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2079,6 +2083,7 @@ func (a *minFloat64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2099,7 +2104,7 @@ func (a *minFloat64OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -2108,7 +2113,7 @@ func (a *minFloat64OrderedAgg) Flush(outputIdx int) { func (a *minFloat64OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minFloat64OrderedAggAlloc struct { @@ -2138,11 +2143,11 @@ type minTimestampOrderedAgg struct { col coldata.Times // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg time.Time - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minTimestampOrderedAgg{} @@ -2174,13 +2179,13 @@ func (a *minTimestampOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2188,10 +2193,9 @@ func (a *minTimestampOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2213,6 +2217,7 @@ func (a *minTimestampOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2223,13 +2228,13 @@ func (a *minTimestampOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2237,10 +2242,9 @@ func (a *minTimestampOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2262,6 +2266,7 @@ func (a *minTimestampOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2274,13 +2279,13 @@ func (a *minTimestampOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2288,10 +2293,9 @@ func (a *minTimestampOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2313,6 +2317,7 @@ func (a *minTimestampOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2322,13 +2327,13 @@ func (a *minTimestampOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2336,10 +2341,9 @@ func (a *minTimestampOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2361,6 +2365,7 @@ func (a *minTimestampOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2381,7 +2386,7 @@ func (a *minTimestampOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -2390,7 +2395,7 @@ func (a *minTimestampOrderedAgg) Flush(outputIdx int) { func (a *minTimestampOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minTimestampOrderedAggAlloc struct { @@ -2420,11 +2425,11 @@ type minIntervalOrderedAgg struct { col coldata.Durations // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg duration.Duration - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minIntervalOrderedAgg{} @@ -2456,13 +2461,13 @@ func (a *minIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2470,10 +2475,9 @@ func (a *minIntervalOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2488,6 +2492,7 @@ func (a *minIntervalOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2498,13 +2503,13 @@ func (a *minIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2512,10 +2517,9 @@ func (a *minIntervalOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2530,6 +2534,7 @@ func (a *minIntervalOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2542,13 +2547,13 @@ func (a *minIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2556,10 +2561,9 @@ func (a *minIntervalOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2574,6 +2578,7 @@ func (a *minIntervalOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2583,13 +2588,13 @@ func (a *minIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2597,10 +2602,9 @@ func (a *minIntervalOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2615,6 +2619,7 @@ func (a *minIntervalOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2635,7 +2640,7 @@ func (a *minIntervalOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -2644,7 +2649,7 @@ func (a *minIntervalOrderedAgg) Flush(outputIdx int) { func (a *minIntervalOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minIntervalOrderedAggAlloc struct { @@ -2674,11 +2679,11 @@ type minJSONOrderedAgg struct { col *coldata.JSONs // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg json.JSON - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minJSONOrderedAgg{} @@ -2713,13 +2718,13 @@ func (a *minJSONOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2727,7 +2732,7 @@ func (a *minJSONOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -2741,7 +2746,6 @@ func (a *minJSONOrderedAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2773,6 +2777,7 @@ func (a *minJSONOrderedAgg) Compute( } } + a.numNonNull++ } } } else { @@ -2783,13 +2788,13 @@ func (a *minJSONOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2797,7 +2802,7 @@ func (a *minJSONOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -2811,7 +2816,6 @@ func (a *minJSONOrderedAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2843,6 +2847,7 @@ func (a *minJSONOrderedAgg) Compute( } } + a.numNonNull++ } } } @@ -2855,13 +2860,13 @@ func (a *minJSONOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2869,7 +2874,7 @@ func (a *minJSONOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -2883,7 +2888,6 @@ func (a *minJSONOrderedAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2915,6 +2919,7 @@ func (a *minJSONOrderedAgg) Compute( } } + a.numNonNull++ } } } else { @@ -2924,13 +2929,13 @@ func (a *minJSONOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -2938,7 +2943,7 @@ func (a *minJSONOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -2952,7 +2957,6 @@ func (a *minJSONOrderedAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2984,6 +2988,7 @@ func (a *minJSONOrderedAgg) Compute( } } + a.numNonNull++ } } } @@ -3007,7 +3012,7 @@ func (a *minJSONOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -3016,14 +3021,22 @@ func (a *minJSONOrderedAgg) Flush(outputIdx int) { if a.curAgg != nil { oldCurAggSize = a.curAgg.Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *minJSONOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + var oldCurAggSize uintptr + if a.curAgg != nil { + oldCurAggSize = a.curAgg.Size() + } + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type minJSONOrderedAggAlloc struct { @@ -3053,11 +3066,11 @@ type minDatumOrderedAgg struct { col coldata.DatumVec // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg interface{} - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minDatumOrderedAgg{} @@ -3093,13 +3106,13 @@ func (a *minDatumOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3107,10 +3120,9 @@ func (a *minDatumOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3127,6 +3139,7 @@ func (a *minDatumOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -3137,13 +3150,13 @@ func (a *minDatumOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3151,10 +3164,9 @@ func (a *minDatumOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3171,6 +3183,7 @@ func (a *minDatumOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -3183,13 +3196,13 @@ func (a *minDatumOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3197,10 +3210,9 @@ func (a *minDatumOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3217,6 +3229,7 @@ func (a *minDatumOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -3226,13 +3239,13 @@ func (a *minDatumOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3240,10 +3253,9 @@ func (a *minDatumOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3260,6 +3272,7 @@ func (a *minDatumOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -3284,7 +3297,7 @@ func (a *minDatumOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -3294,14 +3307,23 @@ func (a *minDatumOrderedAgg) Flush(outputIdx int) { if a.curAgg != nil { oldCurAggSize = a.curAgg.(tree.Datum).Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *minDatumOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + + var oldCurAggSize uintptr + if a.curAgg != nil { + oldCurAggSize = a.curAgg.(tree.Datum).Size() + } + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type minDatumOrderedAggAlloc struct { @@ -3400,11 +3422,11 @@ type maxBoolOrderedAgg struct { col coldata.Bools // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg bool - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxBoolOrderedAgg{} @@ -3436,13 +3458,13 @@ func (a *maxBoolOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3450,10 +3472,9 @@ func (a *maxBoolOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3476,6 +3497,7 @@ func (a *maxBoolOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -3486,13 +3508,13 @@ func (a *maxBoolOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3500,10 +3522,9 @@ func (a *maxBoolOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3526,6 +3547,7 @@ func (a *maxBoolOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -3538,13 +3560,13 @@ func (a *maxBoolOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3552,10 +3574,9 @@ func (a *maxBoolOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3578,6 +3599,7 @@ func (a *maxBoolOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -3587,13 +3609,13 @@ func (a *maxBoolOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3601,10 +3623,9 @@ func (a *maxBoolOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3627,6 +3648,7 @@ func (a *maxBoolOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -3647,7 +3669,7 @@ func (a *maxBoolOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -3656,7 +3678,7 @@ func (a *maxBoolOrderedAgg) Flush(outputIdx int) { func (a *maxBoolOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxBoolOrderedAggAlloc struct { @@ -3686,11 +3708,11 @@ type maxBytesOrderedAgg struct { col *coldata.Bytes // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg []byte - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxBytesOrderedAgg{} @@ -3722,13 +3744,13 @@ func (a *maxBytesOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3736,10 +3758,9 @@ func (a *maxBytesOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3754,6 +3775,7 @@ func (a *maxBytesOrderedAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } else { @@ -3764,13 +3786,13 @@ func (a *maxBytesOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3778,10 +3800,9 @@ func (a *maxBytesOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3796,6 +3817,7 @@ func (a *maxBytesOrderedAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } @@ -3808,13 +3830,13 @@ func (a *maxBytesOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3822,10 +3844,9 @@ func (a *maxBytesOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3840,6 +3861,7 @@ func (a *maxBytesOrderedAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } else { @@ -3849,13 +3871,13 @@ func (a *maxBytesOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3863,10 +3885,9 @@ func (a *maxBytesOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3881,6 +3902,7 @@ func (a *maxBytesOrderedAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } @@ -3901,20 +3923,25 @@ func (a *maxBytesOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) } oldCurAggSize := len(a.curAgg) - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *maxBytesOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + oldCurAggSize := len(a.curAgg) + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type maxBytesOrderedAggAlloc struct { @@ -3944,11 +3971,11 @@ type maxDecimalOrderedAgg struct { col coldata.Decimals // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxDecimalOrderedAgg{} @@ -3980,13 +4007,13 @@ func (a *maxDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -3994,10 +4021,9 @@ func (a *maxDecimalOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4012,6 +4038,7 @@ func (a *maxDecimalOrderedAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } else { @@ -4022,13 +4049,13 @@ func (a *maxDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4036,10 +4063,9 @@ func (a *maxDecimalOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4054,6 +4080,7 @@ func (a *maxDecimalOrderedAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } @@ -4066,13 +4093,13 @@ func (a *maxDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4080,10 +4107,9 @@ func (a *maxDecimalOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4098,6 +4124,7 @@ func (a *maxDecimalOrderedAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } else { @@ -4107,13 +4134,13 @@ func (a *maxDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4121,10 +4148,9 @@ func (a *maxDecimalOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4139,6 +4165,7 @@ func (a *maxDecimalOrderedAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } @@ -4159,7 +4186,7 @@ func (a *maxDecimalOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -4168,7 +4195,7 @@ func (a *maxDecimalOrderedAgg) Flush(outputIdx int) { func (a *maxDecimalOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxDecimalOrderedAggAlloc struct { @@ -4198,11 +4225,11 @@ type maxInt16OrderedAgg struct { col coldata.Int16s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int16 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxInt16OrderedAgg{} @@ -4234,13 +4261,13 @@ func (a *maxInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4248,10 +4275,9 @@ func (a *maxInt16OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4277,6 +4303,7 @@ func (a *maxInt16OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -4287,13 +4314,13 @@ func (a *maxInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4301,10 +4328,9 @@ func (a *maxInt16OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4330,6 +4356,7 @@ func (a *maxInt16OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -4342,13 +4369,13 @@ func (a *maxInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4356,10 +4383,9 @@ func (a *maxInt16OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4385,6 +4411,7 @@ func (a *maxInt16OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -4394,13 +4421,13 @@ func (a *maxInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4408,10 +4435,9 @@ func (a *maxInt16OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4437,6 +4463,7 @@ func (a *maxInt16OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -4457,7 +4484,7 @@ func (a *maxInt16OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -4466,7 +4493,7 @@ func (a *maxInt16OrderedAgg) Flush(outputIdx int) { func (a *maxInt16OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxInt16OrderedAggAlloc struct { @@ -4496,11 +4523,11 @@ type maxInt32OrderedAgg struct { col coldata.Int32s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int32 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxInt32OrderedAgg{} @@ -4532,13 +4559,13 @@ func (a *maxInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4546,10 +4573,9 @@ func (a *maxInt32OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4575,6 +4601,7 @@ func (a *maxInt32OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -4585,13 +4612,13 @@ func (a *maxInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4599,10 +4626,9 @@ func (a *maxInt32OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4628,6 +4654,7 @@ func (a *maxInt32OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -4640,13 +4667,13 @@ func (a *maxInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4654,10 +4681,9 @@ func (a *maxInt32OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4683,6 +4709,7 @@ func (a *maxInt32OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -4692,13 +4719,13 @@ func (a *maxInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4706,10 +4733,9 @@ func (a *maxInt32OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4735,6 +4761,7 @@ func (a *maxInt32OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -4755,7 +4782,7 @@ func (a *maxInt32OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -4764,7 +4791,7 @@ func (a *maxInt32OrderedAgg) Flush(outputIdx int) { func (a *maxInt32OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxInt32OrderedAggAlloc struct { @@ -4794,11 +4821,11 @@ type maxInt64OrderedAgg struct { col coldata.Int64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxInt64OrderedAgg{} @@ -4830,13 +4857,13 @@ func (a *maxInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4844,10 +4871,9 @@ func (a *maxInt64OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4873,6 +4899,7 @@ func (a *maxInt64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -4883,13 +4910,13 @@ func (a *maxInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4897,10 +4924,9 @@ func (a *maxInt64OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4926,6 +4952,7 @@ func (a *maxInt64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -4938,13 +4965,13 @@ func (a *maxInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -4952,10 +4979,9 @@ func (a *maxInt64OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -4981,6 +5007,7 @@ func (a *maxInt64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -4990,13 +5017,13 @@ func (a *maxInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5004,10 +5031,9 @@ func (a *maxInt64OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5033,6 +5059,7 @@ func (a *maxInt64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -5053,7 +5080,7 @@ func (a *maxInt64OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -5062,7 +5089,7 @@ func (a *maxInt64OrderedAgg) Flush(outputIdx int) { func (a *maxInt64OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxInt64OrderedAggAlloc struct { @@ -5092,11 +5119,11 @@ type maxFloat64OrderedAgg struct { col coldata.Float64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg float64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxFloat64OrderedAgg{} @@ -5128,13 +5155,13 @@ func (a *maxFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5142,10 +5169,9 @@ func (a *maxFloat64OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5179,6 +5205,7 @@ func (a *maxFloat64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -5189,13 +5216,13 @@ func (a *maxFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5203,10 +5230,9 @@ func (a *maxFloat64OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5240,6 +5266,7 @@ func (a *maxFloat64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -5252,13 +5279,13 @@ func (a *maxFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5266,10 +5293,9 @@ func (a *maxFloat64OrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5303,6 +5329,7 @@ func (a *maxFloat64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -5312,13 +5339,13 @@ func (a *maxFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5326,10 +5353,9 @@ func (a *maxFloat64OrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5363,6 +5389,7 @@ func (a *maxFloat64OrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -5383,7 +5410,7 @@ func (a *maxFloat64OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -5392,7 +5419,7 @@ func (a *maxFloat64OrderedAgg) Flush(outputIdx int) { func (a *maxFloat64OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxFloat64OrderedAggAlloc struct { @@ -5422,11 +5449,11 @@ type maxTimestampOrderedAgg struct { col coldata.Times // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg time.Time - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxTimestampOrderedAgg{} @@ -5458,13 +5485,13 @@ func (a *maxTimestampOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5472,10 +5499,9 @@ func (a *maxTimestampOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5497,6 +5523,7 @@ func (a *maxTimestampOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -5507,13 +5534,13 @@ func (a *maxTimestampOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5521,10 +5548,9 @@ func (a *maxTimestampOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5546,6 +5572,7 @@ func (a *maxTimestampOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -5558,13 +5585,13 @@ func (a *maxTimestampOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5572,10 +5599,9 @@ func (a *maxTimestampOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5597,6 +5623,7 @@ func (a *maxTimestampOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -5606,13 +5633,13 @@ func (a *maxTimestampOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5620,10 +5647,9 @@ func (a *maxTimestampOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5645,6 +5671,7 @@ func (a *maxTimestampOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -5665,7 +5692,7 @@ func (a *maxTimestampOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -5674,7 +5701,7 @@ func (a *maxTimestampOrderedAgg) Flush(outputIdx int) { func (a *maxTimestampOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxTimestampOrderedAggAlloc struct { @@ -5704,11 +5731,11 @@ type maxIntervalOrderedAgg struct { col coldata.Durations // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg duration.Duration - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxIntervalOrderedAgg{} @@ -5740,13 +5767,13 @@ func (a *maxIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5754,10 +5781,9 @@ func (a *maxIntervalOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5772,6 +5798,7 @@ func (a *maxIntervalOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -5782,13 +5809,13 @@ func (a *maxIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5796,10 +5823,9 @@ func (a *maxIntervalOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5814,6 +5840,7 @@ func (a *maxIntervalOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -5826,13 +5853,13 @@ func (a *maxIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5840,10 +5867,9 @@ func (a *maxIntervalOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5858,6 +5884,7 @@ func (a *maxIntervalOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -5867,13 +5894,13 @@ func (a *maxIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -5881,10 +5908,9 @@ func (a *maxIntervalOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -5899,6 +5925,7 @@ func (a *maxIntervalOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -5919,7 +5946,7 @@ func (a *maxIntervalOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -5928,7 +5955,7 @@ func (a *maxIntervalOrderedAgg) Flush(outputIdx int) { func (a *maxIntervalOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxIntervalOrderedAggAlloc struct { @@ -5958,11 +5985,11 @@ type maxJSONOrderedAgg struct { col *coldata.JSONs // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg json.JSON - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxJSONOrderedAgg{} @@ -5997,13 +6024,13 @@ func (a *maxJSONOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -6011,7 +6038,7 @@ func (a *maxJSONOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -6025,7 +6052,6 @@ func (a *maxJSONOrderedAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -6057,6 +6083,7 @@ func (a *maxJSONOrderedAgg) Compute( } } + a.numNonNull++ } } } else { @@ -6067,13 +6094,13 @@ func (a *maxJSONOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -6081,7 +6108,7 @@ func (a *maxJSONOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -6095,7 +6122,6 @@ func (a *maxJSONOrderedAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -6127,6 +6153,7 @@ func (a *maxJSONOrderedAgg) Compute( } } + a.numNonNull++ } } } @@ -6139,13 +6166,13 @@ func (a *maxJSONOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -6153,7 +6180,7 @@ func (a *maxJSONOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -6167,7 +6194,6 @@ func (a *maxJSONOrderedAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -6199,6 +6225,7 @@ func (a *maxJSONOrderedAgg) Compute( } } + a.numNonNull++ } } } else { @@ -6208,13 +6235,13 @@ func (a *maxJSONOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -6222,7 +6249,7 @@ func (a *maxJSONOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -6236,7 +6263,6 @@ func (a *maxJSONOrderedAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -6268,6 +6294,7 @@ func (a *maxJSONOrderedAgg) Compute( } } + a.numNonNull++ } } } @@ -6291,7 +6318,7 @@ func (a *maxJSONOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -6300,14 +6327,22 @@ func (a *maxJSONOrderedAgg) Flush(outputIdx int) { if a.curAgg != nil { oldCurAggSize = a.curAgg.Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *maxJSONOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + var oldCurAggSize uintptr + if a.curAgg != nil { + oldCurAggSize = a.curAgg.Size() + } + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type maxJSONOrderedAggAlloc struct { @@ -6337,11 +6372,11 @@ type maxDatumOrderedAgg struct { col coldata.DatumVec // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg interface{} - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxDatumOrderedAgg{} @@ -6377,13 +6412,13 @@ func (a *maxDatumOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -6391,10 +6426,9 @@ func (a *maxDatumOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -6411,6 +6445,7 @@ func (a *maxDatumOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -6421,13 +6456,13 @@ func (a *maxDatumOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -6435,10 +6470,9 @@ func (a *maxDatumOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -6455,6 +6489,7 @@ func (a *maxDatumOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -6467,13 +6502,13 @@ func (a *maxDatumOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -6481,10 +6516,9 @@ func (a *maxDatumOrderedAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -6501,6 +6535,7 @@ func (a *maxDatumOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -6510,13 +6545,13 @@ func (a *maxDatumOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col.Set(a.curIdx, a.curAgg) } a.curIdx++ - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -6524,10 +6559,9 @@ func (a *maxDatumOrderedAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -6544,6 +6578,7 @@ func (a *maxDatumOrderedAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -6568,7 +6603,7 @@ func (a *maxDatumOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col.Set(outputIdx, a.curAgg) @@ -6578,14 +6613,23 @@ func (a *maxDatumOrderedAgg) Flush(outputIdx int) { if a.curAgg != nil { oldCurAggSize = a.curAgg.(tree.Datum).Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg eagerly. We can't do this for the window + // variants because they may reuse curAgg between subsequent window frames. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } func (a *maxDatumOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + + var oldCurAggSize uintptr + if a.curAgg != nil { + oldCurAggSize = a.curAgg.(tree.Datum).Size() + } + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type maxDatumOrderedAggAlloc struct { diff --git a/pkg/sql/colexec/colexecagg/ordered_sum_agg.eg.go b/pkg/sql/colexec/colexecagg/ordered_sum_agg.eg.go index 9c3772e96239..a3c76c7efe2d 100644 --- a/pkg/sql/colexec/colexecagg/ordered_sum_agg.eg.go +++ b/pkg/sql/colexec/colexecagg/ordered_sum_agg.eg.go @@ -75,10 +75,10 @@ type sumInt16OrderedAgg struct { curAgg apd.Decimal // col points to the output vector we are updating. col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool - overloadHelper execgen.OverloadHelper + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 + overloadHelper execgen.OverloadHelper } var _ AggregateFunc = &sumInt16OrderedAgg{} @@ -113,7 +113,7 @@ func (a *sumInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -121,7 +121,7 @@ func (a *sumInt16OrderedAgg) Compute( a.curIdx++ a.curAgg = zeroDecimalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -141,7 +141,7 @@ func (a *sumInt16OrderedAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -152,7 +152,7 @@ func (a *sumInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -179,7 +179,7 @@ func (a *sumInt16OrderedAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -192,7 +192,7 @@ func (a *sumInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -200,7 +200,7 @@ func (a *sumInt16OrderedAgg) Compute( a.curIdx++ a.curAgg = zeroDecimalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -219,7 +219,7 @@ func (a *sumInt16OrderedAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -229,7 +229,7 @@ func (a *sumInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -255,7 +255,7 @@ func (a *sumInt16OrderedAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -276,7 +276,7 @@ func (a *sumInt16OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curAgg @@ -286,7 +286,7 @@ func (a *sumInt16OrderedAgg) Flush(outputIdx int) { func (a *sumInt16OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() a.curAgg = zeroDecimalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumInt16OrderedAggAlloc struct { @@ -317,10 +317,10 @@ type sumInt32OrderedAgg struct { curAgg apd.Decimal // col points to the output vector we are updating. col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool - overloadHelper execgen.OverloadHelper + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 + overloadHelper execgen.OverloadHelper } var _ AggregateFunc = &sumInt32OrderedAgg{} @@ -355,7 +355,7 @@ func (a *sumInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -363,7 +363,7 @@ func (a *sumInt32OrderedAgg) Compute( a.curIdx++ a.curAgg = zeroDecimalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -383,7 +383,7 @@ func (a *sumInt32OrderedAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -394,7 +394,7 @@ func (a *sumInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -421,7 +421,7 @@ func (a *sumInt32OrderedAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -434,7 +434,7 @@ func (a *sumInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -442,7 +442,7 @@ func (a *sumInt32OrderedAgg) Compute( a.curIdx++ a.curAgg = zeroDecimalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -461,7 +461,7 @@ func (a *sumInt32OrderedAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -471,7 +471,7 @@ func (a *sumInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -497,7 +497,7 @@ func (a *sumInt32OrderedAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -518,7 +518,7 @@ func (a *sumInt32OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curAgg @@ -528,7 +528,7 @@ func (a *sumInt32OrderedAgg) Flush(outputIdx int) { func (a *sumInt32OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() a.curAgg = zeroDecimalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumInt32OrderedAggAlloc struct { @@ -559,10 +559,10 @@ type sumInt64OrderedAgg struct { curAgg apd.Decimal // col points to the output vector we are updating. col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool - overloadHelper execgen.OverloadHelper + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 + overloadHelper execgen.OverloadHelper } var _ AggregateFunc = &sumInt64OrderedAgg{} @@ -597,7 +597,7 @@ func (a *sumInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -605,7 +605,7 @@ func (a *sumInt64OrderedAgg) Compute( a.curIdx++ a.curAgg = zeroDecimalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -625,7 +625,7 @@ func (a *sumInt64OrderedAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -636,7 +636,7 @@ func (a *sumInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -663,7 +663,7 @@ func (a *sumInt64OrderedAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -676,7 +676,7 @@ func (a *sumInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -684,7 +684,7 @@ func (a *sumInt64OrderedAgg) Compute( a.curIdx++ a.curAgg = zeroDecimalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -703,7 +703,7 @@ func (a *sumInt64OrderedAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -713,7 +713,7 @@ func (a *sumInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -739,7 +739,7 @@ func (a *sumInt64OrderedAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -760,7 +760,7 @@ func (a *sumInt64OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curAgg @@ -770,7 +770,7 @@ func (a *sumInt64OrderedAgg) Flush(outputIdx int) { func (a *sumInt64OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() a.curAgg = zeroDecimalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumInt64OrderedAggAlloc struct { @@ -801,9 +801,9 @@ type sumDecimalOrderedAgg struct { curAgg apd.Decimal // col points to the output vector we are updating. col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &sumDecimalOrderedAgg{} @@ -835,7 +835,7 @@ func (a *sumDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -843,7 +843,7 @@ func (a *sumDecimalOrderedAgg) Compute( a.curIdx++ a.curAgg = zeroDecimalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -862,7 +862,7 @@ func (a *sumDecimalOrderedAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -873,7 +873,7 @@ func (a *sumDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -899,7 +899,7 @@ func (a *sumDecimalOrderedAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -912,7 +912,7 @@ func (a *sumDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -920,7 +920,7 @@ func (a *sumDecimalOrderedAgg) Compute( a.curIdx++ a.curAgg = zeroDecimalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -938,7 +938,7 @@ func (a *sumDecimalOrderedAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -948,7 +948,7 @@ func (a *sumDecimalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -973,7 +973,7 @@ func (a *sumDecimalOrderedAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -994,7 +994,7 @@ func (a *sumDecimalOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curAgg @@ -1004,7 +1004,7 @@ func (a *sumDecimalOrderedAgg) Flush(outputIdx int) { func (a *sumDecimalOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() a.curAgg = zeroDecimalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumDecimalOrderedAggAlloc struct { @@ -1035,9 +1035,9 @@ type sumFloat64OrderedAgg struct { curAgg float64 // col points to the output vector we are updating. col []float64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &sumFloat64OrderedAgg{} @@ -1069,7 +1069,7 @@ func (a *sumFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -1077,7 +1077,7 @@ func (a *sumFloat64OrderedAgg) Compute( a.curIdx++ a.curAgg = zeroFloat64Value - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1093,7 +1093,7 @@ func (a *sumFloat64OrderedAgg) Compute( a.curAgg = float64(a.curAgg) + float64(v) } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -1104,7 +1104,7 @@ func (a *sumFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -1127,7 +1127,7 @@ func (a *sumFloat64OrderedAgg) Compute( a.curAgg = float64(a.curAgg) + float64(v) } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -1140,7 +1140,7 @@ func (a *sumFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -1148,7 +1148,7 @@ func (a *sumFloat64OrderedAgg) Compute( a.curIdx++ a.curAgg = zeroFloat64Value - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1163,7 +1163,7 @@ func (a *sumFloat64OrderedAgg) Compute( a.curAgg = float64(a.curAgg) + float64(v) } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -1173,7 +1173,7 @@ func (a *sumFloat64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -1195,7 +1195,7 @@ func (a *sumFloat64OrderedAgg) Compute( a.curAgg = float64(a.curAgg) + float64(v) } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -1216,7 +1216,7 @@ func (a *sumFloat64OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curAgg @@ -1226,7 +1226,7 @@ func (a *sumFloat64OrderedAgg) Flush(outputIdx int) { func (a *sumFloat64OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() a.curAgg = zeroFloat64Value - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumFloat64OrderedAggAlloc struct { @@ -1257,9 +1257,9 @@ type sumIntervalOrderedAgg struct { curAgg duration.Duration // col points to the output vector we are updating. col []duration.Duration - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &sumIntervalOrderedAgg{} @@ -1291,7 +1291,7 @@ func (a *sumIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -1299,7 +1299,7 @@ func (a *sumIntervalOrderedAgg) Compute( a.curIdx++ a.curAgg = zeroIntervalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1310,7 +1310,7 @@ func (a *sumIntervalOrderedAgg) Compute( //gcassert:bce v := col.Get(i) a.curAgg = a.curAgg.Add(v) - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -1321,7 +1321,7 @@ func (a *sumIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -1339,7 +1339,7 @@ func (a *sumIntervalOrderedAgg) Compute( //gcassert:bce v := col.Get(i) a.curAgg = a.curAgg.Add(v) - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -1352,7 +1352,7 @@ func (a *sumIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -1360,7 +1360,7 @@ func (a *sumIntervalOrderedAgg) Compute( a.curIdx++ a.curAgg = zeroIntervalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -1370,7 +1370,7 @@ func (a *sumIntervalOrderedAgg) Compute( if !isNull { v := col.Get(i) a.curAgg = a.curAgg.Add(v) - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -1380,7 +1380,7 @@ func (a *sumIntervalOrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -1397,7 +1397,7 @@ func (a *sumIntervalOrderedAgg) Compute( if !isNull { v := col.Get(i) a.curAgg = a.curAgg.Add(v) - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -1418,7 +1418,7 @@ func (a *sumIntervalOrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curAgg @@ -1428,7 +1428,7 @@ func (a *sumIntervalOrderedAgg) Flush(outputIdx int) { func (a *sumIntervalOrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() a.curAgg = zeroIntervalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumIntervalOrderedAggAlloc struct { diff --git a/pkg/sql/colexec/colexecagg/ordered_sum_int_agg.eg.go b/pkg/sql/colexec/colexecagg/ordered_sum_int_agg.eg.go index 9c9ad6822ccf..5fa5564eac74 100644 --- a/pkg/sql/colexec/colexecagg/ordered_sum_int_agg.eg.go +++ b/pkg/sql/colexec/colexecagg/ordered_sum_int_agg.eg.go @@ -56,9 +56,9 @@ type sumIntInt16OrderedAgg struct { curAgg int64 // col points to the output vector we are updating. col []int64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &sumIntInt16OrderedAgg{} @@ -90,7 +90,7 @@ func (a *sumIntInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -98,7 +98,7 @@ func (a *sumIntInt16OrderedAgg) Compute( a.curIdx++ a.curAgg = zeroInt64Value - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -117,7 +117,7 @@ func (a *sumIntInt16OrderedAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -128,7 +128,7 @@ func (a *sumIntInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -154,7 +154,7 @@ func (a *sumIntInt16OrderedAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -167,7 +167,7 @@ func (a *sumIntInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -175,7 +175,7 @@ func (a *sumIntInt16OrderedAgg) Compute( a.curIdx++ a.curAgg = zeroInt64Value - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -193,7 +193,7 @@ func (a *sumIntInt16OrderedAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -203,7 +203,7 @@ func (a *sumIntInt16OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -228,7 +228,7 @@ func (a *sumIntInt16OrderedAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -249,7 +249,7 @@ func (a *sumIntInt16OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curAgg @@ -259,7 +259,7 @@ func (a *sumIntInt16OrderedAgg) Flush(outputIdx int) { func (a *sumIntInt16OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() a.curAgg = zeroInt64Value - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumIntInt16OrderedAggAlloc struct { @@ -290,9 +290,9 @@ type sumIntInt32OrderedAgg struct { curAgg int64 // col points to the output vector we are updating. col []int64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &sumIntInt32OrderedAgg{} @@ -324,7 +324,7 @@ func (a *sumIntInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -332,7 +332,7 @@ func (a *sumIntInt32OrderedAgg) Compute( a.curIdx++ a.curAgg = zeroInt64Value - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -351,7 +351,7 @@ func (a *sumIntInt32OrderedAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -362,7 +362,7 @@ func (a *sumIntInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -388,7 +388,7 @@ func (a *sumIntInt32OrderedAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -401,7 +401,7 @@ func (a *sumIntInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -409,7 +409,7 @@ func (a *sumIntInt32OrderedAgg) Compute( a.curIdx++ a.curAgg = zeroInt64Value - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -427,7 +427,7 @@ func (a *sumIntInt32OrderedAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -437,7 +437,7 @@ func (a *sumIntInt32OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -462,7 +462,7 @@ func (a *sumIntInt32OrderedAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -483,7 +483,7 @@ func (a *sumIntInt32OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curAgg @@ -493,7 +493,7 @@ func (a *sumIntInt32OrderedAgg) Flush(outputIdx int) { func (a *sumIntInt32OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() a.curAgg = zeroInt64Value - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumIntInt32OrderedAggAlloc struct { @@ -524,9 +524,9 @@ type sumIntInt64OrderedAgg struct { curAgg int64 // col points to the output vector we are updating. col []int64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &sumIntInt64OrderedAgg{} @@ -558,7 +558,7 @@ func (a *sumIntInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -566,7 +566,7 @@ func (a *sumIntInt64OrderedAgg) Compute( a.curIdx++ a.curAgg = zeroInt64Value - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -585,7 +585,7 @@ func (a *sumIntInt64OrderedAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -596,7 +596,7 @@ func (a *sumIntInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -622,7 +622,7 @@ func (a *sumIntInt64OrderedAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -635,7 +635,7 @@ func (a *sumIntInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -643,7 +643,7 @@ func (a *sumIntInt64OrderedAgg) Compute( a.curIdx++ a.curAgg = zeroInt64Value - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } a.isFirstGroup = false } @@ -661,7 +661,7 @@ func (a *sumIntInt64OrderedAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -671,7 +671,7 @@ func (a *sumIntInt64OrderedAgg) Compute( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -696,7 +696,7 @@ func (a *sumIntInt64OrderedAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -717,7 +717,7 @@ func (a *sumIntInt64OrderedAgg) Flush(outputIdx int) { _ = outputIdx outputIdx = a.curIdx a.curIdx++ - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curAgg @@ -727,7 +727,7 @@ func (a *sumIntInt64OrderedAgg) Flush(outputIdx int) { func (a *sumIntInt64OrderedAgg) Reset() { a.orderedAggregateFuncBase.Reset() a.curAgg = zeroInt64Value - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumIntInt64OrderedAggAlloc struct { diff --git a/pkg/sql/colexec/colexecagg/sum_agg_tmpl.go b/pkg/sql/colexec/colexecagg/sum_agg_tmpl.go index c66d2272909d..ad80be8b2dac 100644 --- a/pkg/sql/colexec/colexecagg/sum_agg_tmpl.go +++ b/pkg/sql/colexec/colexecagg/sum_agg_tmpl.go @@ -50,6 +50,12 @@ func _ASSIGN_ADD(_, _, _, _, _, _ string) { colexecerror.InternalError(errors.AssertionFailedf("")) } +// _ASSIGN_SUBTRACT is the template subtraction function for assigning the first +// input to the result of the second input - the third input. +func _ASSIGN_SUBTRACT(_, _, _, _, _, _ string) { + colexecerror.InternalError(errors.AssertionFailedf("")) +} + // */}} func newSum_SUMKIND_AGGKINDAggAlloc( @@ -87,9 +93,9 @@ type sum_SUMKIND_TYPE_AGGKINDAgg struct { curAgg _RET_GOTYPE // col points to the output vector we are updating. col []_RET_GOTYPE - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 // {{if .NeedsHelper}} // {{/* // overloadHelper is used only when we perform the summation of integers @@ -200,10 +206,16 @@ func (a *sum_SUMKIND_TYPE_AGGKINDAgg) Flush(outputIdx int) { outputIdx = a.curIdx a.curIdx++ // {{end}} - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // {{if eq "_AGGKIND" "Window"}} + // We need to copy the value because window functions reuse the aggregation + // between rows. + execgen.COPYVAL(a.col[outputIdx], a.curAgg) + // {{else}} a.col[outputIdx] = a.curAgg + // {{end}} } } @@ -212,7 +224,7 @@ func (a *sum_SUMKIND_TYPE_AGGKINDAgg) Reset() { a.orderedAggregateFuncBase.Reset() // {{end}} a.curAgg = zero_RET_TYPEValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sum_SUMKIND_TYPE_AGGKINDAggAlloc struct { @@ -236,6 +248,38 @@ func (a *sum_SUMKIND_TYPE_AGGKINDAggAlloc) newAggFunc() AggregateFunc { return f } +// {{if eq "_AGGKIND" "Window"}} + +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). +func (a *sum_SUMKIND_TYPE_AGGKINDAgg) Remove( + vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int, +) { + // {{if .NeedsHelper}} + // In order to inline the templated code of overloads, we need to have a + // "_overloadHelper" local variable of type "overloadHelper". + _overloadHelper := a.overloadHelper + // {{end}} + execgen.SETVARIABLESIZE(oldCurAggSize, a.curAgg) + vec := vecs[inputIdxs[0]] + col, nulls := vec.TemplateType(), vec.Nulls() + _, _ = col.Get(endIdx-1), col.Get(startIdx) + if nulls.MaybeHasNulls() { + for i := startIdx; i < endIdx; i++ { + _REMOVE_ROW(a, nulls, i, true) + } + } else { + for i := startIdx; i < endIdx; i++ { + _REMOVE_ROW(a, nulls, i, false) + } + } + execgen.SETVARIABLESIZE(newCurAggSize, a.curAgg) + if newCurAggSize != oldCurAggSize { + a.allocator.AdjustMemoryUsage(int64(newCurAggSize - oldCurAggSize)) + } +} + +// {{end}} // {{end}} // {{end}} // {{end}} @@ -258,7 +302,7 @@ func _ACCUMULATE_SUM( if !a.isFirstGroup { // If we encounter a new group, and we haven't found any non-nulls for the // current group, the output for this group should be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(a.curIdx) } else { a.col[a.curIdx] = a.curAgg @@ -273,7 +317,7 @@ func _ACCUMULATE_SUM( // nulls, this will be updated unconditionally below. // */}} // {{if .HasNulls}} - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 // {{end}} } a.isFirstGroup = false @@ -292,9 +336,30 @@ func _ACCUMULATE_SUM( // {{end}} v := col.Get(i) _ASSIGN_ADD(a.curAgg, a.curAgg, v, _, _, col) - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } // {{end}} // {{/* } // */}} + +// {{/* +// _REMOVE_ROW removes the value of the ith row from the output for the +// current aggregation. +func _REMOVE_ROW(a *sum_SUMKIND_TYPE_AGGKINDAgg, nulls *coldata.Nulls, i int, _HAS_NULLS bool) { // */}} + // {{define "removeRow"}} + var isNull bool + // {{if .HasNulls}} + isNull = nulls.NullAt(i) + // {{else}} + isNull = false + // {{end}} + if !isNull { + //gcassert:bce + v := col.Get(i) + _ASSIGN_SUBTRACT(a.curAgg, a.curAgg, v, _, _, col) + a.numNonNull-- + } + // {{end}} + // {{/* +} // */}} diff --git a/pkg/sql/colexec/colexecagg/window_avg_agg.eg.go b/pkg/sql/colexec/colexecagg/window_avg_agg.eg.go index 1a2796ccabde..a2751062e5c9 100644 --- a/pkg/sql/colexec/colexecagg/window_avg_agg.eg.go +++ b/pkg/sql/colexec/colexecagg/window_avg_agg.eg.go @@ -74,15 +74,12 @@ type avgInt16WindowAgg struct { // so we can index into the slice once per group, instead of on each // iteration. curSum apd.Decimal - // curCount keeps track of the number of elements that we've seen + // curCount keeps track of the number of non-null elements that we've seen // belonging to the current group. curCount int64 // col points to the statically-typed output vector. - col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool - overloadHelper execgen.OverloadHelper + col []apd.Decimal + overloadHelper execgen.OverloadHelper } var _ AggregateFunc = &avgInt16WindowAgg{} @@ -124,7 +121,6 @@ func (a *avgInt16WindowAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -146,7 +142,6 @@ func (a *avgInt16WindowAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -160,7 +155,7 @@ func (a *avgInt16WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // NULL. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(outputIdx) } else { @@ -174,7 +169,6 @@ func (a *avgInt16WindowAgg) Flush(outputIdx int) { func (a *avgInt16WindowAgg) Reset() { a.curSum = zeroDecimalValue a.curCount = 0 - a.foundNonNullForCurrentGroup = false } type avgInt16WindowAggAlloc struct { @@ -198,21 +192,77 @@ func (a *avgInt16WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). +func (a *avgInt16WindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + // In order to inline the templated code of overloads, we need to have a + // "_overloadHelper" local variable of type "overloadHelper". + _overloadHelper := a.overloadHelper + oldCurSumSize := tree.SizeOfDecimal(&a.curSum) + vec := vecs[inputIdxs[0]] + col, nulls := vec.Int16(), vec.Nulls() + _, _ = col.Get(endIdx-1), col.Get(startIdx) + if nulls.MaybeHasNulls() { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = nulls.NullAt(i) + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + + tmpDec := &_overloadHelper.TmpDec1 + tmpDec.SetInt64(int64(v)) + if _, err := tree.ExactCtx.Sub(&a.curSum, &a.curSum, tmpDec); err != nil { + colexecerror.ExpectedError(err) + } + } + + a.curCount-- + } + } + } else { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = false + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + + tmpDec := &_overloadHelper.TmpDec1 + tmpDec.SetInt64(int64(v)) + if _, err := tree.ExactCtx.Sub(&a.curSum, &a.curSum, tmpDec); err != nil { + colexecerror.ExpectedError(err) + } + } + + a.curCount-- + } + } + } + newCurSumSize := tree.SizeOfDecimal(&a.curSum) + if newCurSumSize != oldCurSumSize { + a.allocator.AdjustMemoryUsage(int64(newCurSumSize - oldCurSumSize)) + } +} + type avgInt32WindowAgg struct { unorderedAggregateFuncBase // curSum keeps track of the sum of elements belonging to the current group, // so we can index into the slice once per group, instead of on each // iteration. curSum apd.Decimal - // curCount keeps track of the number of elements that we've seen + // curCount keeps track of the number of non-null elements that we've seen // belonging to the current group. curCount int64 // col points to the statically-typed output vector. - col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool - overloadHelper execgen.OverloadHelper + col []apd.Decimal + overloadHelper execgen.OverloadHelper } var _ AggregateFunc = &avgInt32WindowAgg{} @@ -254,7 +304,6 @@ func (a *avgInt32WindowAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -276,7 +325,6 @@ func (a *avgInt32WindowAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -290,7 +338,7 @@ func (a *avgInt32WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // NULL. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(outputIdx) } else { @@ -304,7 +352,6 @@ func (a *avgInt32WindowAgg) Flush(outputIdx int) { func (a *avgInt32WindowAgg) Reset() { a.curSum = zeroDecimalValue a.curCount = 0 - a.foundNonNullForCurrentGroup = false } type avgInt32WindowAggAlloc struct { @@ -328,21 +375,77 @@ func (a *avgInt32WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). +func (a *avgInt32WindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + // In order to inline the templated code of overloads, we need to have a + // "_overloadHelper" local variable of type "overloadHelper". + _overloadHelper := a.overloadHelper + oldCurSumSize := tree.SizeOfDecimal(&a.curSum) + vec := vecs[inputIdxs[0]] + col, nulls := vec.Int32(), vec.Nulls() + _, _ = col.Get(endIdx-1), col.Get(startIdx) + if nulls.MaybeHasNulls() { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = nulls.NullAt(i) + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + + tmpDec := &_overloadHelper.TmpDec1 + tmpDec.SetInt64(int64(v)) + if _, err := tree.ExactCtx.Sub(&a.curSum, &a.curSum, tmpDec); err != nil { + colexecerror.ExpectedError(err) + } + } + + a.curCount-- + } + } + } else { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = false + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + + tmpDec := &_overloadHelper.TmpDec1 + tmpDec.SetInt64(int64(v)) + if _, err := tree.ExactCtx.Sub(&a.curSum, &a.curSum, tmpDec); err != nil { + colexecerror.ExpectedError(err) + } + } + + a.curCount-- + } + } + } + newCurSumSize := tree.SizeOfDecimal(&a.curSum) + if newCurSumSize != oldCurSumSize { + a.allocator.AdjustMemoryUsage(int64(newCurSumSize - oldCurSumSize)) + } +} + type avgInt64WindowAgg struct { unorderedAggregateFuncBase // curSum keeps track of the sum of elements belonging to the current group, // so we can index into the slice once per group, instead of on each // iteration. curSum apd.Decimal - // curCount keeps track of the number of elements that we've seen + // curCount keeps track of the number of non-null elements that we've seen // belonging to the current group. curCount int64 // col points to the statically-typed output vector. - col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool - overloadHelper execgen.OverloadHelper + col []apd.Decimal + overloadHelper execgen.OverloadHelper } var _ AggregateFunc = &avgInt64WindowAgg{} @@ -384,7 +487,6 @@ func (a *avgInt64WindowAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -406,7 +508,6 @@ func (a *avgInt64WindowAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -420,7 +521,7 @@ func (a *avgInt64WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // NULL. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(outputIdx) } else { @@ -434,7 +535,6 @@ func (a *avgInt64WindowAgg) Flush(outputIdx int) { func (a *avgInt64WindowAgg) Reset() { a.curSum = zeroDecimalValue a.curCount = 0 - a.foundNonNullForCurrentGroup = false } type avgInt64WindowAggAlloc struct { @@ -458,20 +558,76 @@ func (a *avgInt64WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). +func (a *avgInt64WindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + // In order to inline the templated code of overloads, we need to have a + // "_overloadHelper" local variable of type "overloadHelper". + _overloadHelper := a.overloadHelper + oldCurSumSize := tree.SizeOfDecimal(&a.curSum) + vec := vecs[inputIdxs[0]] + col, nulls := vec.Int64(), vec.Nulls() + _, _ = col.Get(endIdx-1), col.Get(startIdx) + if nulls.MaybeHasNulls() { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = nulls.NullAt(i) + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + + tmpDec := &_overloadHelper.TmpDec1 + tmpDec.SetInt64(int64(v)) + if _, err := tree.ExactCtx.Sub(&a.curSum, &a.curSum, tmpDec); err != nil { + colexecerror.ExpectedError(err) + } + } + + a.curCount-- + } + } + } else { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = false + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + + tmpDec := &_overloadHelper.TmpDec1 + tmpDec.SetInt64(int64(v)) + if _, err := tree.ExactCtx.Sub(&a.curSum, &a.curSum, tmpDec); err != nil { + colexecerror.ExpectedError(err) + } + } + + a.curCount-- + } + } + } + newCurSumSize := tree.SizeOfDecimal(&a.curSum) + if newCurSumSize != oldCurSumSize { + a.allocator.AdjustMemoryUsage(int64(newCurSumSize - oldCurSumSize)) + } +} + type avgDecimalWindowAgg struct { unorderedAggregateFuncBase // curSum keeps track of the sum of elements belonging to the current group, // so we can index into the slice once per group, instead of on each // iteration. curSum apd.Decimal - // curCount keeps track of the number of elements that we've seen + // curCount keeps track of the number of non-null elements that we've seen // belonging to the current group. curCount int64 // col points to the statically-typed output vector. col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool } var _ AggregateFunc = &avgDecimalWindowAgg{} @@ -509,7 +665,6 @@ func (a *avgDecimalWindowAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -530,7 +685,6 @@ func (a *avgDecimalWindowAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -544,7 +698,7 @@ func (a *avgDecimalWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // NULL. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(outputIdx) } else { @@ -558,7 +712,6 @@ func (a *avgDecimalWindowAgg) Flush(outputIdx int) { func (a *avgDecimalWindowAgg) Reset() { a.curSum = zeroDecimalValue a.curCount = 0 - a.foundNonNullForCurrentGroup = false } type avgDecimalWindowAggAlloc struct { @@ -582,20 +735,71 @@ func (a *avgDecimalWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). +func (a *avgDecimalWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + oldCurSumSize := tree.SizeOfDecimal(&a.curSum) + vec := vecs[inputIdxs[0]] + col, nulls := vec.Decimal(), vec.Nulls() + _, _ = col.Get(endIdx-1), col.Get(startIdx) + if nulls.MaybeHasNulls() { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = nulls.NullAt(i) + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + + _, err := tree.ExactCtx.Sub(&a.curSum, &a.curSum, &v) + if err != nil { + colexecerror.ExpectedError(err) + } + } + + a.curCount-- + } + } + } else { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = false + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + + _, err := tree.ExactCtx.Sub(&a.curSum, &a.curSum, &v) + if err != nil { + colexecerror.ExpectedError(err) + } + } + + a.curCount-- + } + } + } + newCurSumSize := tree.SizeOfDecimal(&a.curSum) + if newCurSumSize != oldCurSumSize { + a.allocator.AdjustMemoryUsage(int64(newCurSumSize - oldCurSumSize)) + } +} + type avgFloat64WindowAgg struct { unorderedAggregateFuncBase // curSum keeps track of the sum of elements belonging to the current group, // so we can index into the slice once per group, instead of on each // iteration. curSum float64 - // curCount keeps track of the number of elements that we've seen + // curCount keeps track of the number of non-null elements that we've seen // belonging to the current group. curCount int64 // col points to the statically-typed output vector. col []float64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool } var _ AggregateFunc = &avgFloat64WindowAgg{} @@ -630,7 +834,6 @@ func (a *avgFloat64WindowAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -648,7 +851,6 @@ func (a *avgFloat64WindowAgg) Compute( } a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -662,7 +864,7 @@ func (a *avgFloat64WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // NULL. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curSum / float64(a.curCount) @@ -672,7 +874,6 @@ func (a *avgFloat64WindowAgg) Flush(outputIdx int) { func (a *avgFloat64WindowAgg) Reset() { a.curSum = zeroFloat64Value a.curCount = 0 - a.foundNonNullForCurrentGroup = false } type avgFloat64WindowAggAlloc struct { @@ -696,20 +897,65 @@ func (a *avgFloat64WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). +func (a *avgFloat64WindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + var oldCurSumSize uintptr + vec := vecs[inputIdxs[0]] + col, nulls := vec.Float64(), vec.Nulls() + _, _ = col.Get(endIdx-1), col.Get(startIdx) + if nulls.MaybeHasNulls() { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = nulls.NullAt(i) + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + + a.curSum = float64(a.curSum) - float64(v) + } + + a.curCount-- + } + } + } else { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = false + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + + a.curSum = float64(a.curSum) - float64(v) + } + + a.curCount-- + } + } + } + var newCurSumSize uintptr + if newCurSumSize != oldCurSumSize { + a.allocator.AdjustMemoryUsage(int64(newCurSumSize - oldCurSumSize)) + } +} + type avgIntervalWindowAgg struct { unorderedAggregateFuncBase // curSum keeps track of the sum of elements belonging to the current group, // so we can index into the slice once per group, instead of on each // iteration. curSum duration.Duration - // curCount keeps track of the number of elements that we've seen + // curCount keeps track of the number of non-null elements that we've seen // belonging to the current group. curCount int64 // col points to the statically-typed output vector. col []duration.Duration - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool } var _ AggregateFunc = &avgIntervalWindowAgg{} @@ -739,7 +985,6 @@ func (a *avgIntervalWindowAgg) Compute( v := col.Get(i) a.curSum = a.curSum.Add(v) a.curCount++ - a.foundNonNullForCurrentGroup = true } } } else { @@ -752,7 +997,6 @@ func (a *avgIntervalWindowAgg) Compute( v := col.Get(i) a.curSum = a.curSum.Add(v) a.curCount++ - a.foundNonNullForCurrentGroup = true } } } @@ -766,7 +1010,7 @@ func (a *avgIntervalWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // NULL. - if !a.foundNonNullForCurrentGroup { + if a.curCount == 0 { a.nulls.SetNull(outputIdx) } else { a.col[outputIdx] = a.curSum.Div(int64(a.curCount)) @@ -776,7 +1020,6 @@ func (a *avgIntervalWindowAgg) Flush(outputIdx int) { func (a *avgIntervalWindowAgg) Reset() { a.curSum = zeroIntervalValue a.curCount = 0 - a.foundNonNullForCurrentGroup = false } type avgIntervalWindowAggAlloc struct { @@ -799,3 +1042,41 @@ func (a *avgIntervalWindowAggAlloc) newAggFunc() AggregateFunc { a.aggFuncs = a.aggFuncs[1:] return f } + +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). +func (a *avgIntervalWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + var oldCurSumSize uintptr + vec := vecs[inputIdxs[0]] + col, nulls := vec.Interval(), vec.Nulls() + _, _ = col.Get(endIdx-1), col.Get(startIdx) + if nulls.MaybeHasNulls() { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = nulls.NullAt(i) + if !isNull { + //gcassert:bce + v := col.Get(i) + a.curSum = a.curSum.Sub(v) + a.curCount-- + } + } + } else { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = false + if !isNull { + //gcassert:bce + v := col.Get(i) + a.curSum = a.curSum.Sub(v) + a.curCount-- + } + } + } + var newCurSumSize uintptr + if newCurSumSize != oldCurSumSize { + a.allocator.AdjustMemoryUsage(int64(newCurSumSize - oldCurSumSize)) + } +} diff --git a/pkg/sql/colexec/colexecagg/window_count_agg.eg.go b/pkg/sql/colexec/colexecagg/window_count_agg.eg.go index c630403240db..cd8cda89d41c 100644 --- a/pkg/sql/colexec/colexecagg/window_count_agg.eg.go +++ b/pkg/sql/colexec/colexecagg/window_count_agg.eg.go @@ -42,7 +42,6 @@ func (a *countRowsWindowAgg) SetOutput(vec coldata.Vec) { func (a *countRowsWindowAgg) Compute( vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int, sel []int, ) { - var oldCurAggSize uintptr // Unnecessary memory accounting can have significant overhead for window // aggregate functions because Compute is called at least once for every row. // For this reason, we do not use PerformOperation here. @@ -54,10 +53,6 @@ func (a *countRowsWindowAgg) Compute( a.curAgg += y } } - var newCurAggSize uintptr - if newCurAggSize != oldCurAggSize { - a.allocator.AdjustMemoryUsage(int64(newCurAggSize - oldCurAggSize)) - } } func (a *countRowsWindowAgg) Flush(outputIdx int) { @@ -115,7 +110,6 @@ func (a *countWindowAgg) SetOutput(vec coldata.Vec) { func (a *countWindowAgg) Compute( vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int, sel []int, ) { - var oldCurAggSize uintptr // If this is a COUNT(col) aggregator and there are nulls in this batch, // we must check each value for nullity. Note that it is only legal to do a // COUNT aggregate on a single column. @@ -141,10 +135,6 @@ func (a *countWindowAgg) Compute( a.curAgg += y } } - var newCurAggSize uintptr - if newCurAggSize != oldCurAggSize { - a.allocator.AdjustMemoryUsage(int64(newCurAggSize - oldCurAggSize)) - } } func (a *countWindowAgg) Flush(outputIdx int) { @@ -155,6 +145,32 @@ func (a *countWindowAgg) Reset() { a.curAgg = 0 } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). +func (a *countWindowAgg) Remove( + vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int, +) { + nulls := vecs[inputIdxs[0]].Nulls() + if nulls.MaybeHasNulls() { + for i := startIdx; i < endIdx; i++ { + + var y int64 + y = int64(0) + if !nulls.NullAt(i) { + y = 1 + } + a.curAgg -= y + } + } else { + for i := startIdx; i < endIdx; i++ { + + var y int64 + y = int64(1) + a.curAgg -= y + } + } +} + type countWindowAggAlloc struct { aggAllocBase aggFuncs []countWindowAgg diff --git a/pkg/sql/colexec/colexecagg/window_min_max_agg.eg.go b/pkg/sql/colexec/colexecagg/window_min_max_agg.eg.go index 74fc138cad4a..e969b2270340 100644 --- a/pkg/sql/colexec/colexecagg/window_min_max_agg.eg.go +++ b/pkg/sql/colexec/colexecagg/window_min_max_agg.eg.go @@ -116,11 +116,11 @@ type minBoolWindowAgg struct { col coldata.Bools // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg bool - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minBoolWindowAgg{} @@ -146,10 +146,9 @@ func (a *minBoolWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -172,6 +171,7 @@ func (a *minBoolWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -180,10 +180,9 @@ func (a *minBoolWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -206,6 +205,7 @@ func (a *minBoolWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -219,15 +219,18 @@ func (a *minBoolWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *minBoolWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minBoolWindowAggAlloc struct { @@ -251,17 +254,25 @@ func (a *minBoolWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minBoolWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minBoolWindowAgg")) +} + type minBytesWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col *coldata.Bytes // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg []byte - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minBytesWindowAgg{} @@ -287,10 +298,9 @@ func (a *minBytesWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -305,6 +315,7 @@ func (a *minBytesWindowAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } else { @@ -313,10 +324,9 @@ func (a *minBytesWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -331,6 +341,7 @@ func (a *minBytesWindowAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } @@ -344,19 +355,22 @@ func (a *minBytesWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = append(a.curAgg[:0], a.curAgg...) a.col.Set(outputIdx, a.curAgg) } - oldCurAggSize := len(a.curAgg) - // Release the reference to curAgg eagerly. - a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) - a.curAgg = nil } func (a *minBytesWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + oldCurAggSize := len(a.curAgg) + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type minBytesWindowAggAlloc struct { @@ -380,17 +394,25 @@ func (a *minBytesWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minBytesWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minBytesWindowAgg")) +} + type minDecimalWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Decimals // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minDecimalWindowAgg{} @@ -416,10 +438,9 @@ func (a *minDecimalWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -434,6 +455,7 @@ func (a *minDecimalWindowAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } else { @@ -442,10 +464,9 @@ func (a *minDecimalWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -460,6 +481,7 @@ func (a *minDecimalWindowAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } @@ -473,15 +495,18 @@ func (a *minDecimalWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg.Set(&a.curAgg) a.col.Set(outputIdx, a.curAgg) } } func (a *minDecimalWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minDecimalWindowAggAlloc struct { @@ -505,17 +530,25 @@ func (a *minDecimalWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minDecimalWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minDecimalWindowAgg")) +} + type minInt16WindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Int16s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int16 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minInt16WindowAgg{} @@ -541,10 +574,9 @@ func (a *minInt16WindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -570,6 +602,7 @@ func (a *minInt16WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -578,10 +611,9 @@ func (a *minInt16WindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -607,6 +639,7 @@ func (a *minInt16WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -620,15 +653,18 @@ func (a *minInt16WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *minInt16WindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minInt16WindowAggAlloc struct { @@ -652,17 +688,25 @@ func (a *minInt16WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minInt16WindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minInt16WindowAgg")) +} + type minInt32WindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Int32s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int32 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minInt32WindowAgg{} @@ -688,10 +732,9 @@ func (a *minInt32WindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -717,6 +760,7 @@ func (a *minInt32WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -725,10 +769,9 @@ func (a *minInt32WindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -754,6 +797,7 @@ func (a *minInt32WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -767,15 +811,18 @@ func (a *minInt32WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *minInt32WindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minInt32WindowAggAlloc struct { @@ -799,17 +846,25 @@ func (a *minInt32WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minInt32WindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minInt32WindowAgg")) +} + type minInt64WindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Int64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minInt64WindowAgg{} @@ -835,10 +890,9 @@ func (a *minInt64WindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -864,6 +918,7 @@ func (a *minInt64WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -872,10 +927,9 @@ func (a *minInt64WindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -901,6 +955,7 @@ func (a *minInt64WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -914,15 +969,18 @@ func (a *minInt64WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *minInt64WindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minInt64WindowAggAlloc struct { @@ -946,17 +1004,25 @@ func (a *minInt64WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minInt64WindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minInt64WindowAgg")) +} + type minFloat64WindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Float64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg float64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minFloat64WindowAgg{} @@ -982,10 +1048,9 @@ func (a *minFloat64WindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1019,6 +1084,7 @@ func (a *minFloat64WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1027,10 +1093,9 @@ func (a *minFloat64WindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1064,6 +1129,7 @@ func (a *minFloat64WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1077,15 +1143,18 @@ func (a *minFloat64WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *minFloat64WindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minFloat64WindowAggAlloc struct { @@ -1109,17 +1178,25 @@ func (a *minFloat64WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minFloat64WindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minFloat64WindowAgg")) +} + type minTimestampWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Times // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg time.Time - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minTimestampWindowAgg{} @@ -1145,10 +1222,9 @@ func (a *minTimestampWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1170,6 +1246,7 @@ func (a *minTimestampWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1178,10 +1255,9 @@ func (a *minTimestampWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1203,6 +1279,7 @@ func (a *minTimestampWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1216,15 +1293,18 @@ func (a *minTimestampWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *minTimestampWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minTimestampWindowAggAlloc struct { @@ -1248,17 +1328,25 @@ func (a *minTimestampWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minTimestampWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minTimestampWindowAgg")) +} + type minIntervalWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Durations // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg duration.Duration - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minIntervalWindowAgg{} @@ -1284,10 +1372,9 @@ func (a *minIntervalWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1302,6 +1389,7 @@ func (a *minIntervalWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1310,10 +1398,9 @@ func (a *minIntervalWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1328,6 +1415,7 @@ func (a *minIntervalWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1341,15 +1429,18 @@ func (a *minIntervalWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *minIntervalWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type minIntervalWindowAggAlloc struct { @@ -1373,17 +1464,25 @@ func (a *minIntervalWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minIntervalWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minIntervalWindowAgg")) +} + type minJSONWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col *coldata.JSONs // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg json.JSON - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minJSONWindowAgg{} @@ -1412,7 +1511,7 @@ func (a *minJSONWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -1426,7 +1525,6 @@ func (a *minJSONWindowAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1458,6 +1556,7 @@ func (a *minJSONWindowAgg) Compute( } } + a.numNonNull++ } } } else { @@ -1466,7 +1565,7 @@ func (a *minJSONWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -1480,7 +1579,6 @@ func (a *minJSONWindowAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1512,6 +1610,7 @@ func (a *minJSONWindowAgg) Compute( } } + a.numNonNull++ } } } @@ -1528,24 +1627,38 @@ func (a *minJSONWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + + var _err error + var _bytes []byte + _bytes, _err = json.EncodeJSON(nil, a.curAgg) + if _err != nil { + colexecerror.ExpectedError(_err) + } + a.curAgg, _err = json.FromEncoding(_bytes) + if _err != nil { + colexecerror.ExpectedError(_err) + } + a.col.Set(outputIdx, a.curAgg) } +} + +func (a *minJSONWindowAgg) Reset() { + a.numNonNull = 0 var oldCurAggSize uintptr if a.curAgg != nil { oldCurAggSize = a.curAgg.Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } -func (a *minJSONWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false -} - type minJSONWindowAggAlloc struct { aggAllocBase aggFuncs []minJSONWindowAgg @@ -1567,17 +1680,25 @@ func (a *minJSONWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minJSONWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minJSONWindowAgg")) +} + type minDatumWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.DatumVec // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg interface{} - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &minDatumWindowAgg{} @@ -1607,10 +1728,9 @@ func (a *minDatumWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1627,6 +1747,7 @@ func (a *minDatumWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1635,10 +1756,9 @@ func (a *minDatumWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1655,6 +1775,7 @@ func (a *minDatumWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1672,25 +1793,28 @@ func (a *minDatumWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } +} + +func (a *minDatumWindowAgg) Reset() { + a.numNonNull = 0 var oldCurAggSize uintptr if a.curAgg != nil { oldCurAggSize = a.curAgg.(tree.Datum).Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } -func (a *minDatumWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false -} - type minDatumWindowAggAlloc struct { aggAllocBase aggFuncs []minDatumWindowAgg @@ -1712,6 +1836,14 @@ func (a *minDatumWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*minDatumWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on minDatumWindowAgg")) +} + func newMaxWindowAggAlloc( allocator *colmem.Allocator, t *types.T, allocSize int64, ) aggregateFuncAlloc { @@ -1787,11 +1919,11 @@ type maxBoolWindowAgg struct { col coldata.Bools // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg bool - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxBoolWindowAgg{} @@ -1817,10 +1949,9 @@ func (a *maxBoolWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1843,6 +1974,7 @@ func (a *maxBoolWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -1851,10 +1983,9 @@ func (a *maxBoolWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1877,6 +2008,7 @@ func (a *maxBoolWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -1890,15 +2022,18 @@ func (a *maxBoolWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *maxBoolWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxBoolWindowAggAlloc struct { @@ -1922,17 +2057,25 @@ func (a *maxBoolWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxBoolWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxBoolWindowAgg")) +} + type maxBytesWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col *coldata.Bytes // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg []byte - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxBytesWindowAgg{} @@ -1958,10 +2101,9 @@ func (a *maxBytesWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -1976,6 +2118,7 @@ func (a *maxBytesWindowAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } else { @@ -1984,10 +2127,9 @@ func (a *maxBytesWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = append(a.curAgg[:0], val...) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2002,6 +2144,7 @@ func (a *maxBytesWindowAgg) Compute( a.curAgg = append(a.curAgg[:0], candidate...) } } + a.numNonNull++ } } } @@ -2015,19 +2158,22 @@ func (a *maxBytesWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = append(a.curAgg[:0], a.curAgg...) a.col.Set(outputIdx, a.curAgg) } - oldCurAggSize := len(a.curAgg) - // Release the reference to curAgg eagerly. - a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) - a.curAgg = nil } func (a *maxBytesWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 + oldCurAggSize := len(a.curAgg) + // Release the reference to curAgg. + a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) + a.curAgg = nil } type maxBytesWindowAggAlloc struct { @@ -2051,17 +2197,25 @@ func (a *maxBytesWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxBytesWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxBytesWindowAgg")) +} + type maxDecimalWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Decimals // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxDecimalWindowAgg{} @@ -2087,10 +2241,9 @@ func (a *maxDecimalWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2105,6 +2258,7 @@ func (a *maxDecimalWindowAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } else { @@ -2113,10 +2267,9 @@ func (a *maxDecimalWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg.Set(&val) - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2131,6 +2284,7 @@ func (a *maxDecimalWindowAgg) Compute( a.curAgg.Set(&candidate) } } + a.numNonNull++ } } } @@ -2144,15 +2298,18 @@ func (a *maxDecimalWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg.Set(&a.curAgg) a.col.Set(outputIdx, a.curAgg) } } func (a *maxDecimalWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxDecimalWindowAggAlloc struct { @@ -2176,17 +2333,25 @@ func (a *maxDecimalWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxDecimalWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxDecimalWindowAgg")) +} + type maxInt16WindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Int16s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int16 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxInt16WindowAgg{} @@ -2212,10 +2377,9 @@ func (a *maxInt16WindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2241,6 +2405,7 @@ func (a *maxInt16WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2249,10 +2414,9 @@ func (a *maxInt16WindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2278,6 +2442,7 @@ func (a *maxInt16WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2291,15 +2456,18 @@ func (a *maxInt16WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *maxInt16WindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxInt16WindowAggAlloc struct { @@ -2323,17 +2491,25 @@ func (a *maxInt16WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxInt16WindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxInt16WindowAgg")) +} + type maxInt32WindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Int32s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int32 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxInt32WindowAgg{} @@ -2359,10 +2535,9 @@ func (a *maxInt32WindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2388,6 +2563,7 @@ func (a *maxInt32WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2396,10 +2572,9 @@ func (a *maxInt32WindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2425,6 +2600,7 @@ func (a *maxInt32WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2438,15 +2614,18 @@ func (a *maxInt32WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *maxInt32WindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxInt32WindowAggAlloc struct { @@ -2470,17 +2649,25 @@ func (a *maxInt32WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxInt32WindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxInt32WindowAgg")) +} + type maxInt64WindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Int64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg int64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxInt64WindowAgg{} @@ -2506,10 +2693,9 @@ func (a *maxInt64WindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2535,6 +2721,7 @@ func (a *maxInt64WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2543,10 +2730,9 @@ func (a *maxInt64WindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2572,6 +2758,7 @@ func (a *maxInt64WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2585,15 +2772,18 @@ func (a *maxInt64WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *maxInt64WindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxInt64WindowAggAlloc struct { @@ -2617,17 +2807,25 @@ func (a *maxInt64WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxInt64WindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxInt64WindowAgg")) +} + type maxFloat64WindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Float64s // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg float64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxFloat64WindowAgg{} @@ -2653,10 +2851,9 @@ func (a *maxFloat64WindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2690,6 +2887,7 @@ func (a *maxFloat64WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2698,10 +2896,9 @@ func (a *maxFloat64WindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2735,6 +2932,7 @@ func (a *maxFloat64WindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2748,15 +2946,18 @@ func (a *maxFloat64WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *maxFloat64WindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxFloat64WindowAggAlloc struct { @@ -2780,17 +2981,25 @@ func (a *maxFloat64WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxFloat64WindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxFloat64WindowAgg")) +} + type maxTimestampWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Times // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg time.Time - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxTimestampWindowAgg{} @@ -2816,10 +3025,9 @@ func (a *maxTimestampWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2841,6 +3049,7 @@ func (a *maxTimestampWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2849,10 +3058,9 @@ func (a *maxTimestampWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2874,6 +3082,7 @@ func (a *maxTimestampWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -2887,15 +3096,18 @@ func (a *maxTimestampWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *maxTimestampWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxTimestampWindowAggAlloc struct { @@ -2919,17 +3131,25 @@ func (a *maxTimestampWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxTimestampWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxTimestampWindowAgg")) +} + type maxIntervalWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.Durations // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg duration.Duration - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxIntervalWindowAgg{} @@ -2955,10 +3175,9 @@ func (a *maxIntervalWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2973,6 +3192,7 @@ func (a *maxIntervalWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -2981,10 +3201,9 @@ func (a *maxIntervalWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -2999,6 +3218,7 @@ func (a *maxIntervalWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -3012,15 +3232,18 @@ func (a *maxIntervalWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } } func (a *maxIntervalWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type maxIntervalWindowAggAlloc struct { @@ -3044,17 +3267,25 @@ func (a *maxIntervalWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxIntervalWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxIntervalWindowAgg")) +} + type maxJSONWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col *coldata.JSONs // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg json.JSON - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxJSONWindowAgg{} @@ -3083,7 +3314,7 @@ func (a *maxJSONWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -3097,7 +3328,6 @@ func (a *maxJSONWindowAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3129,6 +3359,7 @@ func (a *maxJSONWindowAgg) Compute( } } + a.numNonNull++ } } } else { @@ -3137,7 +3368,7 @@ func (a *maxJSONWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) var _err error @@ -3151,7 +3382,6 @@ func (a *maxJSONWindowAgg) Compute( colexecerror.ExpectedError(_err) } - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3183,6 +3413,7 @@ func (a *maxJSONWindowAgg) Compute( } } + a.numNonNull++ } } } @@ -3199,24 +3430,38 @@ func (a *maxJSONWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + + var _err error + var _bytes []byte + _bytes, _err = json.EncodeJSON(nil, a.curAgg) + if _err != nil { + colexecerror.ExpectedError(_err) + } + a.curAgg, _err = json.FromEncoding(_bytes) + if _err != nil { + colexecerror.ExpectedError(_err) + } + a.col.Set(outputIdx, a.curAgg) } +} + +func (a *maxJSONWindowAgg) Reset() { + a.numNonNull = 0 var oldCurAggSize uintptr if a.curAgg != nil { oldCurAggSize = a.curAgg.Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } -func (a *maxJSONWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false -} - type maxJSONWindowAggAlloc struct { aggAllocBase aggFuncs []maxJSONWindowAgg @@ -3238,17 +3483,25 @@ func (a *maxJSONWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxJSONWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxJSONWindowAgg")) +} + type maxDatumWindowAgg struct { unorderedAggregateFuncBase // col points to the output vector we are updating. col coldata.DatumVec // curAgg holds the running min/max, so we can index into the slice once per // group, instead of on each iteration. - // NOTE: if foundNonNullForCurrentGroup is false, curAgg is undefined. + // NOTE: if numNonNull is zero, curAgg is undefined. curAgg interface{} - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &maxDatumWindowAgg{} @@ -3278,10 +3531,9 @@ func (a *maxDatumWindowAgg) Compute( var isNull bool isNull = nulls.NullAt(i) if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3298,6 +3550,7 @@ func (a *maxDatumWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } else { @@ -3306,10 +3559,9 @@ func (a *maxDatumWindowAgg) Compute( var isNull bool isNull = false if !isNull { - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { val := col.Get(i) a.curAgg = val - a.foundNonNullForCurrentGroup = true } else { var cmp bool candidate := col.Get(i) @@ -3326,6 +3578,7 @@ func (a *maxDatumWindowAgg) Compute( a.curAgg = candidate } } + a.numNonNull++ } } } @@ -3343,25 +3596,28 @@ func (a *maxDatumWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should // be null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.curAgg = a.curAgg a.col.Set(outputIdx, a.curAgg) } +} + +func (a *maxDatumWindowAgg) Reset() { + a.numNonNull = 0 var oldCurAggSize uintptr if a.curAgg != nil { oldCurAggSize = a.curAgg.(tree.Datum).Size() } - // Release the reference to curAgg eagerly. + // Release the reference to curAgg. a.allocator.AdjustMemoryUsage(-int64(oldCurAggSize)) a.curAgg = nil } -func (a *maxDatumWindowAgg) Reset() { - a.foundNonNullForCurrentGroup = false -} - type maxDatumWindowAggAlloc struct { aggAllocBase aggFuncs []maxDatumWindowAgg @@ -3382,3 +3638,11 @@ func (a *maxDatumWindowAggAlloc) newAggFunc() AggregateFunc { a.aggFuncs = a.aggFuncs[1:] return f } + +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). This allows min and max operators to be used when +// the window frame only grows. For the case when the window frame can shrink, +// a specialized implementation is needed (see min_max_removable_agg_tmpl.go). +func (*maxDatumWindowAgg) Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) { + colexecerror.InternalError(errors.AssertionFailedf("Remove called on maxDatumWindowAgg")) +} diff --git a/pkg/sql/colexec/colexecagg/window_sum_agg.eg.go b/pkg/sql/colexec/colexecagg/window_sum_agg.eg.go index a6dd72037cf1..1622ba2d9b2d 100644 --- a/pkg/sql/colexec/colexecagg/window_sum_agg.eg.go +++ b/pkg/sql/colexec/colexecagg/window_sum_agg.eg.go @@ -75,10 +75,10 @@ type sumInt16WindowAgg struct { curAgg apd.Decimal // col points to the output vector we are updating. col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool - overloadHelper execgen.OverloadHelper + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 + overloadHelper execgen.OverloadHelper } var _ AggregateFunc = &sumInt16WindowAgg{} @@ -119,7 +119,7 @@ func (a *sumInt16WindowAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -140,7 +140,7 @@ func (a *sumInt16WindowAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -154,16 +154,18 @@ func (a *sumInt16WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { - a.col[outputIdx] = a.curAgg + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.col[outputIdx].Set(&a.curAgg) } } func (a *sumInt16WindowAgg) Reset() { a.curAgg = zeroDecimalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumInt16WindowAggAlloc struct { @@ -187,6 +189,67 @@ func (a *sumInt16WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). +func (a *sumInt16WindowAgg) Remove( + vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int, +) { + // In order to inline the templated code of overloads, we need to have a + // "_overloadHelper" local variable of type "overloadHelper". + _overloadHelper := a.overloadHelper + oldCurAggSize := tree.SizeOfDecimal(&a.curAgg) + vec := vecs[inputIdxs[0]] + col, nulls := vec.Int16(), vec.Nulls() + _, _ = col.Get(endIdx-1), col.Get(startIdx) + if nulls.MaybeHasNulls() { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = nulls.NullAt(i) + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + + tmpDec := &_overloadHelper.TmpDec1 + tmpDec.SetInt64(int64(v)) + if _, err := tree.ExactCtx.Sub(&a.curAgg, &a.curAgg, tmpDec); err != nil { + colexecerror.ExpectedError(err) + } + } + + a.numNonNull-- + } + } + } else { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = false + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + + tmpDec := &_overloadHelper.TmpDec1 + tmpDec.SetInt64(int64(v)) + if _, err := tree.ExactCtx.Sub(&a.curAgg, &a.curAgg, tmpDec); err != nil { + colexecerror.ExpectedError(err) + } + } + + a.numNonNull-- + } + } + } + newCurAggSize := tree.SizeOfDecimal(&a.curAgg) + if newCurAggSize != oldCurAggSize { + a.allocator.AdjustMemoryUsage(int64(newCurAggSize - oldCurAggSize)) + } +} + type sumInt32WindowAgg struct { unorderedAggregateFuncBase // curAgg holds the running total, so we can index into the slice once per @@ -194,10 +257,10 @@ type sumInt32WindowAgg struct { curAgg apd.Decimal // col points to the output vector we are updating. col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool - overloadHelper execgen.OverloadHelper + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 + overloadHelper execgen.OverloadHelper } var _ AggregateFunc = &sumInt32WindowAgg{} @@ -238,7 +301,7 @@ func (a *sumInt32WindowAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -259,7 +322,7 @@ func (a *sumInt32WindowAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -273,16 +336,18 @@ func (a *sumInt32WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { - a.col[outputIdx] = a.curAgg + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.col[outputIdx].Set(&a.curAgg) } } func (a *sumInt32WindowAgg) Reset() { a.curAgg = zeroDecimalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumInt32WindowAggAlloc struct { @@ -306,6 +371,67 @@ func (a *sumInt32WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). +func (a *sumInt32WindowAgg) Remove( + vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int, +) { + // In order to inline the templated code of overloads, we need to have a + // "_overloadHelper" local variable of type "overloadHelper". + _overloadHelper := a.overloadHelper + oldCurAggSize := tree.SizeOfDecimal(&a.curAgg) + vec := vecs[inputIdxs[0]] + col, nulls := vec.Int32(), vec.Nulls() + _, _ = col.Get(endIdx-1), col.Get(startIdx) + if nulls.MaybeHasNulls() { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = nulls.NullAt(i) + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + + tmpDec := &_overloadHelper.TmpDec1 + tmpDec.SetInt64(int64(v)) + if _, err := tree.ExactCtx.Sub(&a.curAgg, &a.curAgg, tmpDec); err != nil { + colexecerror.ExpectedError(err) + } + } + + a.numNonNull-- + } + } + } else { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = false + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + + tmpDec := &_overloadHelper.TmpDec1 + tmpDec.SetInt64(int64(v)) + if _, err := tree.ExactCtx.Sub(&a.curAgg, &a.curAgg, tmpDec); err != nil { + colexecerror.ExpectedError(err) + } + } + + a.numNonNull-- + } + } + } + newCurAggSize := tree.SizeOfDecimal(&a.curAgg) + if newCurAggSize != oldCurAggSize { + a.allocator.AdjustMemoryUsage(int64(newCurAggSize - oldCurAggSize)) + } +} + type sumInt64WindowAgg struct { unorderedAggregateFuncBase // curAgg holds the running total, so we can index into the slice once per @@ -313,10 +439,10 @@ type sumInt64WindowAgg struct { curAgg apd.Decimal // col points to the output vector we are updating. col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool - overloadHelper execgen.OverloadHelper + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 + overloadHelper execgen.OverloadHelper } var _ AggregateFunc = &sumInt64WindowAgg{} @@ -357,7 +483,7 @@ func (a *sumInt64WindowAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -378,7 +504,7 @@ func (a *sumInt64WindowAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -392,16 +518,18 @@ func (a *sumInt64WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { - a.col[outputIdx] = a.curAgg + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.col[outputIdx].Set(&a.curAgg) } } func (a *sumInt64WindowAgg) Reset() { a.curAgg = zeroDecimalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumInt64WindowAggAlloc struct { @@ -425,6 +553,67 @@ func (a *sumInt64WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). +func (a *sumInt64WindowAgg) Remove( + vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int, +) { + // In order to inline the templated code of overloads, we need to have a + // "_overloadHelper" local variable of type "overloadHelper". + _overloadHelper := a.overloadHelper + oldCurAggSize := tree.SizeOfDecimal(&a.curAgg) + vec := vecs[inputIdxs[0]] + col, nulls := vec.Int64(), vec.Nulls() + _, _ = col.Get(endIdx-1), col.Get(startIdx) + if nulls.MaybeHasNulls() { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = nulls.NullAt(i) + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + + tmpDec := &_overloadHelper.TmpDec1 + tmpDec.SetInt64(int64(v)) + if _, err := tree.ExactCtx.Sub(&a.curAgg, &a.curAgg, tmpDec); err != nil { + colexecerror.ExpectedError(err) + } + } + + a.numNonNull-- + } + } + } else { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = false + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + + tmpDec := &_overloadHelper.TmpDec1 + tmpDec.SetInt64(int64(v)) + if _, err := tree.ExactCtx.Sub(&a.curAgg, &a.curAgg, tmpDec); err != nil { + colexecerror.ExpectedError(err) + } + } + + a.numNonNull-- + } + } + } + newCurAggSize := tree.SizeOfDecimal(&a.curAgg) + if newCurAggSize != oldCurAggSize { + a.allocator.AdjustMemoryUsage(int64(newCurAggSize - oldCurAggSize)) + } +} + type sumDecimalWindowAgg struct { unorderedAggregateFuncBase // curAgg holds the running total, so we can index into the slice once per @@ -432,9 +621,9 @@ type sumDecimalWindowAgg struct { curAgg apd.Decimal // col points to the output vector we are updating. col []apd.Decimal - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &sumDecimalWindowAgg{} @@ -471,7 +660,7 @@ func (a *sumDecimalWindowAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -491,7 +680,7 @@ func (a *sumDecimalWindowAgg) Compute( } } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -505,16 +694,18 @@ func (a *sumDecimalWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { - a.col[outputIdx] = a.curAgg + // We need to copy the value because window functions reuse the aggregation + // between rows. + a.col[outputIdx].Set(&a.curAgg) } } func (a *sumDecimalWindowAgg) Reset() { a.curAgg = zeroDecimalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumDecimalWindowAggAlloc struct { @@ -538,6 +729,62 @@ func (a *sumDecimalWindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). +func (a *sumDecimalWindowAgg) Remove( + vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int, +) { + oldCurAggSize := tree.SizeOfDecimal(&a.curAgg) + vec := vecs[inputIdxs[0]] + col, nulls := vec.Decimal(), vec.Nulls() + _, _ = col.Get(endIdx-1), col.Get(startIdx) + if nulls.MaybeHasNulls() { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = nulls.NullAt(i) + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + + _, err := tree.ExactCtx.Sub(&a.curAgg, &a.curAgg, &v) + if err != nil { + colexecerror.ExpectedError(err) + } + } + + a.numNonNull-- + } + } + } else { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = false + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + + _, err := tree.ExactCtx.Sub(&a.curAgg, &a.curAgg, &v) + if err != nil { + colexecerror.ExpectedError(err) + } + } + + a.numNonNull-- + } + } + } + newCurAggSize := tree.SizeOfDecimal(&a.curAgg) + if newCurAggSize != oldCurAggSize { + a.allocator.AdjustMemoryUsage(int64(newCurAggSize - oldCurAggSize)) + } +} + type sumFloat64WindowAgg struct { unorderedAggregateFuncBase // curAgg holds the running total, so we can index into the slice once per @@ -545,9 +792,9 @@ type sumFloat64WindowAgg struct { curAgg float64 // col points to the output vector we are updating. col []float64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &sumFloat64WindowAgg{} @@ -581,7 +828,7 @@ func (a *sumFloat64WindowAgg) Compute( a.curAgg = float64(a.curAgg) + float64(v) } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -598,7 +845,7 @@ func (a *sumFloat64WindowAgg) Compute( a.curAgg = float64(a.curAgg) + float64(v) } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -612,16 +859,18 @@ func (a *sumFloat64WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. a.col[outputIdx] = a.curAgg } } func (a *sumFloat64WindowAgg) Reset() { a.curAgg = zeroFloat64Value - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumFloat64WindowAggAlloc struct { @@ -645,6 +894,56 @@ func (a *sumFloat64WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). +func (a *sumFloat64WindowAgg) Remove( + vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int, +) { + var oldCurAggSize uintptr + vec := vecs[inputIdxs[0]] + col, nulls := vec.Float64(), vec.Nulls() + _, _ = col.Get(endIdx-1), col.Get(startIdx) + if nulls.MaybeHasNulls() { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = nulls.NullAt(i) + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + + a.curAgg = float64(a.curAgg) - float64(v) + } + + a.numNonNull-- + } + } + } else { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = false + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + + a.curAgg = float64(a.curAgg) - float64(v) + } + + a.numNonNull-- + } + } + } + var newCurAggSize uintptr + if newCurAggSize != oldCurAggSize { + a.allocator.AdjustMemoryUsage(int64(newCurAggSize - oldCurAggSize)) + } +} + type sumIntervalWindowAgg struct { unorderedAggregateFuncBase // curAgg holds the running total, so we can index into the slice once per @@ -652,9 +951,9 @@ type sumIntervalWindowAgg struct { curAgg duration.Duration // col points to the output vector we are updating. col []duration.Duration - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &sumIntervalWindowAgg{} @@ -683,7 +982,7 @@ func (a *sumIntervalWindowAgg) Compute( //gcassert:bce v := col.Get(i) a.curAgg = a.curAgg.Add(v) - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -695,7 +994,7 @@ func (a *sumIntervalWindowAgg) Compute( //gcassert:bce v := col.Get(i) a.curAgg = a.curAgg.Add(v) - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -709,16 +1008,18 @@ func (a *sumIntervalWindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. a.col[outputIdx] = a.curAgg } } func (a *sumIntervalWindowAgg) Reset() { a.curAgg = zeroIntervalValue - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumIntervalWindowAggAlloc struct { @@ -741,3 +1042,43 @@ func (a *sumIntervalWindowAggAlloc) newAggFunc() AggregateFunc { a.aggFuncs = a.aggFuncs[1:] return f } + +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). +func (a *sumIntervalWindowAgg) Remove( + vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int, +) { + var oldCurAggSize uintptr + vec := vecs[inputIdxs[0]] + col, nulls := vec.Interval(), vec.Nulls() + _, _ = col.Get(endIdx-1), col.Get(startIdx) + if nulls.MaybeHasNulls() { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = nulls.NullAt(i) + if !isNull { + //gcassert:bce + v := col.Get(i) + a.curAgg = a.curAgg.Sub(v) + a.numNonNull-- + } + } + } else { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = false + if !isNull { + //gcassert:bce + v := col.Get(i) + a.curAgg = a.curAgg.Sub(v) + a.numNonNull-- + } + } + } + var newCurAggSize uintptr + if newCurAggSize != oldCurAggSize { + a.allocator.AdjustMemoryUsage(int64(newCurAggSize - oldCurAggSize)) + } +} diff --git a/pkg/sql/colexec/colexecagg/window_sum_int_agg.eg.go b/pkg/sql/colexec/colexecagg/window_sum_int_agg.eg.go index 8980608eab32..2049209e3bcb 100644 --- a/pkg/sql/colexec/colexecagg/window_sum_int_agg.eg.go +++ b/pkg/sql/colexec/colexecagg/window_sum_int_agg.eg.go @@ -56,9 +56,9 @@ type sumIntInt16WindowAgg struct { curAgg int64 // col points to the output vector we are updating. col []int64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &sumIntInt16WindowAgg{} @@ -95,7 +95,7 @@ func (a *sumIntInt16WindowAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -115,7 +115,7 @@ func (a *sumIntInt16WindowAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -129,16 +129,18 @@ func (a *sumIntInt16WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. a.col[outputIdx] = a.curAgg } } func (a *sumIntInt16WindowAgg) Reset() { a.curAgg = zeroInt64Value - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumIntInt16WindowAggAlloc struct { @@ -162,6 +164,62 @@ func (a *sumIntInt16WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). +func (a *sumIntInt16WindowAgg) Remove( + vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int, +) { + var oldCurAggSize uintptr + vec := vecs[inputIdxs[0]] + col, nulls := vec.Int16(), vec.Nulls() + _, _ = col.Get(endIdx-1), col.Get(startIdx) + if nulls.MaybeHasNulls() { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = nulls.NullAt(i) + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + result := int64(a.curAgg) - int64(v) + if (result < int64(a.curAgg)) != (int64(v) > 0) { + colexecerror.ExpectedError(tree.ErrIntOutOfRange) + } + a.curAgg = result + } + + a.numNonNull-- + } + } + } else { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = false + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + result := int64(a.curAgg) - int64(v) + if (result < int64(a.curAgg)) != (int64(v) > 0) { + colexecerror.ExpectedError(tree.ErrIntOutOfRange) + } + a.curAgg = result + } + + a.numNonNull-- + } + } + } + var newCurAggSize uintptr + if newCurAggSize != oldCurAggSize { + a.allocator.AdjustMemoryUsage(int64(newCurAggSize - oldCurAggSize)) + } +} + type sumIntInt32WindowAgg struct { unorderedAggregateFuncBase // curAgg holds the running total, so we can index into the slice once per @@ -169,9 +227,9 @@ type sumIntInt32WindowAgg struct { curAgg int64 // col points to the output vector we are updating. col []int64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &sumIntInt32WindowAgg{} @@ -208,7 +266,7 @@ func (a *sumIntInt32WindowAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -228,7 +286,7 @@ func (a *sumIntInt32WindowAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -242,16 +300,18 @@ func (a *sumIntInt32WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. a.col[outputIdx] = a.curAgg } } func (a *sumIntInt32WindowAgg) Reset() { a.curAgg = zeroInt64Value - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumIntInt32WindowAggAlloc struct { @@ -275,6 +335,62 @@ func (a *sumIntInt32WindowAggAlloc) newAggFunc() AggregateFunc { return f } +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). +func (a *sumIntInt32WindowAgg) Remove( + vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int, +) { + var oldCurAggSize uintptr + vec := vecs[inputIdxs[0]] + col, nulls := vec.Int32(), vec.Nulls() + _, _ = col.Get(endIdx-1), col.Get(startIdx) + if nulls.MaybeHasNulls() { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = nulls.NullAt(i) + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + result := int64(a.curAgg) - int64(v) + if (result < int64(a.curAgg)) != (int64(v) > 0) { + colexecerror.ExpectedError(tree.ErrIntOutOfRange) + } + a.curAgg = result + } + + a.numNonNull-- + } + } + } else { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = false + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + result := int64(a.curAgg) - int64(v) + if (result < int64(a.curAgg)) != (int64(v) > 0) { + colexecerror.ExpectedError(tree.ErrIntOutOfRange) + } + a.curAgg = result + } + + a.numNonNull-- + } + } + } + var newCurAggSize uintptr + if newCurAggSize != oldCurAggSize { + a.allocator.AdjustMemoryUsage(int64(newCurAggSize - oldCurAggSize)) + } +} + type sumIntInt64WindowAgg struct { unorderedAggregateFuncBase // curAgg holds the running total, so we can index into the slice once per @@ -282,9 +398,9 @@ type sumIntInt64WindowAgg struct { curAgg int64 // col points to the output vector we are updating. col []int64 - // foundNonNullForCurrentGroup tracks if we have seen any non-null values - // for the group that is currently being aggregated. - foundNonNullForCurrentGroup bool + // numNonNull tracks the number of non-null values we have seen for the group + // that is currently being aggregated. + numNonNull uint64 } var _ AggregateFunc = &sumIntInt64WindowAgg{} @@ -321,7 +437,7 @@ func (a *sumIntInt64WindowAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } else { @@ -341,7 +457,7 @@ func (a *sumIntInt64WindowAgg) Compute( a.curAgg = result } - a.foundNonNullForCurrentGroup = true + a.numNonNull++ } } } @@ -355,16 +471,18 @@ func (a *sumIntInt64WindowAgg) Flush(outputIdx int) { // The aggregation is finished. Flush the last value. If we haven't found // any non-nulls for this group so far, the output for this group should be // null. - if !a.foundNonNullForCurrentGroup { + if a.numNonNull == 0 { a.nulls.SetNull(outputIdx) } else { + // We need to copy the value because window functions reuse the aggregation + // between rows. a.col[outputIdx] = a.curAgg } } func (a *sumIntInt64WindowAgg) Reset() { a.curAgg = zeroInt64Value - a.foundNonNullForCurrentGroup = false + a.numNonNull = 0 } type sumIntInt64WindowAggAlloc struct { @@ -387,3 +505,59 @@ func (a *sumIntInt64WindowAggAlloc) newAggFunc() AggregateFunc { a.aggFuncs = a.aggFuncs[1:] return f } + +// Remove implements the slidingWindowAggregateFunc interface (see +// window_aggregator_tmpl.go). +func (a *sumIntInt64WindowAgg) Remove( + vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int, +) { + var oldCurAggSize uintptr + vec := vecs[inputIdxs[0]] + col, nulls := vec.Int64(), vec.Nulls() + _, _ = col.Get(endIdx-1), col.Get(startIdx) + if nulls.MaybeHasNulls() { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = nulls.NullAt(i) + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + result := int64(a.curAgg) - int64(v) + if (result < int64(a.curAgg)) != (int64(v) > 0) { + colexecerror.ExpectedError(tree.ErrIntOutOfRange) + } + a.curAgg = result + } + + a.numNonNull-- + } + } + } else { + for i := startIdx; i < endIdx; i++ { + + var isNull bool + isNull = false + if !isNull { + //gcassert:bce + v := col.Get(i) + + { + result := int64(a.curAgg) - int64(v) + if (result < int64(a.curAgg)) != (int64(v) > 0) { + colexecerror.ExpectedError(tree.ErrIntOutOfRange) + } + a.curAgg = result + } + + a.numNonNull-- + } + } + } + var newCurAggSize uintptr + if newCurAggSize != oldCurAggSize { + a.allocator.AdjustMemoryUsage(int64(newCurAggSize - oldCurAggSize)) + } +} diff --git a/pkg/sql/colexec/colexecwindow/BUILD.bazel b/pkg/sql/colexec/colexecwindow/BUILD.bazel index fedfe5587336..91ca8ea0ca5f 100644 --- a/pkg/sql/colexec/colexecwindow/BUILD.bazel +++ b/pkg/sql/colexec/colexecwindow/BUILD.bazel @@ -5,8 +5,9 @@ go_library( name = "colexecwindow", srcs = [ "buffered_window.go", + "count_rows_aggregator.go", + "min_max_queue.go", "partitioner.go", - "window_aggregator.go", "window_functions_util.go", ":gen-exec", # keep ], @@ -19,7 +20,7 @@ go_library( "//pkg/sql/catalog/descpb", "//pkg/sql/colcontainer", # keep "//pkg/sql/colconv", # keep - "//pkg/sql/colexec/colexecagg", + "//pkg/sql/colexec/colexecagg", # keep "//pkg/sql/colexec/colexecbase", "//pkg/sql/colexec/colexecutils", # keep "//pkg/sql/colexec/execgen", # keep @@ -28,6 +29,7 @@ go_library( "//pkg/sql/colmem", # keep "//pkg/sql/execinfra", # keep "//pkg/sql/execinfrapb", # keep + "//pkg/sql/memsize", # keep "//pkg/sql/randgen", "//pkg/sql/rowenc", # keep "//pkg/sql/sem/builtins", # keep @@ -35,6 +37,7 @@ go_library( "//pkg/sql/types", # keep "//pkg/util/duration", # keep "//pkg/util/encoding", + "//pkg/util/json", # keep "//pkg/util/mon", # keep "//pkg/util/timeutil/pgdate", # keep "@com_github_cockroachdb_apd_v2//:apd", # keep @@ -50,6 +53,7 @@ go_test( "dep_test.go", "inject_setup_test.go", "main_test.go", + "min_max_queue_test.go", "window_framer_test.go", "window_functions_test.go", ], @@ -94,12 +98,14 @@ targets = [ ("lag.eg.go", "lead_lag_tmpl.go"), ("last_value.eg.go", "first_last_nth_value_tmpl.go"), ("lead.eg.go", "lead_lag_tmpl.go"), + ("min_max_removable_agg.eg.go", "min_max_removable_agg_tmpl.go"), ("nth_value.eg.go", "first_last_nth_value_tmpl.go"), ("ntile.eg.go", "ntile_tmpl.go"), ("range_offset_handler.eg.go", "range_offset_handler_tmpl.go"), ("rank.eg.go", "rank_tmpl.go"), ("relative_rank.eg.go", "relative_rank_tmpl.go"), ("row_number.eg.go", "row_number_tmpl.go"), + ("window_aggregator.eg.go", "window_aggregator_tmpl.go"), ("window_framer.eg.go", "window_framer_tmpl.go"), ("window_peer_grouper.eg.go", "window_peer_grouper_tmpl.go"), ] diff --git a/pkg/sql/colexec/colexecwindow/buffered_window.go b/pkg/sql/colexec/colexecwindow/buffered_window.go index b5c30b79170a..71580e5f5d24 100644 --- a/pkg/sql/colexec/colexecwindow/buffered_window.go +++ b/pkg/sql/colexec/colexecwindow/buffered_window.go @@ -30,7 +30,7 @@ import ( // newBufferedWindowOperator creates a new Operator that computes the given // window function. func newBufferedWindowOperator( - args *WindowArgs, windower bufferedWindower, outputColType *types.T, + args *WindowArgs, windower bufferedWindower, outputColType *types.T, memoryLimit int64, ) colexecop.Operator { outputTypes := make([]*types.T, len(args.InputTypes), len(args.InputTypes)+1) copy(outputTypes, args.InputTypes) @@ -41,7 +41,7 @@ func newBufferedWindowOperator( windowInitFields: windowInitFields{ OneInputNode: colexecop.NewOneInputNode(input), allocator: args.MainAllocator, - memoryLimit: args.MemoryLimit, + memoryLimit: memoryLimit, diskQueueCfg: args.QueueCfg, fdSemaphore: args.FdSemaphore, outputTypes: outputTypes, diff --git a/pkg/sql/colexec/colexecwindow/count_rows_aggregator.go b/pkg/sql/colexec/colexecwindow/count_rows_aggregator.go new file mode 100644 index 000000000000..8ecc40a15137 --- /dev/null +++ b/pkg/sql/colexec/colexecwindow/count_rows_aggregator.go @@ -0,0 +1,107 @@ +// Copyright 2021 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 colexecwindow + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/types" +) + +// NewCountRowsOperator creates a new Operator that computes the count_rows +// aggregate window function. +func NewCountRowsOperator( + args *WindowArgs, frame *execinfrapb.WindowerSpec_Frame, ordering *execinfrapb.Ordering, +) colexecop.Operator { + // Because the buffer is potentially used multiple times per-row, it is + // important to prevent it from spilling to disk if possible. For this reason, + // we give the buffer half of the memory budget even though it will generally + // store less columns than the queue. + bufferMemLimit := int64(float64(args.MemoryLimit) * 0.5) + mainMemLimit := args.MemoryLimit - bufferMemLimit + framer := newWindowFramer(args.EvalCtx, frame, ordering, args.InputTypes, args.PeersColIdx) + colsToStore := framer.getColsToStore(nil /* oldColsToStore */) + buffer := colexecutils.NewSpillingBuffer( + args.BufferAllocator, bufferMemLimit, args.QueueCfg, + args.FdSemaphore, args.InputTypes, args.DiskAcc, colsToStore...) + windower := &countRowsWindowAggregator{ + partitionSeekerBase: partitionSeekerBase{ + partitionColIdx: args.PartitionColIdx, + buffer: buffer, + }, + allocator: args.MainAllocator, + outputColIdx: args.OutputColIdx, + framer: framer, + } + return newBufferedWindowOperator(args, windower, types.Int, mainMemLimit) +} + +type countRowsWindowAggregator struct { + partitionSeekerBase + colexecop.CloserHelper + allocator *colmem.Allocator + outputColIdx int + framer windowFramer +} + +var _ bufferedWindower = &countRowsWindowAggregator{} + +// transitionToProcessing implements the bufferedWindower interface. +func (a *countRowsWindowAggregator) transitionToProcessing() { + a.framer.startPartition(a.Ctx, a.partitionSize, a.buffer) +} + +// startNewPartition implements the bufferedWindower interface. +func (a *countRowsWindowAggregator) startNewPartition() { + a.partitionSize = 0 + a.buffer.Reset(a.Ctx) +} + +// Init implements the bufferedWindower interface. +func (a *countRowsWindowAggregator) Init(ctx context.Context) { + a.InitHelper.Init(ctx) +} + +// Close implements the bufferedWindower interface. +func (a *countRowsWindowAggregator) Close() { + if !a.CloserHelper.Close() { + return + } + a.framer.close() + a.buffer.Close(a.EnsureCtx()) +} + +// processBatch implements the bufferedWindower interface. +func (a *countRowsWindowAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if startIdx >= endIdx { + // No processing needs to be performed. + return + } + outVec := batch.ColVec(a.outputColIdx) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + outCol := outVec.Int64() + _, _ = outCol[startIdx], outCol[endIdx-1] + for i := startIdx; i < endIdx; i++ { + var cnt int + a.framer.next(a.Ctx) + for _, interval := range a.framer.frameIntervals() { + cnt += interval.end - interval.start + } + //gcassert:bce + outCol[i] = int64(cnt) + } + }) +} diff --git a/pkg/sql/colexec/colexecwindow/first_last_nth_value_tmpl.go b/pkg/sql/colexec/colexecwindow/first_last_nth_value_tmpl.go index 83ec21f55c59..4a7b3f776ae4 100644 --- a/pkg/sql/colexec/colexecwindow/first_last_nth_value_tmpl.go +++ b/pkg/sql/colexec/colexecwindow/first_last_nth_value_tmpl.go @@ -64,6 +64,7 @@ func New_UPPERCASE_NAMEOperator( // store a single column. TODO(drewk): play around with benchmarks to find a // good empirically-supported fraction to use. bufferMemLimit := int64(float64(args.MemoryLimit) * 0.10) + mainMemLimit := args.MemoryLimit - bufferMemLimit buffer := colexecutils.NewSpillingBuffer( args.BufferAllocator, bufferMemLimit, args.QueueCfg, args.FdSemaphore, args.InputTypes, args.DiskAcc, colsToStore...) @@ -87,7 +88,7 @@ func New_UPPERCASE_NAMEOperator( // {{if .IsNthValue}} windower.nColIdx = argIdxs[1] // {{end}} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil // {{end}} } // {{end}} diff --git a/pkg/sql/colexec/colexecwindow/first_value.eg.go b/pkg/sql/colexec/colexecwindow/first_value.eg.go index 748e6c0d0609..7182dc35b648 100644 --- a/pkg/sql/colexec/colexecwindow/first_value.eg.go +++ b/pkg/sql/colexec/colexecwindow/first_value.eg.go @@ -39,6 +39,7 @@ func NewFirstValueOperator( // store a single column. TODO(drewk): play around with benchmarks to find a // good empirically-supported fraction to use. bufferMemLimit := int64(float64(args.MemoryLimit) * 0.10) + mainMemLimit := args.MemoryLimit - bufferMemLimit buffer := colexecutils.NewSpillingBuffer( args.BufferAllocator, bufferMemLimit, args.QueueCfg, args.FdSemaphore, args.InputTypes, args.DiskAcc, colsToStore...) @@ -58,69 +59,69 @@ func NewFirstValueOperator( case -1: default: windower := &firstValueBoolWindow{firstValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case types.BytesFamily: switch argType.Width() { case -1: default: windower := &firstValueBytesWindow{firstValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case types.DecimalFamily: switch argType.Width() { case -1: default: windower := &firstValueDecimalWindow{firstValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case types.IntFamily: switch argType.Width() { case 16: windower := &firstValueInt16Window{firstValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil case 32: windower := &firstValueInt32Window{firstValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil case -1: default: windower := &firstValueInt64Window{firstValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case types.FloatFamily: switch argType.Width() { case -1: default: windower := &firstValueFloat64Window{firstValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case types.TimestampTZFamily: switch argType.Width() { case -1: default: windower := &firstValueTimestampWindow{firstValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case types.IntervalFamily: switch argType.Width() { case -1: default: windower := &firstValueIntervalWindow{firstValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case types.JsonFamily: switch argType.Width() { case -1: default: windower := &firstValueJSONWindow{firstValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case typeconv.DatumVecCanonicalTypeFamily: switch argType.Width() { case -1: default: windower := &firstValueDatumWindow{firstValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } } return nil, errors.Errorf("unsupported firstValue window operator type %s", argType.Name()) diff --git a/pkg/sql/colexec/colexecwindow/lag.eg.go b/pkg/sql/colexec/colexecwindow/lag.eg.go index 0e57bd8c885a..49c474259aa3 100644 --- a/pkg/sql/colexec/colexecwindow/lag.eg.go +++ b/pkg/sql/colexec/colexecwindow/lag.eg.go @@ -32,6 +32,7 @@ func NewLagOperator( // store a single column. TODO(drewk): play around with benchmarks to find a // good empirically-supported fraction to use. bufferMemLimit := int64(float64(args.MemoryLimit) * 0.10) + mainMemLimit := args.MemoryLimit - bufferMemLimit buffer := colexecutils.NewSpillingBuffer( args.BufferAllocator, bufferMemLimit, args.QueueCfg, args.FdSemaphore, args.InputTypes, args.DiskAcc, argIdx) @@ -51,59 +52,70 @@ func NewLagOperator( switch argType.Width() { case -1: default: - return newBufferedWindowOperator(args, &lagBoolWindow{lagBase: base}, argType), nil + return newBufferedWindowOperator( + args, &lagBoolWindow{lagBase: base}, argType, mainMemLimit), nil } case types.BytesFamily: switch argType.Width() { case -1: default: - return newBufferedWindowOperator(args, &lagBytesWindow{lagBase: base}, argType), nil + return newBufferedWindowOperator( + args, &lagBytesWindow{lagBase: base}, argType, mainMemLimit), nil } case types.DecimalFamily: switch argType.Width() { case -1: default: - return newBufferedWindowOperator(args, &lagDecimalWindow{lagBase: base}, argType), nil + return newBufferedWindowOperator( + args, &lagDecimalWindow{lagBase: base}, argType, mainMemLimit), nil } case types.IntFamily: switch argType.Width() { case 16: - return newBufferedWindowOperator(args, &lagInt16Window{lagBase: base}, argType), nil + return newBufferedWindowOperator( + args, &lagInt16Window{lagBase: base}, argType, mainMemLimit), nil case 32: - return newBufferedWindowOperator(args, &lagInt32Window{lagBase: base}, argType), nil + return newBufferedWindowOperator( + args, &lagInt32Window{lagBase: base}, argType, mainMemLimit), nil case -1: default: - return newBufferedWindowOperator(args, &lagInt64Window{lagBase: base}, argType), nil + return newBufferedWindowOperator( + args, &lagInt64Window{lagBase: base}, argType, mainMemLimit), nil } case types.FloatFamily: switch argType.Width() { case -1: default: - return newBufferedWindowOperator(args, &lagFloat64Window{lagBase: base}, argType), nil + return newBufferedWindowOperator( + args, &lagFloat64Window{lagBase: base}, argType, mainMemLimit), nil } case types.TimestampTZFamily: switch argType.Width() { case -1: default: - return newBufferedWindowOperator(args, &lagTimestampWindow{lagBase: base}, argType), nil + return newBufferedWindowOperator( + args, &lagTimestampWindow{lagBase: base}, argType, mainMemLimit), nil } case types.IntervalFamily: switch argType.Width() { case -1: default: - return newBufferedWindowOperator(args, &lagIntervalWindow{lagBase: base}, argType), nil + return newBufferedWindowOperator( + args, &lagIntervalWindow{lagBase: base}, argType, mainMemLimit), nil } case types.JsonFamily: switch argType.Width() { case -1: default: - return newBufferedWindowOperator(args, &lagJSONWindow{lagBase: base}, argType), nil + return newBufferedWindowOperator( + args, &lagJSONWindow{lagBase: base}, argType, mainMemLimit), nil } case typeconv.DatumVecCanonicalTypeFamily: switch argType.Width() { case -1: default: - return newBufferedWindowOperator(args, &lagDatumWindow{lagBase: base}, argType), nil + return newBufferedWindowOperator( + args, &lagDatumWindow{lagBase: base}, argType, mainMemLimit), nil } } return nil, errors.Errorf("unsupported lag window operator type %s", argType.Name()) diff --git a/pkg/sql/colexec/colexecwindow/last_value.eg.go b/pkg/sql/colexec/colexecwindow/last_value.eg.go index 9a15b99920a9..594c16d99059 100644 --- a/pkg/sql/colexec/colexecwindow/last_value.eg.go +++ b/pkg/sql/colexec/colexecwindow/last_value.eg.go @@ -39,6 +39,7 @@ func NewLastValueOperator( // store a single column. TODO(drewk): play around with benchmarks to find a // good empirically-supported fraction to use. bufferMemLimit := int64(float64(args.MemoryLimit) * 0.10) + mainMemLimit := args.MemoryLimit - bufferMemLimit buffer := colexecutils.NewSpillingBuffer( args.BufferAllocator, bufferMemLimit, args.QueueCfg, args.FdSemaphore, args.InputTypes, args.DiskAcc, colsToStore...) @@ -58,69 +59,69 @@ func NewLastValueOperator( case -1: default: windower := &lastValueBoolWindow{lastValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case types.BytesFamily: switch argType.Width() { case -1: default: windower := &lastValueBytesWindow{lastValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case types.DecimalFamily: switch argType.Width() { case -1: default: windower := &lastValueDecimalWindow{lastValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case types.IntFamily: switch argType.Width() { case 16: windower := &lastValueInt16Window{lastValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil case 32: windower := &lastValueInt32Window{lastValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil case -1: default: windower := &lastValueInt64Window{lastValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case types.FloatFamily: switch argType.Width() { case -1: default: windower := &lastValueFloat64Window{lastValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case types.TimestampTZFamily: switch argType.Width() { case -1: default: windower := &lastValueTimestampWindow{lastValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case types.IntervalFamily: switch argType.Width() { case -1: default: windower := &lastValueIntervalWindow{lastValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case types.JsonFamily: switch argType.Width() { case -1: default: windower := &lastValueJSONWindow{lastValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case typeconv.DatumVecCanonicalTypeFamily: switch argType.Width() { case -1: default: windower := &lastValueDatumWindow{lastValueBase: base} - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } } return nil, errors.Errorf("unsupported lastValue window operator type %s", argType.Name()) diff --git a/pkg/sql/colexec/colexecwindow/lead.eg.go b/pkg/sql/colexec/colexecwindow/lead.eg.go index e4e3c57cc4f2..bf3fbcff723a 100644 --- a/pkg/sql/colexec/colexecwindow/lead.eg.go +++ b/pkg/sql/colexec/colexecwindow/lead.eg.go @@ -32,6 +32,7 @@ func NewLeadOperator( // store a single column. TODO(drewk): play around with benchmarks to find a // good empirically-supported fraction to use. bufferMemLimit := int64(float64(args.MemoryLimit) * 0.10) + mainMemLimit := args.MemoryLimit - bufferMemLimit buffer := colexecutils.NewSpillingBuffer( args.BufferAllocator, bufferMemLimit, args.QueueCfg, args.FdSemaphore, args.InputTypes, args.DiskAcc, argIdx) @@ -51,59 +52,70 @@ func NewLeadOperator( switch argType.Width() { case -1: default: - return newBufferedWindowOperator(args, &leadBoolWindow{leadBase: base}, argType), nil + return newBufferedWindowOperator( + args, &leadBoolWindow{leadBase: base}, argType, mainMemLimit), nil } case types.BytesFamily: switch argType.Width() { case -1: default: - return newBufferedWindowOperator(args, &leadBytesWindow{leadBase: base}, argType), nil + return newBufferedWindowOperator( + args, &leadBytesWindow{leadBase: base}, argType, mainMemLimit), nil } case types.DecimalFamily: switch argType.Width() { case -1: default: - return newBufferedWindowOperator(args, &leadDecimalWindow{leadBase: base}, argType), nil + return newBufferedWindowOperator( + args, &leadDecimalWindow{leadBase: base}, argType, mainMemLimit), nil } case types.IntFamily: switch argType.Width() { case 16: - return newBufferedWindowOperator(args, &leadInt16Window{leadBase: base}, argType), nil + return newBufferedWindowOperator( + args, &leadInt16Window{leadBase: base}, argType, mainMemLimit), nil case 32: - return newBufferedWindowOperator(args, &leadInt32Window{leadBase: base}, argType), nil + return newBufferedWindowOperator( + args, &leadInt32Window{leadBase: base}, argType, mainMemLimit), nil case -1: default: - return newBufferedWindowOperator(args, &leadInt64Window{leadBase: base}, argType), nil + return newBufferedWindowOperator( + args, &leadInt64Window{leadBase: base}, argType, mainMemLimit), nil } case types.FloatFamily: switch argType.Width() { case -1: default: - return newBufferedWindowOperator(args, &leadFloat64Window{leadBase: base}, argType), nil + return newBufferedWindowOperator( + args, &leadFloat64Window{leadBase: base}, argType, mainMemLimit), nil } case types.TimestampTZFamily: switch argType.Width() { case -1: default: - return newBufferedWindowOperator(args, &leadTimestampWindow{leadBase: base}, argType), nil + return newBufferedWindowOperator( + args, &leadTimestampWindow{leadBase: base}, argType, mainMemLimit), nil } case types.IntervalFamily: switch argType.Width() { case -1: default: - return newBufferedWindowOperator(args, &leadIntervalWindow{leadBase: base}, argType), nil + return newBufferedWindowOperator( + args, &leadIntervalWindow{leadBase: base}, argType, mainMemLimit), nil } case types.JsonFamily: switch argType.Width() { case -1: default: - return newBufferedWindowOperator(args, &leadJSONWindow{leadBase: base}, argType), nil + return newBufferedWindowOperator( + args, &leadJSONWindow{leadBase: base}, argType, mainMemLimit), nil } case typeconv.DatumVecCanonicalTypeFamily: switch argType.Width() { case -1: default: - return newBufferedWindowOperator(args, &leadDatumWindow{leadBase: base}, argType), nil + return newBufferedWindowOperator( + args, &leadDatumWindow{leadBase: base}, argType, mainMemLimit), nil } } return nil, errors.Errorf("unsupported lead window operator type %s", argType.Name()) diff --git a/pkg/sql/colexec/colexecwindow/lead_lag_tmpl.go b/pkg/sql/colexec/colexecwindow/lead_lag_tmpl.go index 5059d0614bae..c8cb490004ab 100644 --- a/pkg/sql/colexec/colexecwindow/lead_lag_tmpl.go +++ b/pkg/sql/colexec/colexecwindow/lead_lag_tmpl.go @@ -54,6 +54,7 @@ func New_UPPERCASE_NAMEOperator( // store a single column. TODO(drewk): play around with benchmarks to find a // good empirically-supported fraction to use. bufferMemLimit := int64(float64(args.MemoryLimit) * 0.10) + mainMemLimit := args.MemoryLimit - bufferMemLimit buffer := colexecutils.NewSpillingBuffer( args.BufferAllocator, bufferMemLimit, args.QueueCfg, args.FdSemaphore, args.InputTypes, args.DiskAcc, argIdx) @@ -74,7 +75,8 @@ func New_UPPERCASE_NAMEOperator( switch argType.Width() { // {{range .WidthOverloads}} case _TYPE_WIDTH: - return newBufferedWindowOperator(args, &_OP_NAME_TYPEWindow{_OP_NAMEBase: base}, argType), nil + return newBufferedWindowOperator( + args, &_OP_NAME_TYPEWindow{_OP_NAMEBase: base}, argType, mainMemLimit), nil // {{end}} } // {{end}} diff --git a/pkg/sql/colexec/colexecwindow/min_max_queue.go b/pkg/sql/colexec/colexecwindow/min_max_queue.go new file mode 100644 index 000000000000..abe8715bb8e2 --- /dev/null +++ b/pkg/sql/colexec/colexecwindow/min_max_queue.go @@ -0,0 +1,177 @@ +// Copyright 2021 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 colexecwindow + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/errors" +) + +func newMinMaxQueue(maxLength int) minMaxQueue { + return minMaxQueue{maxLength: maxLength, empty: true} +} + +// minMaxQueue buffers uint32 values that are used by the min and max window +// functions to handle the case where the window frame for the current row need +// not include every row that was in the previous frame. minMaxQueue optimizes +// for operations on the start and end of the buffer, such as removing the last +// n values. +type minMaxQueue struct { + buffer []uint32 // an increasing circular buffer of values + head int // the index of the front of the buffer + tail int // the index of the first position after the end of the buffer + + // Indicates whether the buffer is empty. Necessary to distinguish between an + // empty buffer and a buffer that uses all of its capacity. + empty bool + + // The maximum size to which the minMaxQueue can grow. If reached, no new + // elements can be added to the minMaxQueue until some have been removed. + maxLength int +} + +// len returns the number of elements in the minMaxQueue. +// gcassert:inline +func (q *minMaxQueue) len() int { + if q.empty { + return 0 + } + if q.head < q.tail { + return q.tail - q.head + } + return cap(q.buffer) + q.tail - q.head +} + +// isEmpty returns true if the minMaxQueue has no elements. +// gcassert:inline +func (q *minMaxQueue) isEmpty() bool { + return q.empty +} + +// get returns the element at position pos in the minMaxQueue (zero-based). +// gcassert:inline +func (q *minMaxQueue) get(pos int) uint32 { + if q.empty || pos < 0 || pos >= q.len() { + colexecerror.InternalError(errors.AssertionFailedf("index out of bounds: %d", pos)) + } + return q.buffer[(pos+q.head)%cap(q.buffer)] +} + +// getFirst returns the element at the start of the minMaxQueue. +// gcassert:inline +func (q *minMaxQueue) getFirst() uint32 { + if q.empty { + colexecerror.InternalError(errors.AssertionFailedf("getting first from empty minMaxQueue")) + } + return q.buffer[q.head] +} + +// getLast returns the element at the end of the minMaxQueue. +// gcassert:inline +func (q *minMaxQueue) getLast() uint32 { + if q.empty { + colexecerror.InternalError(errors.AssertionFailedf("getting last from empty minMaxQueue")) + } + return q.buffer[(cap(q.buffer)+q.tail-1)%cap(q.buffer)] +} + +// addLast adds element to the end of the minMaxQueue and doubles it's +// underlying slice if necessary, subject to the max length limit. If the +// minMaxQueue has already reached the maximum length, addLast returns true, +// otherwise false. +// gcassert:inline +func (q *minMaxQueue) addLast(element uint32) (reachedLimit bool) { + if q.maybeGrow() { + return true + } + q.buffer[q.tail] = element + q.tail = (q.tail + 1) % cap(q.buffer) + q.empty = false + return false +} + +// removeLast removes a single element from the end of the minMaxQueue. +// gcassert:inline +func (q *minMaxQueue) removeLast() { + if q.empty { + colexecerror.InternalError(errors.AssertionFailedf("removing last from empty ring buffer")) + } + lastPos := (cap(q.buffer) + q.tail - 1) % cap(q.buffer) + q.tail = lastPos + if q.tail == q.head { + q.empty = true + } +} + +// removeAllBefore removes from the minMaxQueue all values in the range +// [0, val). +// gcassert:inline +func (q *minMaxQueue) removeAllBefore(val uint32) { + if q.empty { + return + } + var idx int + length := q.len() + for ; idx < length && q.get(idx) < val; idx++ { + } + if idx == length { + q.empty = true + } + q.head = (q.head + idx) % cap(q.buffer) +} + +func (q *minMaxQueue) grow(n int) { + newBuffer := make([]uint32, n) + if q.head < q.tail { + copy(newBuffer[:q.len()], q.buffer[q.head:q.tail]) + } else { + copy(newBuffer[:cap(q.buffer)-q.head], q.buffer[q.head:]) + copy(newBuffer[cap(q.buffer)-q.head:q.len()], q.buffer[:q.tail]) + } + q.head = 0 + q.tail = cap(q.buffer) + q.buffer = newBuffer +} + +// maybeGrow attempts to double the size of the minMaxQueue, capped at +// maxLength. If the minMaxQueue has already reached maxLength, returns true. +func (q *minMaxQueue) maybeGrow() (reachedLimit bool) { + if q.len() != cap(q.buffer) { + return false + } + if q.len() == q.maxLength { + return true + } + n := 2 * cap(q.buffer) + if n == 0 { + n = 1 + } + if n > q.maxLength { + n = q.maxLength + } + q.grow(n) + return false +} + +// Reset makes minMaxQueue treat its underlying memory as if it were empty. This +// allows for reusing the same memory again without explicitly removing old +// elements. +// gcassert:inline +func (q *minMaxQueue) reset() { + q.head = 0 + q.tail = 0 + q.empty = true +} + +// close releases the minMaxQueue's underlying memory. +func (q *minMaxQueue) close() { + q.buffer = nil +} diff --git a/pkg/sql/colexec/colexecwindow/min_max_queue_test.go b/pkg/sql/colexec/colexecwindow/min_max_queue_test.go new file mode 100644 index 000000000000..ceb48635942f --- /dev/null +++ b/pkg/sql/colexec/colexecwindow/min_max_queue_test.go @@ -0,0 +1,67 @@ +// Copyright 2021 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 colexecwindow + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/util/randutil" + "github.com/stretchr/testify/require" +) + +func TestMinMaxQueue(t *testing.T) { + const ( + chanceToRemove = 0.1 + numIterations = 10 + maxIncrement = 100 + maxValuesToAdd = 1000 + ) + + rng, _ := randutil.NewPseudoRand() + + var queue minMaxQueue + var oracle []uint32 + + for i := 0; i < numIterations; i++ { + oracle = oracle[:0] + queue.reset() + queue.maxLength = rng.Intn(maxValuesToAdd) + if cap(queue.buffer) > queue.maxLength { + // We have to nil out the buffer to ensure that the queue does not grow + // too large, since it will fill out all available capacity. + queue.buffer = nil + } + valuesToAdd := rng.Intn(maxValuesToAdd) + var num uint32 + for j := 0; j < valuesToAdd; j++ { + num += uint32(rng.Intn(maxIncrement)) + 1 // Ensure no duplicates. + if len(oracle) < queue.maxLength { + oracle = append(oracle, num) + } + queue.addLast(num) + if len(oracle) > 0 && rng.Float64() < chanceToRemove { + idx := rng.Intn(len(oracle)) + val := oracle[idx] + oracle = append(oracle[:0], oracle[idx:]...) + queue.removeAllBefore(val) + for k := 0; k < rng.Intn(len(oracle)); k++ { + oracle = oracle[:len(oracle)-1] + queue.removeLast() + } + } + } + require.LessOrEqual(t, queue.len(), queue.maxLength, "queue length exceeds maximum") + require.Equal(t, len(oracle), queue.len(), "expected equal lengths") + for j := 0; j < len(oracle); j++ { + require.Equalf(t, oracle[j], queue.get(j), "wrong value at index: %d", j) + } + } +} diff --git a/pkg/sql/colexec/colexecwindow/min_max_removable_agg.eg.go b/pkg/sql/colexec/colexecwindow/min_max_removable_agg.eg.go new file mode 100644 index 000000000000..b5468c8aec90 --- /dev/null +++ b/pkg/sql/colexec/colexecwindow/min_max_removable_agg.eg.go @@ -0,0 +1,3978 @@ +// Code generated by execgen; DO NOT EDIT. +// Copyright 2021 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 colexecwindow + +import ( + "bytes" + "context" + "math" + "time" + + "github.com/cockroachdb/apd/v2" + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/coldataext" + "github.com/cockroachdb/cockroach/pkg/col/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" + "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/memsize" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/cockroach/pkg/util/json" + "github.com/cockroachdb/errors" +) + +// Workaround for bazel auto-generated code. goimports does not automatically +// pick up the right packages when run within the bazel sandbox. +var ( + _ tree.AggType + _ apd.Context + _ duration.Duration + _ json.JSON + _ = coldataext.CompareDatum + _ = colexecerror.InternalError + _ = memsize.Uint32 +) + +const ( + // The argument column is always the first column in the SpillingBuffer. + argColIdx = 0 + + // The slice of uint32s in the deque can have up to 10,000 values (40KB). + maxQueueLength = 10000 +) + +type minMaxRemovableAggBase struct { + partitionSeekerBase + colexecop.CloserHelper + allocator *colmem.Allocator + outputColIdx int + framer windowFramer + + // A partial deque of indices into the current partition ordered by the value + // of the input column at each index. It contains only indices that are part + // of the current window frame. The first value in the queue is the index of + // the current value for the aggregation (NULL if empty). Under the + // simplifying assumption that the window frame has no exclusion clause, the + // queue does not need to contain any indices smaller than the best index - + // this keeps the queue small in many common cases. + queue minMaxQueue + + // omittedIndex tracks the index where we reached the limit of the length of + // the queue, in which case we may be omitting values that could become + // relevant as the frame shrinks. If the queue becomes empty while this + // index is set, we have to aggregate over the previously omitted values. + // The default (unset) value is -1. + omittedIndex int + + scratchIntervals []windowInterval +} + +// Init implements the bufferedWindower interface. +func (b *minMaxRemovableAggBase) Init(ctx context.Context) { + b.InitHelper.Init(ctx) +} + +// transitionToProcessing implements the bufferedWindower interface. +func (b *minMaxRemovableAggBase) transitionToProcessing() { + b.framer.startPartition(b.Ctx, b.partitionSize, b.buffer) +} + +// startNewPartition implements the bufferedWindower interface. +func (b *minMaxRemovableAggBase) startNewPartition() { + b.partitionSize = 0 + b.buffer.Reset(b.Ctx) + b.queue.reset() +} + +func newMinRemovableAggregator( + args *WindowArgs, framer windowFramer, buffer *colexecutils.SpillingBuffer, argTyp *types.T, +) bufferedWindower { + // Reserve the maximum memory usable by the queue up front to ensure that it + // isn't used by the SpillingBuffer. + args.BufferAllocator.AdjustMemoryUsage(maxQueueLength * memsize.Uint32) + base := minMaxRemovableAggBase{ + partitionSeekerBase: partitionSeekerBase{ + partitionColIdx: args.PartitionColIdx, + buffer: buffer, + }, + allocator: args.MainAllocator, + outputColIdx: args.OutputColIdx, + framer: framer, + queue: newMinMaxQueue(maxQueueLength), + omittedIndex: -1, + } + switch typeconv.TypeFamilyToCanonicalTypeFamily(argTyp.Family()) { + case types.BoolFamily: + switch argTyp.Width() { + case -1: + default: + return &minBoolAggregator{minMaxRemovableAggBase: base} + } + case types.BytesFamily: + switch argTyp.Width() { + case -1: + default: + return &minBytesAggregator{minMaxRemovableAggBase: base} + } + case types.DecimalFamily: + switch argTyp.Width() { + case -1: + default: + return &minDecimalAggregator{minMaxRemovableAggBase: base} + } + case types.IntFamily: + switch argTyp.Width() { + case 16: + return &minInt16Aggregator{minMaxRemovableAggBase: base} + case 32: + return &minInt32Aggregator{minMaxRemovableAggBase: base} + case -1: + default: + return &minInt64Aggregator{minMaxRemovableAggBase: base} + } + case types.FloatFamily: + switch argTyp.Width() { + case -1: + default: + return &minFloat64Aggregator{minMaxRemovableAggBase: base} + } + case types.TimestampTZFamily: + switch argTyp.Width() { + case -1: + default: + return &minTimestampAggregator{minMaxRemovableAggBase: base} + } + case types.IntervalFamily: + switch argTyp.Width() { + case -1: + default: + return &minIntervalAggregator{minMaxRemovableAggBase: base} + } + case types.JsonFamily: + switch argTyp.Width() { + case -1: + default: + return &minJSONAggregator{minMaxRemovableAggBase: base} + } + case typeconv.DatumVecCanonicalTypeFamily: + switch argTyp.Width() { + case -1: + default: + return &minDatumAggregator{minMaxRemovableAggBase: base} + } + } + colexecerror.InternalError( + errors.AssertionFailedf("unexpectedly didn't find min overload for %s type family", argTyp.Name())) + // This code is unreachable, but the compiler cannot infer that. + return nil +} + +type minBoolAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg bool +} + +// processBatch implements the bufferedWindower interface. +func (a *minBoolAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Bool() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Bool() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minBoolAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Bool() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + if !val && a.curAgg { + cmpResult = -1 + } else if val && !a.curAgg { + cmpResult = 1 + } else { + cmpResult = 0 + } + + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Bool().Get(cmpIdx) + + { + var cmpResult int + + if !cmpVal && val { + cmpResult = -1 + } else if cmpVal && !val { + cmpResult = 1 + } else { + cmpResult = 0 + } + + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minBoolAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minBoolAggregator{} +} + +type minBytesAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg []byte +} + +// processBatch implements the bufferedWindower interface. +func (a *minBytesAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Bytes() + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Bytes() + val := col.Get(idx) + a.curAgg = append(a.curAgg[:0], val...) + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = append(a.curAgg[:0], a.curAgg...) + + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minBytesAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Bytes() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + cmpResult = bytes.Compare(val, a.curAgg) + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = append(a.curAgg[:0], val...) + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Bytes().Get(cmpIdx) + + { + var cmpResult int + cmpResult = bytes.Compare(cmpVal, val) + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minBytesAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minBytesAggregator{} +} + +type minDecimalAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg apd.Decimal +} + +// processBatch implements the bufferedWindower interface. +func (a *minDecimalAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Decimal() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Decimal() + val := col.Get(idx) + a.curAgg.Set(&val) + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg.Set(&a.curAgg) + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minDecimalAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Decimal() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + cmpResult = tree.CompareDecimals(&val, &a.curAgg) + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg.Set(&val) + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Decimal().Get(cmpIdx) + + { + var cmpResult int + cmpResult = tree.CompareDecimals(&cmpVal, &val) + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minDecimalAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minDecimalAggregator{} +} + +type minInt16Aggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg int16 +} + +// processBatch implements the bufferedWindower interface. +func (a *minInt16Aggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Int16() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Int16() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minInt16Aggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Int16() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + { + a, b := int64(val), int64(a.curAgg) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Int16().Get(cmpIdx) + + { + var cmpResult int + + { + a, b := int64(cmpVal), int64(val) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minInt16Aggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minInt16Aggregator{} +} + +type minInt32Aggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg int32 +} + +// processBatch implements the bufferedWindower interface. +func (a *minInt32Aggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Int32() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Int32() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minInt32Aggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Int32() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + { + a, b := int64(val), int64(a.curAgg) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Int32().Get(cmpIdx) + + { + var cmpResult int + + { + a, b := int64(cmpVal), int64(val) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minInt32Aggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minInt32Aggregator{} +} + +type minInt64Aggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg int64 +} + +// processBatch implements the bufferedWindower interface. +func (a *minInt64Aggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Int64() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Int64() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minInt64Aggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Int64() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + { + a, b := int64(val), int64(a.curAgg) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Int64().Get(cmpIdx) + + { + var cmpResult int + + { + a, b := int64(cmpVal), int64(val) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minInt64Aggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minInt64Aggregator{} +} + +type minFloat64Aggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg float64 +} + +// processBatch implements the bufferedWindower interface. +func (a *minFloat64Aggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Float64() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Float64() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minFloat64Aggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Float64() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + { + a, b := float64(val), float64(a.curAgg) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else if a == b { + cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 + } + } + + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Float64().Get(cmpIdx) + + { + var cmpResult int + + { + a, b := float64(cmpVal), float64(val) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else if a == b { + cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 + } + } + + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minFloat64Aggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minFloat64Aggregator{} +} + +type minTimestampAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg time.Time +} + +// processBatch implements the bufferedWindower interface. +func (a *minTimestampAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Timestamp() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Timestamp() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minTimestampAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Timestamp() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + if val.Before(a.curAgg) { + cmpResult = -1 + } else if a.curAgg.Before(val) { + cmpResult = 1 + } else { + cmpResult = 0 + } + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Timestamp().Get(cmpIdx) + + { + var cmpResult int + + if cmpVal.Before(val) { + cmpResult = -1 + } else if val.Before(cmpVal) { + cmpResult = 1 + } else { + cmpResult = 0 + } + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minTimestampAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minTimestampAggregator{} +} + +type minIntervalAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg duration.Duration +} + +// processBatch implements the bufferedWindower interface. +func (a *minIntervalAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Interval() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Interval() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minIntervalAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Interval() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + cmpResult = val.Compare(a.curAgg) + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Interval().Get(cmpIdx) + + { + var cmpResult int + cmpResult = cmpVal.Compare(val) + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minIntervalAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minIntervalAggregator{} +} + +type minJSONAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg json.JSON +} + +// processBatch implements the bufferedWindower interface. +func (a *minJSONAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.JSON() + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.JSON() + val := col.Get(idx) + + var _err error + var _bytes []byte + _bytes, _err = json.EncodeJSON(nil, val) + if _err != nil { + colexecerror.ExpectedError(_err) + } + a.curAgg, _err = json.FromEncoding(_bytes) + if _err != nil { + colexecerror.ExpectedError(_err) + } + + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + + var _err error + var _bytes []byte + _bytes, _err = json.EncodeJSON(nil, a.curAgg) + if _err != nil { + colexecerror.ExpectedError(_err) + } + a.curAgg, _err = json.FromEncoding(_bytes) + if _err != nil { + colexecerror.ExpectedError(_err) + } + + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minJSONAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.JSON() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + var err error + cmpResult, err = val.Compare(a.curAgg) + if err != nil { + colexecerror.ExpectedError(err) + } + + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + + var _err error + var _bytes []byte + _bytes, _err = json.EncodeJSON(nil, val) + if _err != nil { + colexecerror.ExpectedError(_err) + } + a.curAgg, _err = json.FromEncoding(_bytes) + if _err != nil { + colexecerror.ExpectedError(_err) + } + + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.JSON().Get(cmpIdx) + + { + var cmpResult int + + var err error + cmpResult, err = cmpVal.Compare(val) + if err != nil { + colexecerror.ExpectedError(err) + } + + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minJSONAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minJSONAggregator{} +} + +type minDatumAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg interface{} +} + +// processBatch implements the bufferedWindower interface. +func (a *minDatumAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Datum() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Datum() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *minDatumAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Datum() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + cmpResult = coldataext.CompareDatum(val, col, a.curAgg) + + cmp = cmpResult < 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Datum().Get(cmpIdx) + + { + var cmpResult int + + cmpResult = coldataext.CompareDatum(cmpVal, col, val) + + cmp = cmpResult < 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *minDatumAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = minDatumAggregator{} +} + +func newMaxRemovableAggregator( + args *WindowArgs, framer windowFramer, buffer *colexecutils.SpillingBuffer, argTyp *types.T, +) bufferedWindower { + // Reserve the maximum memory usable by the queue up front to ensure that it + // isn't used by the SpillingBuffer. + args.BufferAllocator.AdjustMemoryUsage(maxQueueLength * memsize.Uint32) + base := minMaxRemovableAggBase{ + partitionSeekerBase: partitionSeekerBase{ + partitionColIdx: args.PartitionColIdx, + buffer: buffer, + }, + allocator: args.MainAllocator, + outputColIdx: args.OutputColIdx, + framer: framer, + queue: newMinMaxQueue(maxQueueLength), + omittedIndex: -1, + } + switch typeconv.TypeFamilyToCanonicalTypeFamily(argTyp.Family()) { + case types.BoolFamily: + switch argTyp.Width() { + case -1: + default: + return &maxBoolAggregator{minMaxRemovableAggBase: base} + } + case types.BytesFamily: + switch argTyp.Width() { + case -1: + default: + return &maxBytesAggregator{minMaxRemovableAggBase: base} + } + case types.DecimalFamily: + switch argTyp.Width() { + case -1: + default: + return &maxDecimalAggregator{minMaxRemovableAggBase: base} + } + case types.IntFamily: + switch argTyp.Width() { + case 16: + return &maxInt16Aggregator{minMaxRemovableAggBase: base} + case 32: + return &maxInt32Aggregator{minMaxRemovableAggBase: base} + case -1: + default: + return &maxInt64Aggregator{minMaxRemovableAggBase: base} + } + case types.FloatFamily: + switch argTyp.Width() { + case -1: + default: + return &maxFloat64Aggregator{minMaxRemovableAggBase: base} + } + case types.TimestampTZFamily: + switch argTyp.Width() { + case -1: + default: + return &maxTimestampAggregator{minMaxRemovableAggBase: base} + } + case types.IntervalFamily: + switch argTyp.Width() { + case -1: + default: + return &maxIntervalAggregator{minMaxRemovableAggBase: base} + } + case types.JsonFamily: + switch argTyp.Width() { + case -1: + default: + return &maxJSONAggregator{minMaxRemovableAggBase: base} + } + case typeconv.DatumVecCanonicalTypeFamily: + switch argTyp.Width() { + case -1: + default: + return &maxDatumAggregator{minMaxRemovableAggBase: base} + } + } + colexecerror.InternalError( + errors.AssertionFailedf("unexpectedly didn't find max overload for %s type family", argTyp.Name())) + // This code is unreachable, but the compiler cannot infer that. + return nil +} + +type maxBoolAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg bool +} + +// processBatch implements the bufferedWindower interface. +func (a *maxBoolAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Bool() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Bool() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxBoolAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Bool() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + if !val && a.curAgg { + cmpResult = -1 + } else if val && !a.curAgg { + cmpResult = 1 + } else { + cmpResult = 0 + } + + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Bool().Get(cmpIdx) + + { + var cmpResult int + + if !cmpVal && val { + cmpResult = -1 + } else if cmpVal && !val { + cmpResult = 1 + } else { + cmpResult = 0 + } + + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxBoolAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxBoolAggregator{} +} + +type maxBytesAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg []byte +} + +// processBatch implements the bufferedWindower interface. +func (a *maxBytesAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Bytes() + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Bytes() + val := col.Get(idx) + a.curAgg = append(a.curAgg[:0], val...) + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = append(a.curAgg[:0], a.curAgg...) + + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxBytesAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Bytes() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + cmpResult = bytes.Compare(val, a.curAgg) + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = append(a.curAgg[:0], val...) + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Bytes().Get(cmpIdx) + + { + var cmpResult int + cmpResult = bytes.Compare(cmpVal, val) + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxBytesAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxBytesAggregator{} +} + +type maxDecimalAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg apd.Decimal +} + +// processBatch implements the bufferedWindower interface. +func (a *maxDecimalAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Decimal() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Decimal() + val := col.Get(idx) + a.curAgg.Set(&val) + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg.Set(&a.curAgg) + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxDecimalAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Decimal() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + cmpResult = tree.CompareDecimals(&val, &a.curAgg) + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg.Set(&val) + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Decimal().Get(cmpIdx) + + { + var cmpResult int + cmpResult = tree.CompareDecimals(&cmpVal, &val) + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxDecimalAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxDecimalAggregator{} +} + +type maxInt16Aggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg int16 +} + +// processBatch implements the bufferedWindower interface. +func (a *maxInt16Aggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Int16() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Int16() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxInt16Aggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Int16() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + { + a, b := int64(val), int64(a.curAgg) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Int16().Get(cmpIdx) + + { + var cmpResult int + + { + a, b := int64(cmpVal), int64(val) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxInt16Aggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxInt16Aggregator{} +} + +type maxInt32Aggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg int32 +} + +// processBatch implements the bufferedWindower interface. +func (a *maxInt32Aggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Int32() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Int32() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxInt32Aggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Int32() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + { + a, b := int64(val), int64(a.curAgg) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Int32().Get(cmpIdx) + + { + var cmpResult int + + { + a, b := int64(cmpVal), int64(val) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxInt32Aggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxInt32Aggregator{} +} + +type maxInt64Aggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg int64 +} + +// processBatch implements the bufferedWindower interface. +func (a *maxInt64Aggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Int64() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Int64() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxInt64Aggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Int64() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + { + a, b := int64(val), int64(a.curAgg) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Int64().Get(cmpIdx) + + { + var cmpResult int + + { + a, b := int64(cmpVal), int64(val) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxInt64Aggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxInt64Aggregator{} +} + +type maxFloat64Aggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg float64 +} + +// processBatch implements the bufferedWindower interface. +func (a *maxFloat64Aggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Float64() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Float64() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxFloat64Aggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Float64() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + { + a, b := float64(val), float64(a.curAgg) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else if a == b { + cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 + } + } + + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Float64().Get(cmpIdx) + + { + var cmpResult int + + { + a, b := float64(cmpVal), float64(val) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else if a == b { + cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 + } + } + + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxFloat64Aggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxFloat64Aggregator{} +} + +type maxTimestampAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg time.Time +} + +// processBatch implements the bufferedWindower interface. +func (a *maxTimestampAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Timestamp() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Timestamp() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxTimestampAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Timestamp() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + if val.Before(a.curAgg) { + cmpResult = -1 + } else if a.curAgg.Before(val) { + cmpResult = 1 + } else { + cmpResult = 0 + } + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Timestamp().Get(cmpIdx) + + { + var cmpResult int + + if cmpVal.Before(val) { + cmpResult = -1 + } else if val.Before(cmpVal) { + cmpResult = 1 + } else { + cmpResult = 0 + } + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxTimestampAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxTimestampAggregator{} +} + +type maxIntervalAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg duration.Duration +} + +// processBatch implements the bufferedWindower interface. +func (a *maxIntervalAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Interval() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Interval() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxIntervalAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Interval() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + cmpResult = val.Compare(a.curAgg) + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Interval().Get(cmpIdx) + + { + var cmpResult int + cmpResult = cmpVal.Compare(val) + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxIntervalAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxIntervalAggregator{} +} + +type maxJSONAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg json.JSON +} + +// processBatch implements the bufferedWindower interface. +func (a *maxJSONAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.JSON() + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.JSON() + val := col.Get(idx) + + var _err error + var _bytes []byte + _bytes, _err = json.EncodeJSON(nil, val) + if _err != nil { + colexecerror.ExpectedError(_err) + } + a.curAgg, _err = json.FromEncoding(_bytes) + if _err != nil { + colexecerror.ExpectedError(_err) + } + + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + + var _err error + var _bytes []byte + _bytes, _err = json.EncodeJSON(nil, a.curAgg) + if _err != nil { + colexecerror.ExpectedError(_err) + } + a.curAgg, _err = json.FromEncoding(_bytes) + if _err != nil { + colexecerror.ExpectedError(_err) + } + + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxJSONAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.JSON() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + var err error + cmpResult, err = val.Compare(a.curAgg) + if err != nil { + colexecerror.ExpectedError(err) + } + + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + + var _err error + var _bytes []byte + _bytes, _err = json.EncodeJSON(nil, val) + if _err != nil { + colexecerror.ExpectedError(_err) + } + a.curAgg, _err = json.FromEncoding(_bytes) + if _err != nil { + colexecerror.ExpectedError(_err) + } + + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.JSON().Get(cmpIdx) + + { + var cmpResult int + + var err error + cmpResult, err = cmpVal.Compare(val) + if err != nil { + colexecerror.ExpectedError(err) + } + + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxJSONAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxJSONAggregator{} +} + +type maxDatumAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg interface{} +} + +// processBatch implements the bufferedWindower interface. +func (a *maxDatumAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.Datum() + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.Datum() + val := col.Get(idx) + a.curAgg = val + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + a.curAgg = a.curAgg + + // gcassert:bce + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *maxDatumAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.Datum() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + + { + var cmpResult int + + cmpResult = coldataext.CompareDatum(val, col, a.curAgg) + + cmp = cmpResult > 0 + } + + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + a.curAgg = val + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.Datum().Get(cmpIdx) + + { + var cmpResult int + + cmpResult = coldataext.CompareDatum(cmpVal, col, val) + + cmp = cmpResult > 0 + } + + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *maxDatumAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = maxDatumAggregator{} +} + +// getIntervalsGEIdx returns a set of intervals representing all indexes in the +// 'intervals' slice at or after the given index. +func getIntervalsGEIdx(intervals, scratch []windowInterval, idx int) []windowInterval { + scratch = scratch[:0] + for _, interval := range intervals { + if interval.end <= idx { + continue + } + if interval.start >= idx { + scratch = append(scratch, interval) + continue + } + scratch = append(scratch, windowInterval{start: idx, end: interval.end}) + } + return scratch +} diff --git a/pkg/sql/colexec/colexecwindow/min_max_removable_agg_tmpl.go b/pkg/sql/colexec/colexecwindow/min_max_removable_agg_tmpl.go new file mode 100644 index 000000000000..9a967e31e643 --- /dev/null +++ b/pkg/sql/colexec/colexecwindow/min_max_removable_agg_tmpl.go @@ -0,0 +1,320 @@ +// Copyright 2021 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. + +// {{/* +// +build execgen_template +// +// This file is the execgen template for min_max_removable_agg.eg.go. It's +// formatted in a special way, so it's both valid Go and a valid text/template +// input. This permits editing this file with editor support. +// +// */}} + +package colexecwindow + +import ( + "context" + + "github.com/cockroachdb/apd/v2" + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/coldataext" + "github.com/cockroachdb/cockroach/pkg/col/typeconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" + "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/memsize" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/cockroach/pkg/util/json" + "github.com/cockroachdb/errors" +) + +// Workaround for bazel auto-generated code. goimports does not automatically +// pick up the right packages when run within the bazel sandbox. +var ( + _ tree.AggType + _ apd.Context + _ duration.Duration + _ json.JSON + _ = coldataext.CompareDatum + _ = colexecerror.InternalError + _ = memsize.Uint32 +) + +// {{/* +// Declarations to make the template compile properly. + +// _ASSIGN_CMP is the template function for assigning true to the first input +// if the second input compares successfully to the third input. The comparison +// operator is tree.LT for MIN and is tree.GT for MAX. +func _ASSIGN_CMP(_, _, _, _, _, _ string) bool { + colexecerror.InternalError(errors.AssertionFailedf("")) +} + +// */}} + +const ( + // The argument column is always the first column in the SpillingBuffer. + argColIdx = 0 + + // The slice of uint32s in the deque can have up to 10,000 values (40KB). + maxQueueLength = 10000 +) + +type minMaxRemovableAggBase struct { + partitionSeekerBase + colexecop.CloserHelper + allocator *colmem.Allocator + outputColIdx int + framer windowFramer + + // A partial deque of indices into the current partition ordered by the value + // of the input column at each index. It contains only indices that are part + // of the current window frame. The first value in the queue is the index of + // the current value for the aggregation (NULL if empty). Under the + // simplifying assumption that the window frame has no exclusion clause, the + // queue does not need to contain any indices smaller than the best index - + // this keeps the queue small in many common cases. + queue minMaxQueue + + // omittedIndex tracks the index where we reached the limit of the length of + // the queue, in which case we may be omitting values that could become + // relevant as the frame shrinks. If the queue becomes empty while this + // index is set, we have to aggregate over the previously omitted values. + // The default (unset) value is -1. + omittedIndex int + + scratchIntervals []windowInterval +} + +// Init implements the bufferedWindower interface. +func (b *minMaxRemovableAggBase) Init(ctx context.Context) { + b.InitHelper.Init(ctx) +} + +// transitionToProcessing implements the bufferedWindower interface. +func (b *minMaxRemovableAggBase) transitionToProcessing() { + b.framer.startPartition(b.Ctx, b.partitionSize, b.buffer) +} + +// startNewPartition implements the bufferedWindower interface. +func (b *minMaxRemovableAggBase) startNewPartition() { + b.partitionSize = 0 + b.buffer.Reset(b.Ctx) + b.queue.reset() +} + +// {{range .}} +// {{$agg := .Agg}} + +func new_AGG_TITLERemovableAggregator( + args *WindowArgs, framer windowFramer, buffer *colexecutils.SpillingBuffer, argTyp *types.T, +) bufferedWindower { + // Reserve the maximum memory usable by the queue up front to ensure that it + // isn't used by the SpillingBuffer. + args.BufferAllocator.AdjustMemoryUsage(maxQueueLength * memsize.Uint32) + base := minMaxRemovableAggBase{ + partitionSeekerBase: partitionSeekerBase{ + partitionColIdx: args.PartitionColIdx, + buffer: buffer, + }, + allocator: args.MainAllocator, + outputColIdx: args.OutputColIdx, + framer: framer, + queue: newMinMaxQueue(maxQueueLength), + omittedIndex: -1, + } + switch typeconv.TypeFamilyToCanonicalTypeFamily(argTyp.Family()) { + // {{range .Overloads}} + case _CANONICAL_TYPE_FAMILY: + switch argTyp.Width() { + // {{range .WidthOverloads}} + case _TYPE_WIDTH: + return &_AGG_TYPEAggregator{minMaxRemovableAggBase: base} + // {{end}} + } + // {{end}} + } + colexecerror.InternalError( + errors.AssertionFailedf("unexpectedly didn't find _AGG overload for %s type family", argTyp.Name())) + // This code is unreachable, but the compiler cannot infer that. + return nil +} + +// {{range .Overloads}} +// {{range .WidthOverloads}} + +type _AGG_TYPEAggregator struct { + minMaxRemovableAggBase + // curAgg holds the running min/max, so we can index into the output column + // once per row, instead of on each iteration. + // NOTE: if the length of the queue is zero, curAgg is undefined. + curAgg _GOTYPE +} + +// processBatch implements the bufferedWindower interface. +func (a *_AGG_TYPEAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + if endIdx <= startIdx { + // There is no processing to be done. + return + } + outVec := batch.ColVec(a.outputColIdx) + outNulls := outVec.Nulls() + outCol := outVec.TemplateType() + // {{if not .IsBytesLike}} + _, _ = outCol.Get(startIdx), outCol.Get(endIdx-1) + // {{end}} + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + + // Process the toRemove intervals first. + if !a.queue.isEmpty() { + prevBestIdx := a.queue.getFirst() + for _, interval := range toRemove { + if uint32(interval.start) > a.queue.getFirst() { + colexecerror.InternalError(errors.AssertionFailedf( + "expected default exclusion clause for min/max sliding window operator")) + } + a.queue.removeAllBefore(uint32(interval.end)) + } + if !a.queue.isEmpty() { + newBestIdx := a.queue.getFirst() + if newBestIdx != prevBestIdx { + // We need to update curAgg. + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(newBestIdx)) + col := vec.TemplateType() + val := col.Get(idx) + execgen.COPYVAL(a.curAgg, val) + } + } + } + + // Now aggregate over the toAdd intervals. + if a.queue.isEmpty() && a.omittedIndex != -1 { + // We have exhausted all the values that fit in the queue - we need to + // re-aggregate over the current window frame starting from the first + // omitted index. + a.scratchIntervals = getIntervalsGEIdx( + a.framer.frameIntervals(), a.scratchIntervals, a.omittedIndex) + a.omittedIndex = -1 + a.aggregateOverIntervals(a.scratchIntervals) + } else { + a.aggregateOverIntervals(toAdd) + } + + // Set the output value for the current row. + if a.queue.isEmpty() { + outNulls.SetNull(i) + } else { + // The aggregate may be reused between rows, so we need to copy it. + execgen.COPYVAL(a.curAgg, a.curAgg) + + // {{if not .IsBytesLike}} + // gcassert:bce + // {{end}} + outCol.Set(i, a.curAgg) + } + } + }) +} + +// aggregateOverIntervals accumulates all rows represented by the given +// intervals into the current aggregate. +func (a *_AGG_TYPEAggregator) aggregateOverIntervals(intervals []windowInterval) { + for _, interval := range intervals { + var cmp bool + for j := interval.start; j < interval.end; j++ { + idxToAdd := uint32(j) + vec, idx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, j) + nulls := vec.Nulls() + col := vec.TemplateType() + if !nulls.MaybeHasNulls() || !nulls.NullAt(idx) { + val := col.Get(idx) + + // If this is the first value in the frame, it is the best so far. + isBest := a.queue.isEmpty() + if !a.queue.isEmpty() { + // Compare to the best value seen so far. + _ASSIGN_CMP(cmp, val, a.curAgg, _, col, _) + if cmp { + // Reset the queue because the current value replaces all others. + isBest = true + a.queue.reset() + } + isBest = cmp + } + if isBest { + // The queue is already empty, so just add to the end of the queue. + // If any values were omitted from the queue, they would be dominated + // by this one anyway, so reset omittedIndex. + a.queue.addLast(idxToAdd) + execgen.COPYVAL(a.curAgg, val) + a.omittedIndex = -1 + continue + } + + // This is not the best value in the window frame, but we still need to + // keep it in the queue. Iterate from the end of the queue, removing any + // values that are dominated by the current one. Add the current value + // once the last value in the queue is better than the current one. + for !a.queue.isEmpty() { + cmpVec, cmpIdx, _ := a.buffer.GetVecWithTuple(a.Ctx, argColIdx, int(a.queue.getLast())) + cmpVal := cmpVec.TemplateType().Get(cmpIdx) + _ASSIGN_CMP(cmp, cmpVal, val, _, col, _) + if cmp { + break + } + // Any values that could not fit in the queue would also have been + // dominated by the current one, so reset omittedIndex. + a.queue.removeLast() + a.omittedIndex = -1 + } + if a.queue.addLast(idxToAdd) && a.omittedIndex == -1 { + // The value couldn't fit in the queue. Keep track of the first index + // from which the queue could no longer store values. + a.omittedIndex = j + } + } + } + } +} + +func (a *_AGG_TYPEAggregator) Close() { + a.queue.close() + a.framer.close() + a.buffer.Close(a.EnsureCtx()) + *a = _AGG_TYPEAggregator{} +} + +// {{end}} +// {{end}} +// {{end}} + +// getIntervalsGEIdx returns a set of intervals representing all indexes in the +// 'intervals' slice at or after the given index. +func getIntervalsGEIdx(intervals, scratch []windowInterval, idx int) []windowInterval { + scratch = scratch[:0] + for _, interval := range intervals { + if interval.end <= idx { + continue + } + if interval.start >= idx { + scratch = append(scratch, interval) + continue + } + scratch = append(scratch, windowInterval{start: idx, end: interval.end}) + } + return scratch +} diff --git a/pkg/sql/colexec/colexecwindow/nth_value.eg.go b/pkg/sql/colexec/colexecwindow/nth_value.eg.go index 5d4473e53015..ed3fccf5d57f 100644 --- a/pkg/sql/colexec/colexecwindow/nth_value.eg.go +++ b/pkg/sql/colexec/colexecwindow/nth_value.eg.go @@ -41,6 +41,7 @@ func NewNthValueOperator( // store a single column. TODO(drewk): play around with benchmarks to find a // good empirically-supported fraction to use. bufferMemLimit := int64(float64(args.MemoryLimit) * 0.10) + mainMemLimit := args.MemoryLimit - bufferMemLimit buffer := colexecutils.NewSpillingBuffer( args.BufferAllocator, bufferMemLimit, args.QueueCfg, args.FdSemaphore, args.InputTypes, args.DiskAcc, colsToStore...) @@ -61,7 +62,7 @@ func NewNthValueOperator( default: windower := &nthValueBoolWindow{nthValueBase: base} windower.nColIdx = argIdxs[1] - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case types.BytesFamily: switch argType.Width() { @@ -69,7 +70,7 @@ func NewNthValueOperator( default: windower := &nthValueBytesWindow{nthValueBase: base} windower.nColIdx = argIdxs[1] - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case types.DecimalFamily: switch argType.Width() { @@ -77,23 +78,23 @@ func NewNthValueOperator( default: windower := &nthValueDecimalWindow{nthValueBase: base} windower.nColIdx = argIdxs[1] - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case types.IntFamily: switch argType.Width() { case 16: windower := &nthValueInt16Window{nthValueBase: base} windower.nColIdx = argIdxs[1] - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil case 32: windower := &nthValueInt32Window{nthValueBase: base} windower.nColIdx = argIdxs[1] - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil case -1: default: windower := &nthValueInt64Window{nthValueBase: base} windower.nColIdx = argIdxs[1] - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case types.FloatFamily: switch argType.Width() { @@ -101,7 +102,7 @@ func NewNthValueOperator( default: windower := &nthValueFloat64Window{nthValueBase: base} windower.nColIdx = argIdxs[1] - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case types.TimestampTZFamily: switch argType.Width() { @@ -109,7 +110,7 @@ func NewNthValueOperator( default: windower := &nthValueTimestampWindow{nthValueBase: base} windower.nColIdx = argIdxs[1] - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case types.IntervalFamily: switch argType.Width() { @@ -117,7 +118,7 @@ func NewNthValueOperator( default: windower := &nthValueIntervalWindow{nthValueBase: base} windower.nColIdx = argIdxs[1] - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case types.JsonFamily: switch argType.Width() { @@ -125,7 +126,7 @@ func NewNthValueOperator( default: windower := &nthValueJSONWindow{nthValueBase: base} windower.nColIdx = argIdxs[1] - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } case typeconv.DatumVecCanonicalTypeFamily: switch argType.Width() { @@ -133,7 +134,7 @@ func NewNthValueOperator( default: windower := &nthValueDatumWindow{nthValueBase: base} windower.nColIdx = argIdxs[1] - return newBufferedWindowOperator(args, windower, argType), nil + return newBufferedWindowOperator(args, windower, argType, mainMemLimit), nil } } return nil, errors.Errorf("unsupported nthValue window operator type %s", argType.Name()) diff --git a/pkg/sql/colexec/colexecwindow/ntile.eg.go b/pkg/sql/colexec/colexecwindow/ntile.eg.go index 70aceada0c0f..deed9c367fb5 100644 --- a/pkg/sql/colexec/colexecwindow/ntile.eg.go +++ b/pkg/sql/colexec/colexecwindow/ntile.eg.go @@ -40,7 +40,7 @@ func NewNTileOperator(args *WindowArgs, argIdx int) colexecop.Operator { } else { windower = &nTileWithPartition{base} } - return newBufferedWindowOperator(args, windower, types.Int) + return newBufferedWindowOperator(args, windower, types.Int, args.MemoryLimit) } // nTileBase extracts common fields and methods of the two variations of ntile diff --git a/pkg/sql/colexec/colexecwindow/ntile_tmpl.go b/pkg/sql/colexec/colexecwindow/ntile_tmpl.go index 9e233bf64d93..3e251d8291ab 100644 --- a/pkg/sql/colexec/colexecwindow/ntile_tmpl.go +++ b/pkg/sql/colexec/colexecwindow/ntile_tmpl.go @@ -50,7 +50,7 @@ func NewNTileOperator(args *WindowArgs, argIdx int) colexecop.Operator { } else { windower = &nTileWithPartition{base} } - return newBufferedWindowOperator(args, windower, types.Int) + return newBufferedWindowOperator(args, windower, types.Int, args.MemoryLimit) } // nTileBase extracts common fields and methods of the two variations of ntile diff --git a/pkg/sql/colexec/colexecwindow/window_aggregator.eg.go b/pkg/sql/colexec/colexecwindow/window_aggregator.eg.go new file mode 100644 index 000000000000..60effe81c329 --- /dev/null +++ b/pkg/sql/colexec/colexecwindow/window_aggregator.eg.go @@ -0,0 +1,310 @@ +// Code generated by execgen; DO NOT EDIT. +// Copyright 2021 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. + +// {{/* + +// This file is the execgen template for window_aggregator.eg.go. It's formatted +// in a special way, so it's both valid Go and a valid text/template input. This +// permits editing this file with editor support. +// */}} + +package colexecwindow + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" + "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/types" +) + +type slidingWindowAggregateFunc interface { + colexecagg.AggregateFunc + + // Remove removes the indicated rows from the the aggregation. It is used when + // the window frame for the previous row included rows that are not included + // in the current frame. + // Note: the implementations should be careful to account for their memory + // usage. + // Note: endIdx is assumed to be greater than zero. + Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) +} + +// NewWindowAggregatorOperator creates a new Operator that computes aggregate +// window functions. outputColIdx specifies in which coldata.Vec the operator +// should put its output (if there is no such column, a new column is appended). +func NewWindowAggregatorOperator( + args *WindowArgs, + aggType execinfrapb.AggregatorSpec_Func, + frame *execinfrapb.WindowerSpec_Frame, + ordering *execinfrapb.Ordering, + argIdxs []int, + outputType *types.T, + aggAlloc *colexecagg.AggregateFuncsAlloc, + closers colexecop.Closers, +) colexecop.Operator { + // Because the buffer is used multiple times per-row, it is important to + // prevent it from spilling to disk if possible. For this reason, we give the + // buffer half of the memory budget even though it will generally store less + // columns than the queue. + bufferMemLimit := int64(float64(args.MemoryLimit) * 0.5) + mainMemLimit := args.MemoryLimit - bufferMemLimit + framer := newWindowFramer(args.EvalCtx, frame, ordering, args.InputTypes, args.PeersColIdx) + colsToStore := framer.getColsToStore(append([]int{}, argIdxs...)) + buffer := colexecutils.NewSpillingBuffer( + args.BufferAllocator, bufferMemLimit, args.QueueCfg, + args.FdSemaphore, args.InputTypes, args.DiskAcc, colsToStore...) + inputIdxs := make([]uint32, len(argIdxs)) + for i := range inputIdxs { + // We will always store the arg columns first in the buffer. + inputIdxs[i] = uint32(i) + } + base := windowAggregatorBase{ + partitionSeekerBase: partitionSeekerBase{ + partitionColIdx: args.PartitionColIdx, + buffer: buffer, + }, + allocator: args.MainAllocator, + outputColIdx: args.OutputColIdx, + inputIdxs: inputIdxs, + framer: framer, + closers: closers, + vecs: make([]coldata.Vec, len(inputIdxs)), + } + var agg colexecagg.AggregateFunc + if aggAlloc != nil { + agg = aggAlloc.MakeAggregateFuncs()[0] + } + var windower bufferedWindower + switch aggType { + case execinfrapb.Min, execinfrapb.Max: + if WindowFrameCanShrink(frame, ordering) { + // In the case when the window frame for a given row does not necessarily + // include all rows from the previous frame, min and max require a + // specialized implementation that maintains a dequeue of seen values. + if frame.Exclusion != execinfrapb.WindowerSpec_Frame_NO_EXCLUSION { + // TODO(drewk): extend the implementations to work with non-default + // exclusion. For now, we have to use the quadratic-time method. + windower = &windowAggregator{windowAggregatorBase: base, agg: agg} + } else { + switch aggType { + case execinfrapb.Min: + windower = newMinRemovableAggregator(args, framer, buffer, outputType) + case execinfrapb.Max: + windower = newMaxRemovableAggregator(args, framer, buffer, outputType) + } + } + } else { + // When the frame can only grow, the simple sliding window implementation + // is sufficient. + windower = &slidingWindowAggregator{ + windowAggregatorBase: base, + agg: agg.(slidingWindowAggregateFunc), + } + } + default: + if slidingWindowAgg, ok := agg.(slidingWindowAggregateFunc); ok { + windower = &slidingWindowAggregator{windowAggregatorBase: base, agg: slidingWindowAgg} + } else { + windower = &windowAggregator{windowAggregatorBase: base, agg: agg} + } + } + return newBufferedWindowOperator(args, windower, outputType, mainMemLimit) +} + +type windowAggregatorBase struct { + partitionSeekerBase + colexecop.CloserHelper + closers colexecop.Closers + allocator *colmem.Allocator + + outputColIdx int + inputIdxs []uint32 + vecs []coldata.Vec + framer windowFramer +} + +type windowAggregator struct { + windowAggregatorBase + agg colexecagg.AggregateFunc +} + +type slidingWindowAggregator struct { + windowAggregatorBase + agg slidingWindowAggregateFunc +} + +var ( + _ bufferedWindower = &windowAggregator{} + _ bufferedWindower = &slidingWindowAggregator{} +) + +// windowInterval represents rows in the range [start, end). Slices of +// windowIntervals should always be increasing and non-overlapping. +type windowInterval struct { + start int + end int +} + +// transitionToProcessing implements the bufferedWindower interface. +func (a *windowAggregatorBase) transitionToProcessing() { + a.framer.startPartition(a.Ctx, a.partitionSize, a.buffer) +} + +// startNewPartition implements the bufferedWindower interface. +func (a *windowAggregatorBase) startNewPartition() { + a.partitionSize = 0 + a.buffer.Reset(a.Ctx) +} + +// Init implements the bufferedWindower interface. +func (a *windowAggregatorBase) Init(ctx context.Context) { + a.InitHelper.Init(ctx) +} + +// Close implements the bufferedWindower interface. +func (a *windowAggregatorBase) Close() { + if !a.CloserHelper.Close() { + return + } + if err := a.closers.Close(); err != nil { + colexecerror.InternalError(err) + } + a.framer.close() + a.buffer.Close(a.EnsureCtx()) +} + +func (a *windowAggregator) startNewPartition() { + a.windowAggregatorBase.startNewPartition() + a.agg.Reset() +} + +func (a *windowAggregator) Close() { + a.windowAggregatorBase.Close() + a.agg.Reset() + *a = windowAggregator{} +} + +// processBatch implements the bufferedWindower interface. +func (a *windowAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + outVec := batch.ColVec(a.outputColIdx) + a.agg.SetOutput(outVec) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + { + var intervals []windowInterval = a.framer.frameIntervals() + for _, interval := range intervals { + // intervalIdx maintains the index up to which the current interval has + // already been processed. + intervalIdx := interval.start + start, end := interval.start, interval.end + intervalLen := interval.end - interval.start + for intervalLen > 0 { + for j, idx := range a.inputIdxs { + a.vecs[j], start, end = a.buffer.GetVecWithTuple(a.Ctx, int(idx), intervalIdx) + } + if intervalLen < (end - start) { + // This is the last batch in the current interval. + end = start + intervalLen + } + intervalIdx += end - start + intervalLen -= end - start + a.agg.Compute(a.vecs, a.inputIdxs, start, end, nil /* sel */) + } + } + } + a.agg.Flush(i) + a.agg.Reset() + } + }) +} + +func (a *slidingWindowAggregator) startNewPartition() { + a.windowAggregatorBase.startNewPartition() + a.agg.Reset() +} + +func (a *slidingWindowAggregator) Close() { + a.windowAggregatorBase.Close() + a.agg.Reset() + *a = slidingWindowAggregator{} +} + +// processBatch implements the bufferedWindower interface. +func (a *slidingWindowAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + outVec := batch.ColVec(a.outputColIdx) + a.agg.SetOutput(outVec) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + // Process the 'toRemove' intervals first to avoid overflowing. + { + var intervals []windowInterval = toRemove + for _, interval := range intervals { + // intervalIdx maintains the index up to which the current interval has + // already been processed. + intervalIdx := interval.start + start, end := interval.start, interval.end + intervalLen := interval.end - interval.start + for intervalLen > 0 { + for j, idx := range a.inputIdxs { + a.vecs[j], start, end = a.buffer.GetVecWithTuple(a.Ctx, int(idx), intervalIdx) + } + if intervalLen < (end - start) { + // This is the last batch in the current interval. + end = start + intervalLen + } + intervalIdx += end - start + intervalLen -= end - start + a.agg.Remove(a.vecs, a.inputIdxs, start, end) + } + } + } + { + var intervals []windowInterval = toAdd + for _, interval := range intervals { + // intervalIdx maintains the index up to which the current interval has + // already been processed. + intervalIdx := interval.start + start, end := interval.start, interval.end + intervalLen := interval.end - interval.start + for intervalLen > 0 { + for j, idx := range a.inputIdxs { + a.vecs[j], start, end = a.buffer.GetVecWithTuple(a.Ctx, int(idx), intervalIdx) + } + if intervalLen < (end - start) { + // This is the last batch in the current interval. + end = start + intervalLen + } + intervalIdx += end - start + intervalLen -= end - start + a.agg.Compute(a.vecs, a.inputIdxs, start, end, nil /* sel */) + } + } + } + a.agg.Flush(i) + } + }) +} + +// execgen:inline +const _ = "template_aggregateOverIntervals" + +// execgen:inline +const _ = "inlined_aggregateOverIntervals_false" + +// execgen:inline +const _ = "inlined_aggregateOverIntervals_true" diff --git a/pkg/sql/colexec/colexecwindow/window_aggregator.go b/pkg/sql/colexec/colexecwindow/window_aggregator.go deleted file mode 100644 index a2207becf017..000000000000 --- a/pkg/sql/colexec/colexecwindow/window_aggregator.go +++ /dev/null @@ -1,152 +0,0 @@ -// Copyright 2021 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 colexecwindow - -import ( - "context" - - "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" - "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" - "github.com/cockroachdb/cockroach/pkg/sql/colexecop" - "github.com/cockroachdb/cockroach/pkg/sql/colmem" - "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/types" -) - -// NewWindowAggregatorOperator creates a new Operator that computes aggregate -// window functions. outputColIdx specifies in which coldata.Vec the operator -// should put its output (if there is no such column, a new column is appended). -func NewWindowAggregatorOperator( - args *WindowArgs, - frame *execinfrapb.WindowerSpec_Frame, - ordering *execinfrapb.Ordering, - argIdxs []int, - outputType *types.T, - aggAlloc *colexecagg.AggregateFuncsAlloc, - closers colexecop.Closers, -) colexecop.Operator { - // Because the buffer is used multiple times per-row, it is important to - // prevent it from spilling to disk if possible. For this reason, we give the - // buffer half of the memory budget even though it will generally store less - // columns than the queue. - bufferMemLimit := int64(float64(args.MemoryLimit) * 0.5) - framer := newWindowFramer(args.EvalCtx, frame, ordering, args.InputTypes, args.PeersColIdx) - colsToStore := framer.getColsToStore(append([]int{}, argIdxs...)) - buffer := colexecutils.NewSpillingBuffer( - args.BufferAllocator, bufferMemLimit, args.QueueCfg, - args.FdSemaphore, args.InputTypes, args.DiskAcc, colsToStore...) - inputIdxs := make([]uint32, len(argIdxs)) - for i := range inputIdxs { - // We will always store the arg columns first in the buffer. - inputIdxs[i] = uint32(i) - } - windower := &windowAggregator{ - partitionSeekerBase: partitionSeekerBase{ - partitionColIdx: args.PartitionColIdx, - buffer: buffer, - }, - allocator: args.MainAllocator, - outputColIdx: args.OutputColIdx, - inputIdxs: inputIdxs, - agg: aggAlloc.MakeAggregateFuncs()[0], - framer: framer, - closers: closers, - vecs: make([]coldata.Vec, len(inputIdxs)), - } - return newBufferedWindowOperator(args, windower, outputType) -} - -type windowAggregator struct { - partitionSeekerBase - colexecop.CloserHelper - closers colexecop.Closers - allocator *colmem.Allocator - - outputColIdx int - inputIdxs []uint32 - vecs []coldata.Vec - - agg colexecagg.AggregateFunc - framer windowFramer -} - -var _ bufferedWindower = &windowAggregator{} - -type windowInterval struct { - start int - end int -} - -// processBatch implements the bufferedWindower interface. -func (a *windowAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { - outVec := batch.ColVec(a.outputColIdx) - a.agg.SetOutput(outVec) - a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { - for i := startIdx; i < endIdx; i++ { - a.framer.next(a.Ctx) - for _, interval := range a.framer.frameIntervals() { - // intervalIdx maintains the index up to which the current interval has - // already been processed. - intervalIdx := interval.start - start, end := interval.start, interval.end - intervalLen := interval.end - interval.start - for intervalLen > 0 { - for j, idx := range a.inputIdxs { - a.vecs[j], start, end = a.buffer.GetVecWithTuple(a.Ctx, int(idx), intervalIdx) - } - if intervalLen < (end - start) { - // This is the last batch in the current interval. - end = start + intervalLen - } - intervalIdx += end - start - intervalLen -= end - start - a.agg.Compute(a.vecs, a.inputIdxs, start, end, nil /* sel */) - } - } - a.agg.Flush(i) - a.agg.Reset() - } - }) -} - -// transitionToProcessing implements the bufferedWindower interface. -func (a *windowAggregator) transitionToProcessing() { - a.framer.startPartition(a.Ctx, a.partitionSize, a.buffer) -} - -// startNewPartition implements the bufferedWindower interface. -func (a *windowAggregator) startNewPartition() { - a.partitionSize = 0 - a.buffer.Reset(a.Ctx) - a.agg.Reset() -} - -// Init implements the bufferedWindower interface. -func (a *windowAggregator) Init(ctx context.Context) { - if !a.InitHelper.Init(ctx) { - return - } -} - -// Close implements the bufferedWindower interface. -func (a *windowAggregator) Close() { - if !a.CloserHelper.Close() { - return - } - if err := a.closers.Close(); err != nil { - colexecerror.InternalError(err) - } - a.buffer.Close(a.EnsureCtx()) - a.agg.Reset() - *a = windowAggregator{CloserHelper: a.CloserHelper} -} diff --git a/pkg/sql/colexec/colexecwindow/window_aggregator_tmpl.go b/pkg/sql/colexec/colexecwindow/window_aggregator_tmpl.go new file mode 100644 index 000000000000..e30451b4d394 --- /dev/null +++ b/pkg/sql/colexec/colexecwindow/window_aggregator_tmpl.go @@ -0,0 +1,269 @@ +// Copyright 2021 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. + +// {{/* +// +build execgen_template +// +// This file is the execgen template for window_aggregator.eg.go. It's formatted +// in a special way, so it's both valid Go and a valid text/template input. This +// permits editing this file with editor support. +// +// */}} + +package colexecwindow + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" + "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/types" +) + +type slidingWindowAggregateFunc interface { + colexecagg.AggregateFunc + + // Remove removes the indicated rows from the the aggregation. It is used when + // the window frame for the previous row included rows that are not included + // in the current frame. + // Note: the implementations should be careful to account for their memory + // usage. + // Note: endIdx is assumed to be greater than zero. + Remove(vecs []coldata.Vec, inputIdxs []uint32, startIdx, endIdx int) +} + +// NewWindowAggregatorOperator creates a new Operator that computes aggregate +// window functions. outputColIdx specifies in which coldata.Vec the operator +// should put its output (if there is no such column, a new column is appended). +func NewWindowAggregatorOperator( + args *WindowArgs, + aggType execinfrapb.AggregatorSpec_Func, + frame *execinfrapb.WindowerSpec_Frame, + ordering *execinfrapb.Ordering, + argIdxs []int, + outputType *types.T, + aggAlloc *colexecagg.AggregateFuncsAlloc, + closers colexecop.Closers, +) colexecop.Operator { + // Because the buffer is used multiple times per-row, it is important to + // prevent it from spilling to disk if possible. For this reason, we give the + // buffer half of the memory budget even though it will generally store less + // columns than the queue. + bufferMemLimit := int64(float64(args.MemoryLimit) * 0.5) + mainMemLimit := args.MemoryLimit - bufferMemLimit + framer := newWindowFramer(args.EvalCtx, frame, ordering, args.InputTypes, args.PeersColIdx) + colsToStore := framer.getColsToStore(append([]int{}, argIdxs...)) + buffer := colexecutils.NewSpillingBuffer( + args.BufferAllocator, bufferMemLimit, args.QueueCfg, + args.FdSemaphore, args.InputTypes, args.DiskAcc, colsToStore...) + inputIdxs := make([]uint32, len(argIdxs)) + for i := range inputIdxs { + // We will always store the arg columns first in the buffer. + inputIdxs[i] = uint32(i) + } + base := windowAggregatorBase{ + partitionSeekerBase: partitionSeekerBase{ + partitionColIdx: args.PartitionColIdx, + buffer: buffer, + }, + allocator: args.MainAllocator, + outputColIdx: args.OutputColIdx, + inputIdxs: inputIdxs, + framer: framer, + closers: closers, + vecs: make([]coldata.Vec, len(inputIdxs)), + } + var agg colexecagg.AggregateFunc + if aggAlloc != nil { + agg = aggAlloc.MakeAggregateFuncs()[0] + } + var windower bufferedWindower + switch aggType { + case execinfrapb.Min, execinfrapb.Max: + if WindowFrameCanShrink(frame, ordering) { + // In the case when the window frame for a given row does not necessarily + // include all rows from the previous frame, min and max require a + // specialized implementation that maintains a dequeue of seen values. + if frame.Exclusion != execinfrapb.WindowerSpec_Frame_NO_EXCLUSION { + // TODO(drewk): extend the implementations to work with non-default + // exclusion. For now, we have to use the quadratic-time method. + windower = &windowAggregator{windowAggregatorBase: base, agg: agg} + } else { + switch aggType { + case execinfrapb.Min: + windower = newMinRemovableAggregator(args, framer, buffer, outputType) + case execinfrapb.Max: + windower = newMaxRemovableAggregator(args, framer, buffer, outputType) + } + } + } else { + // When the frame can only grow, the simple sliding window implementation + // is sufficient. + windower = &slidingWindowAggregator{ + windowAggregatorBase: base, + agg: agg.(slidingWindowAggregateFunc), + } + } + default: + if slidingWindowAgg, ok := agg.(slidingWindowAggregateFunc); ok { + windower = &slidingWindowAggregator{windowAggregatorBase: base, agg: slidingWindowAgg} + } else { + windower = &windowAggregator{windowAggregatorBase: base, agg: agg} + } + } + return newBufferedWindowOperator(args, windower, outputType, mainMemLimit) +} + +type windowAggregatorBase struct { + partitionSeekerBase + colexecop.CloserHelper + closers colexecop.Closers + allocator *colmem.Allocator + + outputColIdx int + inputIdxs []uint32 + vecs []coldata.Vec + framer windowFramer +} + +type windowAggregator struct { + windowAggregatorBase + agg colexecagg.AggregateFunc +} + +type slidingWindowAggregator struct { + windowAggregatorBase + agg slidingWindowAggregateFunc +} + +var ( + _ bufferedWindower = &windowAggregator{} + _ bufferedWindower = &slidingWindowAggregator{} +) + +// windowInterval represents rows in the range [start, end). Slices of +// windowIntervals should always be increasing and non-overlapping. +type windowInterval struct { + start int + end int +} + +// transitionToProcessing implements the bufferedWindower interface. +func (a *windowAggregatorBase) transitionToProcessing() { + a.framer.startPartition(a.Ctx, a.partitionSize, a.buffer) +} + +// startNewPartition implements the bufferedWindower interface. +func (a *windowAggregatorBase) startNewPartition() { + a.partitionSize = 0 + a.buffer.Reset(a.Ctx) +} + +// Init implements the bufferedWindower interface. +func (a *windowAggregatorBase) Init(ctx context.Context) { + a.InitHelper.Init(ctx) +} + +// Close implements the bufferedWindower interface. +func (a *windowAggregatorBase) Close() { + if !a.CloserHelper.Close() { + return + } + if err := a.closers.Close(); err != nil { + colexecerror.InternalError(err) + } + a.framer.close() + a.buffer.Close(a.EnsureCtx()) +} + +func (a *windowAggregator) startNewPartition() { + a.windowAggregatorBase.startNewPartition() + a.agg.Reset() +} + +func (a *windowAggregator) Close() { + a.windowAggregatorBase.Close() + a.agg.Reset() + *a = windowAggregator{} +} + +// processBatch implements the bufferedWindower interface. +func (a *windowAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + outVec := batch.ColVec(a.outputColIdx) + a.agg.SetOutput(outVec) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + aggregateOverIntervals(a.framer.frameIntervals(), false /* removeRows */) + a.agg.Flush(i) + a.agg.Reset() + } + }) +} + +func (a *slidingWindowAggregator) startNewPartition() { + a.windowAggregatorBase.startNewPartition() + a.agg.Reset() +} + +func (a *slidingWindowAggregator) Close() { + a.windowAggregatorBase.Close() + a.agg.Reset() + *a = slidingWindowAggregator{} +} + +// processBatch implements the bufferedWindower interface. +func (a *slidingWindowAggregator) processBatch(batch coldata.Batch, startIdx, endIdx int) { + outVec := batch.ColVec(a.outputColIdx) + a.agg.SetOutput(outVec) + a.allocator.PerformOperation([]coldata.Vec{outVec}, func() { + for i := startIdx; i < endIdx; i++ { + a.framer.next(a.Ctx) + toAdd, toRemove := a.framer.slidingWindowIntervals() + // Process the 'toRemove' intervals first to avoid overflowing. + aggregateOverIntervals(toRemove, true /* removeRows */) + aggregateOverIntervals(toAdd, false /* removeRows */) + a.agg.Flush(i) + } + }) +} + +// execgen:inline +// execgen:template +func aggregateOverIntervals(intervals []windowInterval, removeRows bool) { + for _, interval := range intervals { + // intervalIdx maintains the index up to which the current interval has + // already been processed. + intervalIdx := interval.start + start, end := interval.start, interval.end + intervalLen := interval.end - interval.start + for intervalLen > 0 { + for j, idx := range a.inputIdxs { + a.vecs[j], start, end = a.buffer.GetVecWithTuple(a.Ctx, int(idx), intervalIdx) + } + if intervalLen < (end - start) { + // This is the last batch in the current interval. + end = start + intervalLen + } + intervalIdx += end - start + intervalLen -= end - start + if removeRows { + a.agg.Remove(a.vecs, a.inputIdxs, start, end) + } else { + a.agg.Compute(a.vecs, a.inputIdxs, start, end, nil /* sel */) + } + } + } +} diff --git a/pkg/sql/colexec/colexecwindow/window_framer.eg.go b/pkg/sql/colexec/colexecwindow/window_framer.eg.go index babdd66eb543..b4111a700729 100644 --- a/pkg/sql/colexec/colexecwindow/window_framer.eg.go +++ b/pkg/sql/colexec/colexecwindow/window_framer.eg.go @@ -65,9 +65,18 @@ type windowFramer interface { // frameIntervals returns a series of intervals that describes the set of all // rows that are part of the frame for the current row. Note that there are at // most three intervals - this case can occur when EXCLUDE TIES is used. - // frameIntervals is used to compute aggregate functions over a window. + // frameIntervals is used to compute aggregate functions over a window. The + // returned intervals cannot be modified. frameIntervals() []windowInterval + // slidingWindowIntervals returns a pair of interval sets that describes the + // rows that should be added to the current aggregation, and those which + // should be removed from the current aggregation. It is used to implement the + // sliding window optimization for aggregate window functions. toAdd specifies + // the rows that should be accumulated in the current aggregation, and + // toRemove specifies those which should be removed. + slidingWindowIntervals() (toAdd, toRemove []windowInterval) + // close should always be called upon closing of the parent operator. It // releases all references to enable garbage collection. close() @@ -1084,7 +1093,13 @@ type windowFramerBase struct { // intervals is a small (at most length 3) slice that is used during // aggregation computation. - intervals []windowInterval + intervals []windowInterval + intervalsAreSet bool + + // prevIntervals, toAdd, and toRemove are used to calculate the intervals + // for calculating aggregate window functions using the sliding window + // optimization. + prevIntervals, toAdd, toRemove []windowInterval } // frameFirstIdx returns the index of the first row in the window frame for @@ -1130,7 +1145,14 @@ func (b *windowFramerBase) frameNthIdx(n int) (idx int) { // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (b *windowFramerBase) frameIntervals() []windowInterval { + if b.intervalsAreSet { + return b.intervals + } + b.intervalsAreSet = true b.intervals = b.intervals[:0] + if b.startIdx >= b.endIdx { + return b.intervals + } b.intervals = append(b.intervals, windowInterval{start: b.startIdx, end: b.endIdx}) return b.intervals } @@ -1174,6 +1196,10 @@ func (b *windowFramerBase) startPartition( b.storedCols = storedCols b.startIdx = 0 b.endIdx = 0 + b.intervals = b.intervals[:0] + b.prevIntervals = b.prevIntervals[:0] + b.toAdd = b.toAdd[:0] + b.toRemove = b.toRemove[:0] } // incrementPeerGroup increments the given index by 'groups' peer groups, @@ -1386,6 +1412,122 @@ func (b *windowFramerBase) excludeTies() bool { return b.exclusion == execinfrapb.WindowerSpec_Frame_EXCLUDE_TIES } +// getSlidingWindowIntervals is a helper function used to calculate the sets of +// rows that are a part of the current window frame, but not the previous one, +// and rows that were a part of the previous window frame, but not the current +// one. getSlidingWindowIntervals expects the intervals stored in currIntervals +// and prevIntervals to be non-overlapping and increasing, and guarantees the +// same invariants for the output intervals. +func getSlidingWindowIntervals( + currIntervals, prevIntervals, toAdd, toRemove []windowInterval, +) ([]windowInterval, []windowInterval) { + toAdd, toRemove = toAdd[:0], toRemove[:0] + var prevIdx, currIdx int + var prev, curr windowInterval + setPrev, setCurr := true, true + for { + // We need to find the set difference currIntervals \ prevIntervals (toAdd) + // and the set difference prevIntervals \ currIntervals (toRemove). To do + // this, take advantage of the fact that both sets of intervals are in + // ascending order, similar to merging sorted lists. Maintain indices into + // each list, and iterate whichever index has the 'smaller' interval + // (e.g. whichever ends first). The portions of the intervals that overlap + // are ignored, while those that don't are added to one of the 'toAdd' and + // 'toRemove' sets. + if prevIdx >= len(prevIntervals) { + // None of the remaining intervals in the current frame were part of the + // previous frame. + if !setCurr { + // The remaining interval stored in curr still hasn't been handled. + toAdd = append(toAdd, curr) + currIdx++ + } + if currIdx < len(currIntervals) { + toAdd = append(toAdd, currIntervals[currIdx:]...) + } + break + } + if currIdx >= len(currIntervals) { + // None of the remaining intervals in the previous frame are part of the + // current frame. + if !setPrev { + // The remaining interval stored in prev still hasn't been handled. + toRemove = append(toRemove, prev) + prevIdx++ + } + if prevIdx < len(prevIntervals) { + toRemove = append(toRemove, prevIntervals[prevIdx:]...) + } + break + } + if setPrev { + prev = prevIntervals[prevIdx] + setPrev = false + } + if setCurr { + curr = currIntervals[currIdx] + setCurr = false + } + if prev == curr { + // This interval has not changed from the previous frame. + prevIdx++ + currIdx++ + setPrev, setCurr = true, true + continue + } + if prev.start >= curr.end { + // The intervals do not overlap, and the curr interval did not exist in + // the previous window frame. + toAdd = append(toAdd, curr) + currIdx++ + setCurr = true + continue + } + if curr.start >= prev.end { + // The intervals do not overlap, and the prev interval existed in the + // previous window frame, but not the current one. + toRemove = append(toRemove, prev) + prevIdx++ + setPrev = true + continue + } + // The intervals overlap but are not equal. + if curr.start < prev.start { + // curr starts before prev. Add the prefix of curr to 'toAdd'. Advance the + // start of curr to the start of prev to reflect that the prefix has + // already been processed. + toAdd = append(toAdd, windowInterval{start: curr.start, end: prev.start}) + curr.start = prev.start + } else if prev.start < curr.start { + // prev starts before curr. Add the prefix of prev to 'toRemove'. Advance + // the start of prev to the start of curr to reflect that the prefix has + // already been processed. + toRemove = append(toRemove, windowInterval{start: prev.start, end: curr.start}) + prev.start = curr.start + } + if curr.end > prev.end { + // prev ends before curr. Set the start of curr to the end of prev to + // indicate that prev has been processed. + curr.start = prev.end + prevIdx++ + setPrev = true + } else if prev.end > curr.end { + // curr ends before prev. Set the start of prev to the end of curr to + // indicate that curr has been processed. + prev.start = curr.end + currIdx++ + setCurr = true + } else { + // prev and curr end at the same index. The prefix of whichever one starts + // first has already been handled. + prevIdx++ + currIdx++ + setPrev, setCurr = true, true + } + } + return toAdd, toRemove +} + type windowFramerRowsUnboundedPrecedingOffsetPreceding struct { windowFramerBase } @@ -1413,12 +1555,25 @@ func (f *windowFramerRowsUnboundedPrecedingOffsetPreceding) next(ctx context.Con if f.endIdx < 0 { f.endIdx = 0 } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsUnboundedPrecedingOffsetPreceding) close() { *f = windowFramerRowsUnboundedPrecedingOffsetPreceding{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsUnboundedPrecedingOffsetPreceding) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRowsUnboundedPrecedingOffsetPrecedingExclude struct { windowFramerBase } @@ -1451,12 +1606,25 @@ func (f *windowFramerRowsUnboundedPrecedingOffsetPrecedingExclude) next(ctx cont } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsUnboundedPrecedingOffsetPrecedingExclude) close() { *f = windowFramerRowsUnboundedPrecedingOffsetPrecedingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsUnboundedPrecedingOffsetPrecedingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRowsUnboundedPrecedingOffsetPrecedingExclude) frameFirstIdx() (idx int) { @@ -1484,6 +1652,10 @@ func (f *windowFramerRowsUnboundedPrecedingOffsetPrecedingExclude) frameNthIdx(n // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRowsUnboundedPrecedingOffsetPrecedingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -1525,12 +1697,25 @@ func (f *windowFramerRowsUnboundedPrecedingCurrentRow) next(ctx context.Context) // Handle the end bound. f.endIdx = f.currentRow + 1 + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsUnboundedPrecedingCurrentRow) close() { *f = windowFramerRowsUnboundedPrecedingCurrentRow{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsUnboundedPrecedingCurrentRow) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRowsUnboundedPrecedingCurrentRowExclude struct { windowFramerBase } @@ -1560,12 +1745,25 @@ func (f *windowFramerRowsUnboundedPrecedingCurrentRowExclude) next(ctx context.C f.endIdx = f.currentRow + 1 // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsUnboundedPrecedingCurrentRowExclude) close() { *f = windowFramerRowsUnboundedPrecedingCurrentRowExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsUnboundedPrecedingCurrentRowExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRowsUnboundedPrecedingCurrentRowExclude) frameFirstIdx() (idx int) { @@ -1593,6 +1791,10 @@ func (f *windowFramerRowsUnboundedPrecedingCurrentRowExclude) frameNthIdx(n int) // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRowsUnboundedPrecedingCurrentRowExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -1639,12 +1841,25 @@ func (f *windowFramerRowsUnboundedPrecedingOffsetFollowing) next(ctx context.Con // overflow when offset is very large. f.endIdx = f.partitionSize } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsUnboundedPrecedingOffsetFollowing) close() { *f = windowFramerRowsUnboundedPrecedingOffsetFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsUnboundedPrecedingOffsetFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRowsUnboundedPrecedingOffsetFollowingExclude struct { windowFramerBase } @@ -1679,12 +1894,25 @@ func (f *windowFramerRowsUnboundedPrecedingOffsetFollowingExclude) next(ctx cont } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsUnboundedPrecedingOffsetFollowingExclude) close() { *f = windowFramerRowsUnboundedPrecedingOffsetFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsUnboundedPrecedingOffsetFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRowsUnboundedPrecedingOffsetFollowingExclude) frameFirstIdx() (idx int) { @@ -1712,6 +1940,10 @@ func (f *windowFramerRowsUnboundedPrecedingOffsetFollowingExclude) frameNthIdx(n // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRowsUnboundedPrecedingOffsetFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -1753,12 +1985,25 @@ func (f *windowFramerRowsUnboundedPrecedingUnboundedFollowing) next(ctx context. // Handle the end bound. f.endIdx = f.partitionSize + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsUnboundedPrecedingUnboundedFollowing) close() { *f = windowFramerRowsUnboundedPrecedingUnboundedFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsUnboundedPrecedingUnboundedFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRowsUnboundedPrecedingUnboundedFollowingExclude struct { windowFramerBase } @@ -1788,12 +2033,25 @@ func (f *windowFramerRowsUnboundedPrecedingUnboundedFollowingExclude) next(ctx c f.endIdx = f.partitionSize // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsUnboundedPrecedingUnboundedFollowingExclude) close() { *f = windowFramerRowsUnboundedPrecedingUnboundedFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsUnboundedPrecedingUnboundedFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRowsUnboundedPrecedingUnboundedFollowingExclude) frameFirstIdx() (idx int) { @@ -1821,6 +2079,10 @@ func (f *windowFramerRowsUnboundedPrecedingUnboundedFollowingExclude) frameNthId // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRowsUnboundedPrecedingUnboundedFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -1868,12 +2130,25 @@ func (f *windowFramerRowsOffsetPrecedingOffsetPreceding) next(ctx context.Contex if f.endIdx < 0 { f.endIdx = 0 } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsOffsetPrecedingOffsetPreceding) close() { *f = windowFramerRowsOffsetPrecedingOffsetPreceding{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsOffsetPrecedingOffsetPreceding) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRowsOffsetPrecedingOffsetPrecedingExclude struct { windowFramerBase } @@ -1909,12 +2184,25 @@ func (f *windowFramerRowsOffsetPrecedingOffsetPrecedingExclude) next(ctx context } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsOffsetPrecedingOffsetPrecedingExclude) close() { *f = windowFramerRowsOffsetPrecedingOffsetPrecedingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsOffsetPrecedingOffsetPrecedingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRowsOffsetPrecedingOffsetPrecedingExclude) frameFirstIdx() (idx int) { @@ -1942,6 +2230,10 @@ func (f *windowFramerRowsOffsetPrecedingOffsetPrecedingExclude) frameNthIdx(n in // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRowsOffsetPrecedingOffsetPrecedingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -1986,12 +2278,25 @@ func (f *windowFramerRowsOffsetPrecedingCurrentRow) next(ctx context.Context) { // Handle the end bound. f.endIdx = f.currentRow + 1 + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsOffsetPrecedingCurrentRow) close() { *f = windowFramerRowsOffsetPrecedingCurrentRow{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsOffsetPrecedingCurrentRow) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRowsOffsetPrecedingCurrentRowExclude struct { windowFramerBase } @@ -2024,12 +2329,25 @@ func (f *windowFramerRowsOffsetPrecedingCurrentRowExclude) next(ctx context.Cont f.endIdx = f.currentRow + 1 // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsOffsetPrecedingCurrentRowExclude) close() { *f = windowFramerRowsOffsetPrecedingCurrentRowExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsOffsetPrecedingCurrentRowExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRowsOffsetPrecedingCurrentRowExclude) frameFirstIdx() (idx int) { @@ -2057,6 +2375,10 @@ func (f *windowFramerRowsOffsetPrecedingCurrentRowExclude) frameNthIdx(n int) (i // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRowsOffsetPrecedingCurrentRowExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -2106,12 +2428,25 @@ func (f *windowFramerRowsOffsetPrecedingOffsetFollowing) next(ctx context.Contex // overflow when offset is very large. f.endIdx = f.partitionSize } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsOffsetPrecedingOffsetFollowing) close() { *f = windowFramerRowsOffsetPrecedingOffsetFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsOffsetPrecedingOffsetFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRowsOffsetPrecedingOffsetFollowingExclude struct { windowFramerBase } @@ -2149,12 +2484,25 @@ func (f *windowFramerRowsOffsetPrecedingOffsetFollowingExclude) next(ctx context } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsOffsetPrecedingOffsetFollowingExclude) close() { *f = windowFramerRowsOffsetPrecedingOffsetFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsOffsetPrecedingOffsetFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRowsOffsetPrecedingOffsetFollowingExclude) frameFirstIdx() (idx int) { @@ -2182,6 +2530,10 @@ func (f *windowFramerRowsOffsetPrecedingOffsetFollowingExclude) frameNthIdx(n in // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRowsOffsetPrecedingOffsetFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -2226,12 +2578,25 @@ func (f *windowFramerRowsOffsetPrecedingUnboundedFollowing) next(ctx context.Con // Handle the end bound. f.endIdx = f.partitionSize + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsOffsetPrecedingUnboundedFollowing) close() { *f = windowFramerRowsOffsetPrecedingUnboundedFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsOffsetPrecedingUnboundedFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRowsOffsetPrecedingUnboundedFollowingExclude struct { windowFramerBase } @@ -2264,12 +2629,25 @@ func (f *windowFramerRowsOffsetPrecedingUnboundedFollowingExclude) next(ctx cont f.endIdx = f.partitionSize // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsOffsetPrecedingUnboundedFollowingExclude) close() { *f = windowFramerRowsOffsetPrecedingUnboundedFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsOffsetPrecedingUnboundedFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRowsOffsetPrecedingUnboundedFollowingExclude) frameFirstIdx() (idx int) { @@ -2297,6 +2675,10 @@ func (f *windowFramerRowsOffsetPrecedingUnboundedFollowingExclude) frameNthIdx(n // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRowsOffsetPrecedingUnboundedFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -2338,12 +2720,25 @@ func (f *windowFramerRowsCurrentRowCurrentRow) next(ctx context.Context) { // Handle the end bound. f.endIdx = f.currentRow + 1 + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsCurrentRowCurrentRow) close() { *f = windowFramerRowsCurrentRowCurrentRow{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsCurrentRowCurrentRow) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRowsCurrentRowCurrentRowExclude struct { windowFramerBase } @@ -2373,12 +2768,25 @@ func (f *windowFramerRowsCurrentRowCurrentRowExclude) next(ctx context.Context) f.endIdx = f.currentRow + 1 // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsCurrentRowCurrentRowExclude) close() { *f = windowFramerRowsCurrentRowCurrentRowExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsCurrentRowCurrentRowExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRowsCurrentRowCurrentRowExclude) frameFirstIdx() (idx int) { @@ -2406,6 +2814,10 @@ func (f *windowFramerRowsCurrentRowCurrentRowExclude) frameNthIdx(n int) (idx in // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRowsCurrentRowCurrentRowExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -2452,12 +2864,25 @@ func (f *windowFramerRowsCurrentRowOffsetFollowing) next(ctx context.Context) { // overflow when offset is very large. f.endIdx = f.partitionSize } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsCurrentRowOffsetFollowing) close() { *f = windowFramerRowsCurrentRowOffsetFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsCurrentRowOffsetFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRowsCurrentRowOffsetFollowingExclude struct { windowFramerBase } @@ -2492,12 +2917,25 @@ func (f *windowFramerRowsCurrentRowOffsetFollowingExclude) next(ctx context.Cont } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsCurrentRowOffsetFollowingExclude) close() { *f = windowFramerRowsCurrentRowOffsetFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsCurrentRowOffsetFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRowsCurrentRowOffsetFollowingExclude) frameFirstIdx() (idx int) { @@ -2525,6 +2963,10 @@ func (f *windowFramerRowsCurrentRowOffsetFollowingExclude) frameNthIdx(n int) (i // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRowsCurrentRowOffsetFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -2566,12 +3008,25 @@ func (f *windowFramerRowsCurrentRowUnboundedFollowing) next(ctx context.Context) // Handle the end bound. f.endIdx = f.partitionSize + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsCurrentRowUnboundedFollowing) close() { *f = windowFramerRowsCurrentRowUnboundedFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsCurrentRowUnboundedFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRowsCurrentRowUnboundedFollowingExclude struct { windowFramerBase } @@ -2601,12 +3056,25 @@ func (f *windowFramerRowsCurrentRowUnboundedFollowingExclude) next(ctx context.C f.endIdx = f.partitionSize // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsCurrentRowUnboundedFollowingExclude) close() { *f = windowFramerRowsCurrentRowUnboundedFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsCurrentRowUnboundedFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRowsCurrentRowUnboundedFollowingExclude) frameFirstIdx() (idx int) { @@ -2634,6 +3102,10 @@ func (f *windowFramerRowsCurrentRowUnboundedFollowingExclude) frameNthIdx(n int) // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRowsCurrentRowUnboundedFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -2685,12 +3157,25 @@ func (f *windowFramerRowsOffsetFollowingOffsetFollowing) next(ctx context.Contex // overflow when offset is very large. f.endIdx = f.partitionSize } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsOffsetFollowingOffsetFollowing) close() { *f = windowFramerRowsOffsetFollowingOffsetFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsOffsetFollowingOffsetFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRowsOffsetFollowingOffsetFollowingExclude struct { windowFramerBase } @@ -2730,12 +3215,25 @@ func (f *windowFramerRowsOffsetFollowingOffsetFollowingExclude) next(ctx context } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsOffsetFollowingOffsetFollowingExclude) close() { *f = windowFramerRowsOffsetFollowingOffsetFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsOffsetFollowingOffsetFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRowsOffsetFollowingOffsetFollowingExclude) frameFirstIdx() (idx int) { @@ -2763,6 +3261,10 @@ func (f *windowFramerRowsOffsetFollowingOffsetFollowingExclude) frameNthIdx(n in // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRowsOffsetFollowingOffsetFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -2809,12 +3311,25 @@ func (f *windowFramerRowsOffsetFollowingUnboundedFollowing) next(ctx context.Con // Handle the end bound. f.endIdx = f.partitionSize + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsOffsetFollowingUnboundedFollowing) close() { *f = windowFramerRowsOffsetFollowingUnboundedFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsOffsetFollowingUnboundedFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRowsOffsetFollowingUnboundedFollowingExclude struct { windowFramerBase } @@ -2849,12 +3364,25 @@ func (f *windowFramerRowsOffsetFollowingUnboundedFollowingExclude) next(ctx cont f.endIdx = f.partitionSize // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRowsOffsetFollowingUnboundedFollowingExclude) close() { *f = windowFramerRowsOffsetFollowingUnboundedFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRowsOffsetFollowingUnboundedFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRowsOffsetFollowingUnboundedFollowingExclude) frameFirstIdx() (idx int) { @@ -2882,6 +3410,10 @@ func (f *windowFramerRowsOffsetFollowingUnboundedFollowingExclude) frameNthIdx(n // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRowsOffsetFollowingUnboundedFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -2931,12 +3463,25 @@ func (f *windowFramerGroupsUnboundedPrecedingOffsetPreceding) next(ctx context.C if currRowIsGroupStart { f.currentGroup++ } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsUnboundedPrecedingOffsetPreceding) close() { *f = windowFramerGroupsUnboundedPrecedingOffsetPreceding{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsUnboundedPrecedingOffsetPreceding) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerGroupsUnboundedPrecedingOffsetPrecedingExclude struct { windowFramerBase } @@ -2973,12 +3518,25 @@ func (f *windowFramerGroupsUnboundedPrecedingOffsetPrecedingExclude) next(ctx co } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsUnboundedPrecedingOffsetPrecedingExclude) close() { *f = windowFramerGroupsUnboundedPrecedingOffsetPrecedingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsUnboundedPrecedingOffsetPrecedingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerGroupsUnboundedPrecedingOffsetPrecedingExclude) frameFirstIdx() (idx int) { @@ -3006,6 +3564,10 @@ func (f *windowFramerGroupsUnboundedPrecedingOffsetPrecedingExclude) frameNthIdx // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerGroupsUnboundedPrecedingOffsetPrecedingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -3050,12 +3612,25 @@ func (f *windowFramerGroupsUnboundedPrecedingCurrentRow) next(ctx context.Contex if currRowIsGroupStart { f.endIdx = f.incrementPeerGroup(ctx, f.endIdx, 1 /* groups */) } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsUnboundedPrecedingCurrentRow) close() { *f = windowFramerGroupsUnboundedPrecedingCurrentRow{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsUnboundedPrecedingCurrentRow) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerGroupsUnboundedPrecedingCurrentRowExclude struct { windowFramerBase } @@ -3087,12 +3662,25 @@ func (f *windowFramerGroupsUnboundedPrecedingCurrentRowExclude) next(ctx context } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsUnboundedPrecedingCurrentRowExclude) close() { *f = windowFramerGroupsUnboundedPrecedingCurrentRowExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsUnboundedPrecedingCurrentRowExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerGroupsUnboundedPrecedingCurrentRowExclude) frameFirstIdx() (idx int) { @@ -3120,6 +3708,10 @@ func (f *windowFramerGroupsUnboundedPrecedingCurrentRowExclude) frameNthIdx(n in // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerGroupsUnboundedPrecedingCurrentRowExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -3169,12 +3761,25 @@ func (f *windowFramerGroupsUnboundedPrecedingOffsetFollowing) next(ctx context.C // whenever the currentRow pointer enters a new peers group. f.endIdx = f.incrementPeerGroup(ctx, f.endIdx, 1 /* groups */) } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsUnboundedPrecedingOffsetFollowing) close() { *f = windowFramerGroupsUnboundedPrecedingOffsetFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsUnboundedPrecedingOffsetFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerGroupsUnboundedPrecedingOffsetFollowingExclude struct { windowFramerBase } @@ -3211,12 +3816,25 @@ func (f *windowFramerGroupsUnboundedPrecedingOffsetFollowingExclude) next(ctx co } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsUnboundedPrecedingOffsetFollowingExclude) close() { *f = windowFramerGroupsUnboundedPrecedingOffsetFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsUnboundedPrecedingOffsetFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerGroupsUnboundedPrecedingOffsetFollowingExclude) frameFirstIdx() (idx int) { @@ -3244,6 +3862,10 @@ func (f *windowFramerGroupsUnboundedPrecedingOffsetFollowingExclude) frameNthIdx // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerGroupsUnboundedPrecedingOffsetFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -3285,12 +3907,25 @@ func (f *windowFramerGroupsUnboundedPrecedingUnboundedFollowing) next(ctx contex // Handle the end bound. f.endIdx = f.partitionSize + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsUnboundedPrecedingUnboundedFollowing) close() { *f = windowFramerGroupsUnboundedPrecedingUnboundedFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsUnboundedPrecedingUnboundedFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerGroupsUnboundedPrecedingUnboundedFollowingExclude struct { windowFramerBase } @@ -3320,12 +3955,25 @@ func (f *windowFramerGroupsUnboundedPrecedingUnboundedFollowingExclude) next(ctx f.endIdx = f.partitionSize // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsUnboundedPrecedingUnboundedFollowingExclude) close() { *f = windowFramerGroupsUnboundedPrecedingUnboundedFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsUnboundedPrecedingUnboundedFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerGroupsUnboundedPrecedingUnboundedFollowingExclude) frameFirstIdx() (idx int) { @@ -3353,6 +4001,10 @@ func (f *windowFramerGroupsUnboundedPrecedingUnboundedFollowingExclude) frameNth // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerGroupsUnboundedPrecedingUnboundedFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -3405,12 +4057,25 @@ func (f *windowFramerGroupsOffsetPrecedingOffsetPreceding) next(ctx context.Cont if currRowIsGroupStart { f.currentGroup++ } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsOffsetPrecedingOffsetPreceding) close() { *f = windowFramerGroupsOffsetPrecedingOffsetPreceding{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsOffsetPrecedingOffsetPreceding) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerGroupsOffsetPrecedingOffsetPrecedingExclude struct { windowFramerBase } @@ -3450,12 +4115,25 @@ func (f *windowFramerGroupsOffsetPrecedingOffsetPrecedingExclude) next(ctx conte } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsOffsetPrecedingOffsetPrecedingExclude) close() { *f = windowFramerGroupsOffsetPrecedingOffsetPrecedingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsOffsetPrecedingOffsetPrecedingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerGroupsOffsetPrecedingOffsetPrecedingExclude) frameFirstIdx() (idx int) { @@ -3483,6 +4161,10 @@ func (f *windowFramerGroupsOffsetPrecedingOffsetPrecedingExclude) frameNthIdx(n // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerGroupsOffsetPrecedingOffsetPrecedingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -3534,12 +4216,25 @@ func (f *windowFramerGroupsOffsetPrecedingCurrentRow) next(ctx context.Context) if currRowIsGroupStart { f.currentGroup++ } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsOffsetPrecedingCurrentRow) close() { *f = windowFramerGroupsOffsetPrecedingCurrentRow{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsOffsetPrecedingCurrentRow) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerGroupsOffsetPrecedingCurrentRowExclude struct { windowFramerBase } @@ -3578,12 +4273,25 @@ func (f *windowFramerGroupsOffsetPrecedingCurrentRowExclude) next(ctx context.Co } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsOffsetPrecedingCurrentRowExclude) close() { *f = windowFramerGroupsOffsetPrecedingCurrentRowExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsOffsetPrecedingCurrentRowExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerGroupsOffsetPrecedingCurrentRowExclude) frameFirstIdx() (idx int) { @@ -3611,6 +4319,10 @@ func (f *windowFramerGroupsOffsetPrecedingCurrentRowExclude) frameNthIdx(n int) // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerGroupsOffsetPrecedingCurrentRowExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -3667,12 +4379,25 @@ func (f *windowFramerGroupsOffsetPrecedingOffsetFollowing) next(ctx context.Cont if currRowIsGroupStart { f.currentGroup++ } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsOffsetPrecedingOffsetFollowing) close() { *f = windowFramerGroupsOffsetPrecedingOffsetFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsOffsetPrecedingOffsetFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerGroupsOffsetPrecedingOffsetFollowingExclude struct { windowFramerBase } @@ -3716,12 +4441,25 @@ func (f *windowFramerGroupsOffsetPrecedingOffsetFollowingExclude) next(ctx conte } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsOffsetPrecedingOffsetFollowingExclude) close() { *f = windowFramerGroupsOffsetPrecedingOffsetFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsOffsetPrecedingOffsetFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerGroupsOffsetPrecedingOffsetFollowingExclude) frameFirstIdx() (idx int) { @@ -3749,6 +4487,10 @@ func (f *windowFramerGroupsOffsetPrecedingOffsetFollowingExclude) frameNthIdx(n // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerGroupsOffsetPrecedingOffsetFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -3798,12 +4540,25 @@ func (f *windowFramerGroupsOffsetPrecedingUnboundedFollowing) next(ctx context.C if currRowIsGroupStart { f.currentGroup++ } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsOffsetPrecedingUnboundedFollowing) close() { *f = windowFramerGroupsOffsetPrecedingUnboundedFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsOffsetPrecedingUnboundedFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerGroupsOffsetPrecedingUnboundedFollowingExclude struct { windowFramerBase } @@ -3840,12 +4595,25 @@ func (f *windowFramerGroupsOffsetPrecedingUnboundedFollowingExclude) next(ctx co } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsOffsetPrecedingUnboundedFollowingExclude) close() { *f = windowFramerGroupsOffsetPrecedingUnboundedFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsOffsetPrecedingUnboundedFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerGroupsOffsetPrecedingUnboundedFollowingExclude) frameFirstIdx() (idx int) { @@ -3873,6 +4641,10 @@ func (f *windowFramerGroupsOffsetPrecedingUnboundedFollowingExclude) frameNthIdx // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerGroupsOffsetPrecedingUnboundedFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -3919,12 +4691,25 @@ func (f *windowFramerGroupsCurrentRowCurrentRow) next(ctx context.Context) { if currRowIsGroupStart { f.endIdx = f.incrementPeerGroup(ctx, f.endIdx, 1 /* groups */) } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsCurrentRowCurrentRow) close() { *f = windowFramerGroupsCurrentRowCurrentRow{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsCurrentRowCurrentRow) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerGroupsCurrentRowCurrentRowExclude struct { windowFramerBase } @@ -3958,12 +4743,25 @@ func (f *windowFramerGroupsCurrentRowCurrentRowExclude) next(ctx context.Context } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsCurrentRowCurrentRowExclude) close() { *f = windowFramerGroupsCurrentRowCurrentRowExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsCurrentRowCurrentRowExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerGroupsCurrentRowCurrentRowExclude) frameFirstIdx() (idx int) { @@ -3991,6 +4789,10 @@ func (f *windowFramerGroupsCurrentRowCurrentRowExclude) frameNthIdx(n int) (idx // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerGroupsCurrentRowCurrentRowExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -4042,12 +4844,25 @@ func (f *windowFramerGroupsCurrentRowOffsetFollowing) next(ctx context.Context) // whenever the currentRow pointer enters a new peers group. f.endIdx = f.incrementPeerGroup(ctx, f.endIdx, 1 /* groups */) } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsCurrentRowOffsetFollowing) close() { *f = windowFramerGroupsCurrentRowOffsetFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsCurrentRowOffsetFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerGroupsCurrentRowOffsetFollowingExclude struct { windowFramerBase } @@ -4086,12 +4901,25 @@ func (f *windowFramerGroupsCurrentRowOffsetFollowingExclude) next(ctx context.Co } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsCurrentRowOffsetFollowingExclude) close() { *f = windowFramerGroupsCurrentRowOffsetFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsCurrentRowOffsetFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerGroupsCurrentRowOffsetFollowingExclude) frameFirstIdx() (idx int) { @@ -4119,6 +4947,10 @@ func (f *windowFramerGroupsCurrentRowOffsetFollowingExclude) frameNthIdx(n int) // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerGroupsCurrentRowOffsetFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -4163,12 +4995,25 @@ func (f *windowFramerGroupsCurrentRowUnboundedFollowing) next(ctx context.Contex // Handle the end bound. f.endIdx = f.partitionSize + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsCurrentRowUnboundedFollowing) close() { *f = windowFramerGroupsCurrentRowUnboundedFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsCurrentRowUnboundedFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerGroupsCurrentRowUnboundedFollowingExclude struct { windowFramerBase } @@ -4200,12 +5045,25 @@ func (f *windowFramerGroupsCurrentRowUnboundedFollowingExclude) next(ctx context f.endIdx = f.partitionSize // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsCurrentRowUnboundedFollowingExclude) close() { *f = windowFramerGroupsCurrentRowUnboundedFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsCurrentRowUnboundedFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerGroupsCurrentRowUnboundedFollowingExclude) frameFirstIdx() (idx int) { @@ -4233,6 +5091,10 @@ func (f *windowFramerGroupsCurrentRowUnboundedFollowingExclude) frameNthIdx(n in // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerGroupsCurrentRowUnboundedFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -4289,12 +5151,25 @@ func (f *windowFramerGroupsOffsetFollowingOffsetFollowing) next(ctx context.Cont // whenever the currentRow pointer enters a new peers group. f.endIdx = f.incrementPeerGroup(ctx, f.endIdx, 1 /* groups */) } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsOffsetFollowingOffsetFollowing) close() { *f = windowFramerGroupsOffsetFollowingOffsetFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsOffsetFollowingOffsetFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerGroupsOffsetFollowingOffsetFollowingExclude struct { windowFramerBase } @@ -4338,12 +5213,25 @@ func (f *windowFramerGroupsOffsetFollowingOffsetFollowingExclude) next(ctx conte } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsOffsetFollowingOffsetFollowingExclude) close() { *f = windowFramerGroupsOffsetFollowingOffsetFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsOffsetFollowingOffsetFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerGroupsOffsetFollowingOffsetFollowingExclude) frameFirstIdx() (idx int) { @@ -4371,6 +5259,10 @@ func (f *windowFramerGroupsOffsetFollowingOffsetFollowingExclude) frameNthIdx(n // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerGroupsOffsetFollowingOffsetFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -4420,12 +5312,25 @@ func (f *windowFramerGroupsOffsetFollowingUnboundedFollowing) next(ctx context.C // Handle the end bound. f.endIdx = f.partitionSize + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsOffsetFollowingUnboundedFollowing) close() { *f = windowFramerGroupsOffsetFollowingUnboundedFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsOffsetFollowingUnboundedFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerGroupsOffsetFollowingUnboundedFollowingExclude struct { windowFramerBase } @@ -4462,12 +5367,25 @@ func (f *windowFramerGroupsOffsetFollowingUnboundedFollowingExclude) next(ctx co f.endIdx = f.partitionSize // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerGroupsOffsetFollowingUnboundedFollowingExclude) close() { *f = windowFramerGroupsOffsetFollowingUnboundedFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerGroupsOffsetFollowingUnboundedFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerGroupsOffsetFollowingUnboundedFollowingExclude) frameFirstIdx() (idx int) { @@ -4495,6 +5413,10 @@ func (f *windowFramerGroupsOffsetFollowingUnboundedFollowingExclude) frameNthIdx // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerGroupsOffsetFollowingUnboundedFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -4540,6 +5462,8 @@ func (f *windowFramerRangeUnboundedPrecedingOffsetPreceding) next(ctx context.Co if currRowIsGroupStart { f.endIdx = f.endHandler.getIdx(ctx, f.currentRow, f.endIdx) } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeUnboundedPrecedingOffsetPreceding) close() { @@ -4547,6 +5471,17 @@ func (f *windowFramerRangeUnboundedPrecedingOffsetPreceding) close() { *f = windowFramerRangeUnboundedPrecedingOffsetPreceding{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeUnboundedPrecedingOffsetPreceding) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRangeUnboundedPrecedingOffsetPrecedingExclude struct { windowFramerBase } @@ -4579,6 +5514,8 @@ func (f *windowFramerRangeUnboundedPrecedingOffsetPrecedingExclude) next(ctx con } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeUnboundedPrecedingOffsetPrecedingExclude) close() { @@ -4586,6 +5523,17 @@ func (f *windowFramerRangeUnboundedPrecedingOffsetPrecedingExclude) close() { *f = windowFramerRangeUnboundedPrecedingOffsetPrecedingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeUnboundedPrecedingOffsetPrecedingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRangeUnboundedPrecedingOffsetPrecedingExclude) frameFirstIdx() (idx int) { @@ -4613,6 +5561,10 @@ func (f *windowFramerRangeUnboundedPrecedingOffsetPrecedingExclude) frameNthIdx( // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRangeUnboundedPrecedingOffsetPrecedingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -4657,12 +5609,25 @@ func (f *windowFramerRangeUnboundedPrecedingCurrentRow) next(ctx context.Context if currRowIsGroupStart { f.endIdx = f.incrementPeerGroup(ctx, f.endIdx, 1 /* groups */) } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeUnboundedPrecedingCurrentRow) close() { *f = windowFramerRangeUnboundedPrecedingCurrentRow{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeUnboundedPrecedingCurrentRow) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRangeUnboundedPrecedingCurrentRowExclude struct { windowFramerBase } @@ -4694,12 +5659,25 @@ func (f *windowFramerRangeUnboundedPrecedingCurrentRowExclude) next(ctx context. } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeUnboundedPrecedingCurrentRowExclude) close() { *f = windowFramerRangeUnboundedPrecedingCurrentRowExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeUnboundedPrecedingCurrentRowExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRangeUnboundedPrecedingCurrentRowExclude) frameFirstIdx() (idx int) { @@ -4727,6 +5705,10 @@ func (f *windowFramerRangeUnboundedPrecedingCurrentRowExclude) frameNthIdx(n int // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRangeUnboundedPrecedingCurrentRowExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -4772,6 +5754,8 @@ func (f *windowFramerRangeUnboundedPrecedingOffsetFollowing) next(ctx context.Co if currRowIsGroupStart { f.endIdx = f.endHandler.getIdx(ctx, f.currentRow, f.endIdx) } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeUnboundedPrecedingOffsetFollowing) close() { @@ -4779,6 +5763,17 @@ func (f *windowFramerRangeUnboundedPrecedingOffsetFollowing) close() { *f = windowFramerRangeUnboundedPrecedingOffsetFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeUnboundedPrecedingOffsetFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRangeUnboundedPrecedingOffsetFollowingExclude struct { windowFramerBase } @@ -4811,6 +5806,8 @@ func (f *windowFramerRangeUnboundedPrecedingOffsetFollowingExclude) next(ctx con } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeUnboundedPrecedingOffsetFollowingExclude) close() { @@ -4818,6 +5815,17 @@ func (f *windowFramerRangeUnboundedPrecedingOffsetFollowingExclude) close() { *f = windowFramerRangeUnboundedPrecedingOffsetFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeUnboundedPrecedingOffsetFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRangeUnboundedPrecedingOffsetFollowingExclude) frameFirstIdx() (idx int) { @@ -4845,6 +5853,10 @@ func (f *windowFramerRangeUnboundedPrecedingOffsetFollowingExclude) frameNthIdx( // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRangeUnboundedPrecedingOffsetFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -4886,12 +5898,25 @@ func (f *windowFramerRangeUnboundedPrecedingUnboundedFollowing) next(ctx context // Handle the end bound. f.endIdx = f.partitionSize + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeUnboundedPrecedingUnboundedFollowing) close() { *f = windowFramerRangeUnboundedPrecedingUnboundedFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeUnboundedPrecedingUnboundedFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRangeUnboundedPrecedingUnboundedFollowingExclude struct { windowFramerBase } @@ -4921,12 +5946,25 @@ func (f *windowFramerRangeUnboundedPrecedingUnboundedFollowingExclude) next(ctx f.endIdx = f.partitionSize // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeUnboundedPrecedingUnboundedFollowingExclude) close() { *f = windowFramerRangeUnboundedPrecedingUnboundedFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeUnboundedPrecedingUnboundedFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRangeUnboundedPrecedingUnboundedFollowingExclude) frameFirstIdx() (idx int) { @@ -4954,6 +5992,10 @@ func (f *windowFramerRangeUnboundedPrecedingUnboundedFollowingExclude) frameNthI // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRangeUnboundedPrecedingUnboundedFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -5002,6 +6044,8 @@ func (f *windowFramerRangeOffsetPrecedingOffsetPreceding) next(ctx context.Conte if currRowIsGroupStart { f.endIdx = f.endHandler.getIdx(ctx, f.currentRow, f.endIdx) } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeOffsetPrecedingOffsetPreceding) close() { @@ -5010,6 +6054,17 @@ func (f *windowFramerRangeOffsetPrecedingOffsetPreceding) close() { *f = windowFramerRangeOffsetPrecedingOffsetPreceding{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeOffsetPrecedingOffsetPreceding) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRangeOffsetPrecedingOffsetPrecedingExclude struct { windowFramerBase } @@ -5045,6 +6100,8 @@ func (f *windowFramerRangeOffsetPrecedingOffsetPrecedingExclude) next(ctx contex } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeOffsetPrecedingOffsetPrecedingExclude) close() { @@ -5053,6 +6110,17 @@ func (f *windowFramerRangeOffsetPrecedingOffsetPrecedingExclude) close() { *f = windowFramerRangeOffsetPrecedingOffsetPrecedingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeOffsetPrecedingOffsetPrecedingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRangeOffsetPrecedingOffsetPrecedingExclude) frameFirstIdx() (idx int) { @@ -5080,6 +6148,10 @@ func (f *windowFramerRangeOffsetPrecedingOffsetPrecedingExclude) frameNthIdx(n i // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRangeOffsetPrecedingOffsetPrecedingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -5127,6 +6199,8 @@ func (f *windowFramerRangeOffsetPrecedingCurrentRow) next(ctx context.Context) { if currRowIsGroupStart { f.endIdx = f.incrementPeerGroup(ctx, f.endIdx, 1 /* groups */) } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeOffsetPrecedingCurrentRow) close() { @@ -5134,6 +6208,17 @@ func (f *windowFramerRangeOffsetPrecedingCurrentRow) close() { *f = windowFramerRangeOffsetPrecedingCurrentRow{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeOffsetPrecedingCurrentRow) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRangeOffsetPrecedingCurrentRowExclude struct { windowFramerBase } @@ -5168,6 +6253,8 @@ func (f *windowFramerRangeOffsetPrecedingCurrentRowExclude) next(ctx context.Con } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeOffsetPrecedingCurrentRowExclude) close() { @@ -5175,6 +6262,17 @@ func (f *windowFramerRangeOffsetPrecedingCurrentRowExclude) close() { *f = windowFramerRangeOffsetPrecedingCurrentRowExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeOffsetPrecedingCurrentRowExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRangeOffsetPrecedingCurrentRowExclude) frameFirstIdx() (idx int) { @@ -5202,6 +6300,10 @@ func (f *windowFramerRangeOffsetPrecedingCurrentRowExclude) frameNthIdx(n int) ( // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRangeOffsetPrecedingCurrentRowExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -5250,6 +6352,8 @@ func (f *windowFramerRangeOffsetPrecedingOffsetFollowing) next(ctx context.Conte if currRowIsGroupStart { f.endIdx = f.endHandler.getIdx(ctx, f.currentRow, f.endIdx) } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeOffsetPrecedingOffsetFollowing) close() { @@ -5258,6 +6362,17 @@ func (f *windowFramerRangeOffsetPrecedingOffsetFollowing) close() { *f = windowFramerRangeOffsetPrecedingOffsetFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeOffsetPrecedingOffsetFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRangeOffsetPrecedingOffsetFollowingExclude struct { windowFramerBase } @@ -5293,6 +6408,8 @@ func (f *windowFramerRangeOffsetPrecedingOffsetFollowingExclude) next(ctx contex } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeOffsetPrecedingOffsetFollowingExclude) close() { @@ -5301,6 +6418,17 @@ func (f *windowFramerRangeOffsetPrecedingOffsetFollowingExclude) close() { *f = windowFramerRangeOffsetPrecedingOffsetFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeOffsetPrecedingOffsetFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRangeOffsetPrecedingOffsetFollowingExclude) frameFirstIdx() (idx int) { @@ -5328,6 +6456,10 @@ func (f *windowFramerRangeOffsetPrecedingOffsetFollowingExclude) frameNthIdx(n i // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRangeOffsetPrecedingOffsetFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -5373,6 +6505,8 @@ func (f *windowFramerRangeOffsetPrecedingUnboundedFollowing) next(ctx context.Co // Handle the end bound. f.endIdx = f.partitionSize + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeOffsetPrecedingUnboundedFollowing) close() { @@ -5380,6 +6514,17 @@ func (f *windowFramerRangeOffsetPrecedingUnboundedFollowing) close() { *f = windowFramerRangeOffsetPrecedingUnboundedFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeOffsetPrecedingUnboundedFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRangeOffsetPrecedingUnboundedFollowingExclude struct { windowFramerBase } @@ -5412,6 +6557,8 @@ func (f *windowFramerRangeOffsetPrecedingUnboundedFollowingExclude) next(ctx con f.endIdx = f.partitionSize // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeOffsetPrecedingUnboundedFollowingExclude) close() { @@ -5419,6 +6566,17 @@ func (f *windowFramerRangeOffsetPrecedingUnboundedFollowingExclude) close() { *f = windowFramerRangeOffsetPrecedingUnboundedFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeOffsetPrecedingUnboundedFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRangeOffsetPrecedingUnboundedFollowingExclude) frameFirstIdx() (idx int) { @@ -5446,6 +6604,10 @@ func (f *windowFramerRangeOffsetPrecedingUnboundedFollowingExclude) frameNthIdx( // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRangeOffsetPrecedingUnboundedFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -5492,12 +6654,25 @@ func (f *windowFramerRangeCurrentRowCurrentRow) next(ctx context.Context) { if currRowIsGroupStart { f.endIdx = f.incrementPeerGroup(ctx, f.endIdx, 1 /* groups */) } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeCurrentRowCurrentRow) close() { *f = windowFramerRangeCurrentRowCurrentRow{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeCurrentRowCurrentRow) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRangeCurrentRowCurrentRowExclude struct { windowFramerBase } @@ -5531,12 +6706,25 @@ func (f *windowFramerRangeCurrentRowCurrentRowExclude) next(ctx context.Context) } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeCurrentRowCurrentRowExclude) close() { *f = windowFramerRangeCurrentRowCurrentRowExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeCurrentRowCurrentRowExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRangeCurrentRowCurrentRowExclude) frameFirstIdx() (idx int) { @@ -5564,6 +6752,10 @@ func (f *windowFramerRangeCurrentRowCurrentRowExclude) frameNthIdx(n int) (idx i // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRangeCurrentRowCurrentRowExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -5611,6 +6803,8 @@ func (f *windowFramerRangeCurrentRowOffsetFollowing) next(ctx context.Context) { if currRowIsGroupStart { f.endIdx = f.endHandler.getIdx(ctx, f.currentRow, f.endIdx) } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeCurrentRowOffsetFollowing) close() { @@ -5618,6 +6812,17 @@ func (f *windowFramerRangeCurrentRowOffsetFollowing) close() { *f = windowFramerRangeCurrentRowOffsetFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeCurrentRowOffsetFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRangeCurrentRowOffsetFollowingExclude struct { windowFramerBase } @@ -5652,6 +6857,8 @@ func (f *windowFramerRangeCurrentRowOffsetFollowingExclude) next(ctx context.Con } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeCurrentRowOffsetFollowingExclude) close() { @@ -5659,6 +6866,17 @@ func (f *windowFramerRangeCurrentRowOffsetFollowingExclude) close() { *f = windowFramerRangeCurrentRowOffsetFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeCurrentRowOffsetFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRangeCurrentRowOffsetFollowingExclude) frameFirstIdx() (idx int) { @@ -5686,6 +6904,10 @@ func (f *windowFramerRangeCurrentRowOffsetFollowingExclude) frameNthIdx(n int) ( // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRangeCurrentRowOffsetFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -5730,12 +6952,25 @@ func (f *windowFramerRangeCurrentRowUnboundedFollowing) next(ctx context.Context // Handle the end bound. f.endIdx = f.partitionSize + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeCurrentRowUnboundedFollowing) close() { *f = windowFramerRangeCurrentRowUnboundedFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeCurrentRowUnboundedFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRangeCurrentRowUnboundedFollowingExclude struct { windowFramerBase } @@ -5767,12 +7002,25 @@ func (f *windowFramerRangeCurrentRowUnboundedFollowingExclude) next(ctx context. f.endIdx = f.partitionSize // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeCurrentRowUnboundedFollowingExclude) close() { *f = windowFramerRangeCurrentRowUnboundedFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeCurrentRowUnboundedFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRangeCurrentRowUnboundedFollowingExclude) frameFirstIdx() (idx int) { @@ -5800,6 +7048,10 @@ func (f *windowFramerRangeCurrentRowUnboundedFollowingExclude) frameNthIdx(n int // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRangeCurrentRowUnboundedFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -5848,6 +7100,8 @@ func (f *windowFramerRangeOffsetFollowingOffsetFollowing) next(ctx context.Conte if currRowIsGroupStart { f.endIdx = f.endHandler.getIdx(ctx, f.currentRow, f.endIdx) } + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeOffsetFollowingOffsetFollowing) close() { @@ -5856,6 +7110,17 @@ func (f *windowFramerRangeOffsetFollowingOffsetFollowing) close() { *f = windowFramerRangeOffsetFollowingOffsetFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeOffsetFollowingOffsetFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRangeOffsetFollowingOffsetFollowingExclude struct { windowFramerBase } @@ -5891,6 +7156,8 @@ func (f *windowFramerRangeOffsetFollowingOffsetFollowingExclude) next(ctx contex } // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeOffsetFollowingOffsetFollowingExclude) close() { @@ -5899,6 +7166,17 @@ func (f *windowFramerRangeOffsetFollowingOffsetFollowingExclude) close() { *f = windowFramerRangeOffsetFollowingOffsetFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeOffsetFollowingOffsetFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRangeOffsetFollowingOffsetFollowingExclude) frameFirstIdx() (idx int) { @@ -5926,6 +7204,10 @@ func (f *windowFramerRangeOffsetFollowingOffsetFollowingExclude) frameNthIdx(n i // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRangeOffsetFollowingOffsetFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() @@ -5971,6 +7253,8 @@ func (f *windowFramerRangeOffsetFollowingUnboundedFollowing) next(ctx context.Co // Handle the end bound. f.endIdx = f.partitionSize + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeOffsetFollowingUnboundedFollowing) close() { @@ -5978,6 +7262,17 @@ func (f *windowFramerRangeOffsetFollowingUnboundedFollowing) close() { *f = windowFramerRangeOffsetFollowingUnboundedFollowing{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeOffsetFollowingUnboundedFollowing) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + type windowFramerRangeOffsetFollowingUnboundedFollowingExclude struct { windowFramerBase } @@ -6010,6 +7305,8 @@ func (f *windowFramerRangeOffsetFollowingUnboundedFollowingExclude) next(ctx con f.endIdx = f.partitionSize // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *windowFramerRangeOffsetFollowingUnboundedFollowingExclude) close() { @@ -6017,6 +7314,17 @@ func (f *windowFramerRangeOffsetFollowingUnboundedFollowingExclude) close() { *f = windowFramerRangeOffsetFollowingUnboundedFollowingExclude{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *windowFramerRangeOffsetFollowingUnboundedFollowingExclude) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // frameFirstIdx returns the index of the first row in the window frame for // the current row. If no such row exists, frameFirstIdx returns -1. func (f *windowFramerRangeOffsetFollowingUnboundedFollowingExclude) frameFirstIdx() (idx int) { @@ -6044,6 +7352,10 @@ func (f *windowFramerRangeOffsetFollowingUnboundedFollowingExclude) frameNthIdx( // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *windowFramerRangeOffsetFollowingUnboundedFollowingExclude) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() diff --git a/pkg/sql/colexec/colexecwindow/window_framer_test.go b/pkg/sql/colexec/colexecwindow/window_framer_test.go index febf13ca7c6f..dcdf601bf6a2 100644 --- a/pkg/sql/colexec/colexecwindow/window_framer_test.go +++ b/pkg/sql/colexec/colexecwindow/window_framer_test.go @@ -521,3 +521,94 @@ func exclusionToExecinfrapb( } return 0 } + +func TestGetSlidingWindowIntervals(t *testing.T) { + testCases := []struct { + prevIntervals []windowInterval + currIntervals []windowInterval + expectedToAdd []windowInterval + expectedToRem []windowInterval + }{ + { + prevIntervals: []windowInterval{ + {0, 2}, + {4, 5}, + }, + currIntervals: []windowInterval{ + {0, 6}, + }, + expectedToAdd: []windowInterval{ + {2, 4}, + {5, 6}, + }, + expectedToRem: nil, + }, + { + prevIntervals: []windowInterval{ + {1, 2}, + }, + currIntervals: []windowInterval{ + {0, 4}, + }, + expectedToAdd: []windowInterval{ + {0, 1}, + {2, 4}, + }, + expectedToRem: nil, + }, + { + prevIntervals: []windowInterval{ + {0, 6}, + }, + currIntervals: []windowInterval{ + {1, 2}, + {5, 7}, + }, + expectedToAdd: []windowInterval{ + {6, 7}, + }, + expectedToRem: []windowInterval{ + {0, 1}, + {2, 5}, + }, + }, + { + prevIntervals: []windowInterval{ + {0, 2}, + {4, 5}, + {6, 8}, + }, + currIntervals: []windowInterval{ + {0, 2}, + {4, 5}, + {6, 8}, + }, + expectedToAdd: nil, + expectedToRem: nil, + }, + { + prevIntervals: []windowInterval{ + {0, 2}, + {4, 5}, + {6, 8}, + }, + currIntervals: []windowInterval{ + {0, 2}, + {4, 5}, + {6, 8}, + {9, 10}, + }, + expectedToAdd: []windowInterval{ + {9, 10}, + }, + expectedToRem: nil, + }, + } + + for i := range testCases { + tc := &testCases[i] + toAdd, toRemove := getSlidingWindowIntervals(tc.currIntervals, tc.prevIntervals, nil, nil) + require.Equalf(t, tc.expectedToAdd, toAdd, "toAdd") + require.Equalf(t, tc.expectedToRem, toRemove, "toRemove") + } +} diff --git a/pkg/sql/colexec/colexecwindow/window_framer_tmpl.go b/pkg/sql/colexec/colexecwindow/window_framer_tmpl.go index 997bb78256bc..58c76a33c2b2 100644 --- a/pkg/sql/colexec/colexecwindow/window_framer_tmpl.go +++ b/pkg/sql/colexec/colexecwindow/window_framer_tmpl.go @@ -86,9 +86,18 @@ type windowFramer interface { // frameIntervals returns a series of intervals that describes the set of all // rows that are part of the frame for the current row. Note that there are at // most three intervals - this case can occur when EXCLUDE TIES is used. - // frameIntervals is used to compute aggregate functions over a window. + // frameIntervals is used to compute aggregate functions over a window. The + // returned intervals cannot be modified. frameIntervals() []windowInterval + // slidingWindowIntervals returns a pair of interval sets that describes the + // rows that should be added to the current aggregation, and those which + // should be removed from the current aggregation. It is used to implement the + // sliding window optimization for aggregate window functions. toAdd specifies + // the rows that should be accumulated in the current aggregation, and + // toRemove specifies those which should be removed. + slidingWindowIntervals() (toAdd, toRemove []windowInterval) + // close should always be called upon closing of the parent operator. It // releases all references to enable garbage collection. close() @@ -191,7 +200,13 @@ type windowFramerBase struct { // intervals is a small (at most length 3) slice that is used during // aggregation computation. - intervals []windowInterval + intervals []windowInterval + intervalsAreSet bool + + // prevIntervals, toAdd, and toRemove are used to calculate the intervals + // for calculating aggregate window functions using the sliding window + // optimization. + prevIntervals, toAdd, toRemove []windowInterval } // frameFirstIdx returns the index of the first row in the window frame for @@ -237,7 +252,14 @@ func (b *windowFramerBase) frameNthIdx(n int) (idx int) { // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (b *windowFramerBase) frameIntervals() []windowInterval { + if b.intervalsAreSet { + return b.intervals + } + b.intervalsAreSet = true b.intervals = b.intervals[:0] + if b.startIdx >= b.endIdx { + return b.intervals + } b.intervals = append(b.intervals, windowInterval{start: b.startIdx, end: b.endIdx}) return b.intervals } @@ -281,6 +303,10 @@ func (b *windowFramerBase) startPartition( b.storedCols = storedCols b.startIdx = 0 b.endIdx = 0 + b.intervals = b.intervals[:0] + b.prevIntervals = b.prevIntervals[:0] + b.toAdd = b.toAdd[:0] + b.toRemove = b.toRemove[:0] } // incrementPeerGroup increments the given index by 'groups' peer groups, @@ -493,6 +519,122 @@ func (b *windowFramerBase) excludeTies() bool { return b.exclusion == execinfrapb.WindowerSpec_Frame_EXCLUDE_TIES } +// getSlidingWindowIntervals is a helper function used to calculate the sets of +// rows that are a part of the current window frame, but not the previous one, +// and rows that were a part of the previous window frame, but not the current +// one. getSlidingWindowIntervals expects the intervals stored in currIntervals +// and prevIntervals to be non-overlapping and increasing, and guarantees the +// same invariants for the output intervals. +func getSlidingWindowIntervals( + currIntervals, prevIntervals, toAdd, toRemove []windowInterval, +) ([]windowInterval, []windowInterval) { + toAdd, toRemove = toAdd[:0], toRemove[:0] + var prevIdx, currIdx int + var prev, curr windowInterval + setPrev, setCurr := true, true + for { + // We need to find the set difference currIntervals \ prevIntervals (toAdd) + // and the set difference prevIntervals \ currIntervals (toRemove). To do + // this, take advantage of the fact that both sets of intervals are in + // ascending order, similar to merging sorted lists. Maintain indices into + // each list, and iterate whichever index has the 'smaller' interval + // (e.g. whichever ends first). The portions of the intervals that overlap + // are ignored, while those that don't are added to one of the 'toAdd' and + // 'toRemove' sets. + if prevIdx >= len(prevIntervals) { + // None of the remaining intervals in the current frame were part of the + // previous frame. + if !setCurr { + // The remaining interval stored in curr still hasn't been handled. + toAdd = append(toAdd, curr) + currIdx++ + } + if currIdx < len(currIntervals) { + toAdd = append(toAdd, currIntervals[currIdx:]...) + } + break + } + if currIdx >= len(currIntervals) { + // None of the remaining intervals in the previous frame are part of the + // current frame. + if !setPrev { + // The remaining interval stored in prev still hasn't been handled. + toRemove = append(toRemove, prev) + prevIdx++ + } + if prevIdx < len(prevIntervals) { + toRemove = append(toRemove, prevIntervals[prevIdx:]...) + } + break + } + if setPrev { + prev = prevIntervals[prevIdx] + setPrev = false + } + if setCurr { + curr = currIntervals[currIdx] + setCurr = false + } + if prev == curr { + // This interval has not changed from the previous frame. + prevIdx++ + currIdx++ + setPrev, setCurr = true, true + continue + } + if prev.start >= curr.end { + // The intervals do not overlap, and the curr interval did not exist in + // the previous window frame. + toAdd = append(toAdd, curr) + currIdx++ + setCurr = true + continue + } + if curr.start >= prev.end { + // The intervals do not overlap, and the prev interval existed in the + // previous window frame, but not the current one. + toRemove = append(toRemove, prev) + prevIdx++ + setPrev = true + continue + } + // The intervals overlap but are not equal. + if curr.start < prev.start { + // curr starts before prev. Add the prefix of curr to 'toAdd'. Advance the + // start of curr to the start of prev to reflect that the prefix has + // already been processed. + toAdd = append(toAdd, windowInterval{start: curr.start, end: prev.start}) + curr.start = prev.start + } else if prev.start < curr.start { + // prev starts before curr. Add the prefix of prev to 'toRemove'. Advance + // the start of prev to the start of curr to reflect that the prefix has + // already been processed. + toRemove = append(toRemove, windowInterval{start: prev.start, end: curr.start}) + prev.start = curr.start + } + if curr.end > prev.end { + // prev ends before curr. Set the start of curr to the end of prev to + // indicate that prev has been processed. + curr.start = prev.end + prevIdx++ + setPrev = true + } else if prev.end > curr.end { + // curr ends before prev. Set the start of prev to the end of curr to + // indicate that curr has been processed. + prev.start = curr.end + currIdx++ + setCurr = true + } else { + // prev and curr end at the same index. The prefix of whichever one starts + // first has already been handled. + prevIdx++ + currIdx++ + setPrev, setCurr = true, true + } + } + return toAdd, toRemove +} + // {{range .}} // {{range .StartBoundTypes}} // {{range .EndBoundTypes}} @@ -671,6 +813,8 @@ func (f *_OP_STRING) next(ctx context.Context) { // Handle exclusion clause. f.handleExcludeForNext(ctx, currRowIsGroupStart) // {{end}} + f.prevIntervals = append(f.prevIntervals[:0], f.intervals...) + f.intervalsAreSet = false } func (f *_OP_STRING) close() { @@ -683,6 +827,17 @@ func (f *_OP_STRING) close() { *f = _OP_STRING{} } +// slidingWindowIntervals returns a pair of interval sets that describes the +// rows that should be added to the current aggregation, and those which +// should be removed from the current aggregation. It is used to implement the +// sliding window optimization for aggregate window functions. +func (f *_OP_STRING) slidingWindowIntervals() (toAdd, toRemove []windowInterval) { + f.toAdd, f.toRemove = f.toAdd[:0], f.toRemove[:0] + f.frameIntervals() + f.toAdd, f.toRemove = getSlidingWindowIntervals(f.intervals, f.prevIntervals, f.toAdd, f.toRemove) + return f.toAdd, f.toRemove +} + // {{if .Exclude}} // frameFirstIdx returns the index of the first row in the window frame for @@ -712,6 +867,10 @@ func (f *_OP_STRING) frameNthIdx(n int) (idx int) { // most three intervals - this case can occur when EXCLUDE TIES is used. // frameIntervals is used to compute aggregate functions over a window. func (f *_OP_STRING) frameIntervals() []windowInterval { + if f.startIdx >= f.endIdx { + f.intervals = f.intervals[:0] + return f.intervals + } if f.excludeStartIdx >= f.endIdx || f.excludeEndIdx <= f.startIdx { // No rows excluded. return f.windowFramerBase.frameIntervals() diff --git a/pkg/sql/colexec/colexecwindow/window_functions_test.go b/pkg/sql/colexec/colexecwindow/window_functions_test.go index f56e57c468ac..3330a71b428b 100644 --- a/pkg/sql/colexec/colexecwindow/window_functions_test.go +++ b/pkg/sql/colexec/colexecwindow/window_functions_test.go @@ -1190,7 +1190,15 @@ func BenchmarkWindowFunctions(b *testing.B) { } } else if fun.AggregateFunc != nil { var argIdxs []int - if *fun.AggregateFunc != execinfrapb.CountRows { + switch *fun.AggregateFunc { + case execinfrapb.CountRows: + // CountRows has a specialized implementation. + return NewCountRowsOperator( + args, + NormalizeWindowFrame(nil), + &execinfrapb.Ordering{Columns: orderingCols}, + ) + default: // Supported aggregate functions other than CountRows take one argument. argIdxs = []int{arg1ColIdx} } @@ -1215,8 +1223,9 @@ func BenchmarkWindowFunctions(b *testing.B) { ) require.NoError(b, err) op = NewWindowAggregatorOperator( - args, NormalizeWindowFrame(nil), &execinfrapb.Ordering{Columns: orderingCols}, - []int{arg1ColIdx}, aggArgs.OutputTypes[0], aggFnsAlloc, toClose) + args, *fun.AggregateFunc, NormalizeWindowFrame(nil), + &execinfrapb.Ordering{Columns: orderingCols}, []int{arg1ColIdx}, + aggArgs.OutputTypes[0], aggFnsAlloc, toClose) } else { require.Fail(b, "expected non-nil window function") } @@ -1259,9 +1268,8 @@ func BenchmarkWindowFunctions(b *testing.B) { runBench := func(fun execinfrapb.WindowerSpec_Func, fnName string, numArgs int) { b.Run(fnName, func(b *testing.B) { for _, nRows := range rowsOptions { - if fun.AggregateFunc != nil && nRows == 32*coldata.BatchSize() { - // Aggregate functions are too slow until the sliding window approach - // is implemented. + if !isWindowFnLinear(fun) && nRows == 32*coldata.BatchSize() { + // Skip functions that scale poorly for the larger row size. continue } b.Run(fmt.Sprintf("rows=%d", nRows), func(b *testing.B) { diff --git a/pkg/sql/colexec/colexecwindow/window_functions_util.go b/pkg/sql/colexec/colexecwindow/window_functions_util.go index 91f66624d41c..1489a6482cae 100644 --- a/pkg/sql/colexec/colexecwindow/window_functions_util.go +++ b/pkg/sql/colexec/colexecwindow/window_functions_util.go @@ -276,3 +276,53 @@ func GetOffsetTypeFromOrderColType(t *testing.T, orderColType *types.T) *types.T } return orderColType } + +// isWindowFnLinear returns whether the vectorized engine has an implementation +// of the given window function that scales linearly when the default window +// frame is used. +func isWindowFnLinear(fn execinfrapb.WindowerSpec_Func) bool { + if fn.WindowFunc != nil { + return true + } + switch *fn.AggregateFunc { + case + execinfrapb.Count, + execinfrapb.CountRows, + execinfrapb.Sum, + execinfrapb.SumInt, + execinfrapb.Avg, + execinfrapb.Min, + execinfrapb.Max: + return true + default: + return false + } +} + +// WindowFrameCanShrink returns true if a sliding window aggregate function over +// the given frame may need to call Remove, which is the case when the frame for +// a given row may not include all rows that were part of the previous frame. +func WindowFrameCanShrink( + frame *execinfrapb.WindowerSpec_Frame, ordering *execinfrapb.Ordering, +) bool { + if frame.Exclusion != execinfrapb.WindowerSpec_Frame_NO_EXCLUSION { + return true + } + if frame.Bounds.Start.BoundType == execinfrapb.WindowerSpec_Frame_UNBOUNDED_PRECEDING { + return false + } + if len(ordering.Columns) == 0 { + // All rows are part of the same peer group. + if frame.Bounds.Start.BoundType == execinfrapb.WindowerSpec_Frame_CURRENT_ROW && + (frame.Mode == execinfrapb.WindowerSpec_Frame_RANGE || + frame.Mode == execinfrapb.WindowerSpec_Frame_GROUPS) { + return false + } + if frame.Mode == execinfrapb.WindowerSpec_Frame_GROUPS && + frame.Bounds.Start.BoundType == execinfrapb.WindowerSpec_Frame_OFFSET_PRECEDING && + frame.Bounds.Start.IntOffset >= 1 { + return false + } + } + return true +} diff --git a/pkg/sql/colexec/execgen/cmd/execgen/BUILD.bazel b/pkg/sql/colexec/execgen/cmd/execgen/BUILD.bazel index 3ce307c5f2e5..ba7f194d68ca 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/BUILD.bazel +++ b/pkg/sql/colexec/execgen/cmd/execgen/BUILD.bazel @@ -33,6 +33,7 @@ go_library( "mergejoinbase_gen.go", "mergejoiner_gen.go", "min_max_agg_gen.go", + "min_max_removable_agg_gen.go", "ntile_gen.go", "ordered_synchronizer_gen.go", "overloads_abbr.go", @@ -58,6 +59,7 @@ go_library( "vec_comparators_gen.go", "vec_gen.go", "vec_to_datum_gen.go", + "window_aggregator_gen.go", "window_framer_gen.go", "window_peer_grouper_gen.go", ], diff --git a/pkg/sql/colexec/execgen/cmd/execgen/agg_gen_util.go b/pkg/sql/colexec/execgen/cmd/execgen/agg_gen_util.go index 87de16e1b44c..5eaaa3b29afc 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/agg_gen_util.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/agg_gen_util.go @@ -57,6 +57,11 @@ type aggTmplInfoBase struct { canonicalTypeFamily types.Family } +// CopyVal is a function that should only be used in templates. +func (b *aggTmplInfoBase) CopyVal(dest, src string) string { + return copyVal(b.canonicalTypeFamily, dest, src) +} + // SetVariableSize is a function that should only be used in templates. See the // comment on setVariableSize for more details. func (b aggTmplInfoBase) SetVariableSize(target, value string) string { @@ -67,4 +72,5 @@ func (b aggTmplInfoBase) SetVariableSize(target, value string) string { var ( a aggTmplInfoBase _ = a.SetVariableSize + _ = a.CopyVal ) diff --git a/pkg/sql/colexec/execgen/cmd/execgen/avg_agg_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/avg_agg_gen.go index 2ef6d9ca7d7c..c700438d175b 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/avg_agg_gen.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/avg_agg_gen.go @@ -30,7 +30,7 @@ type avgTmplInfo struct { RetGoType string RetVecMethod string - addOverload assignFunc + avgOverload assignFunc } func (a avgTmplInfo) AssignAdd(targetElem, leftElem, rightElem, _, _, _ string) string { @@ -41,7 +41,19 @@ func (a avgTmplInfo) AssignAdd(targetElem, leftElem, rightElem, _, _, _ string) lawo := &lastArgWidthOverload{lastArgTypeOverload: &lastArgTypeOverload{ overloadBase: newBinaryOverloadBase(tree.Plus), }} - return a.addOverload(lawo, targetElem, leftElem, rightElem, "", "", "") + return a.avgOverload(lawo, targetElem, leftElem, rightElem, "", "", "") +} + +func (a avgTmplInfo) AssignSubtract( + targetElem, leftElem, rightElem, targetCol, leftCol, rightCol string, +) string { + // Note that we need to create lastArgWidthOverload only in order to tell + // the resolved overload to use Minus overload in particular, so all other + // fields remain unset. + lawo := &lastArgWidthOverload{lastArgTypeOverload: &lastArgTypeOverload{ + overloadBase: newBinaryOverloadBase(tree.Minus), + }} + return a.avgOverload(lawo, targetElem, leftElem, rightElem, targetCol, leftCol, rightCol) } func (a avgTmplInfo) AssignDivInt64(targetElem, leftElem, rightElem, _, _, _ string) string { @@ -69,6 +81,7 @@ func (a avgTmplInfo) AssignDivInt64(targetElem, leftElem, rightElem, _, _, _ str var ( _ = avgTmplInfo{}.AssignAdd _ = avgTmplInfo{}.AssignDivInt64 + _ = avgTmplInfo{}.AssignSubtract ) // avgAggTypeTmplInfo is similar to lastArgTypeOverload and provides a way to @@ -107,9 +120,13 @@ func genAvgAgg(inputFileContents string, wr io.Writer) error { s = assignDivRe.ReplaceAllString(s, makeTemplateFunctionCall("AssignDivInt64", 6)) assignAddRe := makeFunctionRegex("_ASSIGN_ADD", 6) s = assignAddRe.ReplaceAllString(s, makeTemplateFunctionCall("Global.AssignAdd", 6)) + assignSubtractRe := makeFunctionRegex("_ASSIGN_SUBTRACT", 6) + s = assignSubtractRe.ReplaceAllString(s, makeTemplateFunctionCall("Global.AssignSubtract", 6)) accumulateAvg := makeFunctionRegex("_ACCUMULATE_AVG", 5) s = accumulateAvg.ReplaceAllString(s, `{{template "accumulateAvg" buildDict "Global" . "HasNulls" $4 "HasSel" $5}}`) + removeRow := makeFunctionRegex("_REMOVE_ROW", 4) + s = removeRow.ReplaceAllString(s, `{{template "removeRow" buildDict "Global" . "HasNulls" $4}}`) s = replaceManipulationFuncs(s) @@ -149,7 +166,7 @@ func genAvgAgg(inputFileContents string, wr io.Writer) error { InputVecMethod: toVecMethod(inputTypeFamily, inputTypeWidth), RetGoType: toPhysicalRepresentation(retTypeFamily, retTypeWidth), RetVecMethod: toVecMethod(retTypeFamily, retTypeWidth), - addOverload: getSumAddOverload(inputTypeFamily), + avgOverload: getSumAddOverload(inputTypeFamily), }}) } tmplInfos = append(tmplInfos, tmplInfo) diff --git a/pkg/sql/colexec/execgen/cmd/execgen/count_agg_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/count_agg_gen.go index aeb482648b2a..2ca137e75ad2 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/count_agg_gen.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/count_agg_gen.go @@ -26,6 +26,9 @@ func genCountAgg(inputFileContents string, wr io.Writer) error { accumulateSum := makeFunctionRegex("_ACCUMULATE_COUNT", 5) s = accumulateSum.ReplaceAllString(s, `{{template "accumulateCount" buildDict "Global" . "ColWithNulls" $4 "HasSel" $5}}`) + removeRow := makeFunctionRegex("_REMOVE_ROW", 4) + s = removeRow.ReplaceAllString(s, `{{template "removeRow" buildDict "Global" . "ColWithNulls" $4}}`) + s = replaceManipulationFuncs(s) tmpl, err := template.New("count_agg").Funcs(template.FuncMap{"buildDict": buildDict}).Parse(s) diff --git a/pkg/sql/colexec/execgen/cmd/execgen/min_max_removable_agg_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/min_max_removable_agg_gen.go new file mode 100644 index 000000000000..20c554268021 --- /dev/null +++ b/pkg/sql/colexec/execgen/cmd/execgen/min_max_removable_agg_gen.go @@ -0,0 +1,64 @@ +// Copyright 2021 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 main + +import ( + "io" + "strings" + "text/template" + + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" +) + +const minMaxRemovableAggTmpl = "pkg/sql/colexec/colexecwindow/min_max_removable_agg_tmpl.go" + +func genMinMaxRemovableAgg(inputFileContents string, wr io.Writer) error { + r := strings.NewReplacer( + "_CANONICAL_TYPE_FAMILY", "{{.CanonicalTypeFamilyStr}}", + "_TYPE_WIDTH", typeWidthReplacement, + "_AGG_TITLE", "{{.AggTitle}}", + "_AGG", "{{$agg}}", + "_GOTYPE", "{{.GoType}}", + "_TYPE", "{{.VecMethod}}", + "TemplateType", "{{.VecMethod}}", + ) + s := r.Replace(inputFileContents) + + assignCmpRe := makeFunctionRegex("_ASSIGN_CMP", 6) + s = assignCmpRe.ReplaceAllString(s, makeTemplateFunctionCall("Assign", 6)) + + s = replaceManipulationFuncs(s) + + tmpl, err := template.New("min_max_agg").Funcs(template.FuncMap{"buildDict": buildDict}).Parse(s) + if err != nil { + return err + } + return tmpl.Execute(wr, []struct { + Agg string + AggTitle string + Overloads []*oneArgOverload + }{ + { + Agg: "min", + AggTitle: "Min", + Overloads: sameTypeComparisonOpToOverloads[tree.LT], + }, + { + Agg: "max", + AggTitle: "Max", + Overloads: sameTypeComparisonOpToOverloads[tree.GT], + }, + }) +} + +func init() { + registerGenerator(genMinMaxRemovableAgg, "min_max_removable_agg.eg.go", minMaxRemovableAggTmpl) +} diff --git a/pkg/sql/colexec/execgen/cmd/execgen/sum_agg_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/sum_agg_gen.go index 1a499ae8bddc..a4264976574d 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/sum_agg_gen.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/sum_agg_gen.go @@ -30,7 +30,7 @@ type sumAggTmplInfo struct { RetGoType string RetVecMethod string - addOverload assignFunc + sumOverload assignFunc } func (s sumAggTmplInfo) AssignAdd( @@ -42,11 +42,25 @@ func (s sumAggTmplInfo) AssignAdd( lawo := &lastArgWidthOverload{lastArgTypeOverload: &lastArgTypeOverload{ overloadBase: newBinaryOverloadBase(tree.Plus), }} - return s.addOverload(lawo, targetElem, leftElem, rightElem, targetCol, leftCol, rightCol) + return s.sumOverload(lawo, targetElem, leftElem, rightElem, targetCol, leftCol, rightCol) } var _ = sumAggTmplInfo{}.AssignAdd +func (s sumAggTmplInfo) AssignSubtract( + targetElem, leftElem, rightElem, targetCol, leftCol, rightCol string, +) string { + // Note that we need to create lastArgWidthOverload only in order to tell + // the resolved overload to use Minus overload in particular, so all other + // fields remain unset. + lawo := &lastArgWidthOverload{lastArgTypeOverload: &lastArgTypeOverload{ + overloadBase: newBinaryOverloadBase(tree.Minus), + }} + return s.sumOverload(lawo, targetElem, leftElem, rightElem, targetCol, leftCol, rightCol) +} + +var _ = sumAggTmplInfo{}.AssignSubtract + // avgAggTypeTmplInfo is similar to lastArgTypeOverload and provides a way to // see the type family of the overload. This is the top level of data passed to // the template. @@ -111,9 +125,15 @@ func genSumAgg(inputFileContents string, wr io.Writer, isSumInt bool) error { assignAddRe := makeFunctionRegex("_ASSIGN_ADD", 6) s = assignAddRe.ReplaceAllString(s, makeTemplateFunctionCall("Global.AssignAdd", 6)) + assignSubtractRe := makeFunctionRegex("_ASSIGN_SUBTRACT", 6) + s = assignSubtractRe.ReplaceAllString(s, makeTemplateFunctionCall("Global.AssignSubtract", 6)) + accumulateSum := makeFunctionRegex("_ACCUMULATE_SUM", 5) s = accumulateSum.ReplaceAllString(s, `{{template "accumulateSum" buildDict "Global" . "HasNulls" $4 "HasSel" $5}}`) + removeRow := makeFunctionRegex("_REMOVE_ROW", 4) + s = removeRow.ReplaceAllString(s, `{{template "removeRow" buildDict "Global" . "HasNulls" $4}}`) + s = replaceManipulationFuncs(s) tmpl, err := template.New("sum_agg").Funcs(template.FuncMap{"buildDict": buildDict}).Parse(s) @@ -169,7 +189,7 @@ func genSumAgg(inputFileContents string, wr io.Writer, isSumInt bool) error { InputVecMethod: toVecMethod(inputTypeFamily, inputTypeWidth), RetGoType: toPhysicalRepresentation(retTypeFamily, retTypeWidth), RetVecMethod: toVecMethod(retTypeFamily, retTypeWidth), - addOverload: getAddOverload(inputTypeFamily), + sumOverload: getAddOverload(inputTypeFamily), }}) } tmplInfos = append(tmplInfos, tmplInfo) diff --git a/pkg/sql/colexec/execgen/cmd/execgen/window_aggregator_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/window_aggregator_gen.go new file mode 100644 index 000000000000..1e572478c3ca --- /dev/null +++ b/pkg/sql/colexec/execgen/cmd/execgen/window_aggregator_gen.go @@ -0,0 +1,24 @@ +// Copyright 2021 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 main + +import "io" + +const windowAggregatorTmpl = "pkg/sql/colexec/colexecwindow/window_aggregator_tmpl.go" + +func genWindowAggregator(inputFileContents string, outputFile io.Writer) error { + _, err := outputFile.Write([]byte(inputFileContents)) + return err +} + +func init() { + registerGenerator(genWindowAggregator, "window_aggregator.eg.go", windowAggregatorTmpl) +} diff --git a/pkg/sql/distsql/columnar_operators_test.go b/pkg/sql/distsql/columnar_operators_test.go index 2b540e10ebc6..8dec40787f7d 100644 --- a/pkg/sql/distsql/columnar_operators_test.go +++ b/pkg/sql/distsql/columnar_operators_test.go @@ -1179,6 +1179,14 @@ func TestWindowFunctionsAgainstProcessor(t *testing.T) { // on. continue } + if strings.Contains(err.Error(), "integer out of range") && + fun.AggregateFunc != nil && *fun.AggregateFunc == execinfrapb.SumInt { + // The columnar implementation of this window function uses the + // sliding window optimization, but the row engine version + // doesn't. As a result, in some cases the row engine will + // overflow while the vectorized engine doesn't. + continue + } fmt.Printf("window function: %s\n", funcName) fmt.Printf("partitionCols: %v\n", partitionBy) fmt.Print("ordering: ") diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index fd36e0c2f967..bc0e91cda231 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -960,6 +960,7 @@ type ExecutorConfig struct { FeatureFlagMetrics *featureflag.DenialMetrics TestingKnobs ExecutorTestingKnobs + MigrationTestingKnobs *migration.TestingKnobs PGWireTestingKnobs *PGWireTestingKnobs SchemaChangerTestingKnobs *SchemaChangerTestingKnobs NewSchemaChangerTestingKnobs *scexec.NewSchemaChangerTestingKnobs diff --git a/pkg/sql/gcjob_test/gc_job_test.go b/pkg/sql/gcjob_test/gc_job_test.go index 978099246aa3..52112fe85654 100644 --- a/pkg/sql/gcjob_test/gc_job_test.go +++ b/pkg/sql/gcjob_test/gc_job_test.go @@ -427,6 +427,7 @@ func TestGCJobRetry(t *testing.T) { var failed atomic.Value failed.Store(false) params := base.TestServerArgs{} + params.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals() params.Knobs.Store = &kvserver.StoreTestingKnobs{ TestingRequestFilter: func(ctx context.Context, request roachpb.BatchRequest) *roachpb.Error { _, ok := request.GetArg(roachpb.ClearRange) diff --git a/pkg/sql/lex/BUILD.bazel b/pkg/sql/lex/BUILD.bazel index c416990e4b1f..c5523afe3b6a 100644 --- a/pkg/sql/lex/BUILD.bazel +++ b/pkg/sql/lex/BUILD.bazel @@ -16,7 +16,6 @@ go_library( go_test( name = "lex_test", - size = "small", srcs = ["encode_test.go"], deps = [ ":lex", diff --git a/pkg/sql/logictest/BUILD.bazel b/pkg/sql/logictest/BUILD.bazel index 46cd05e226c2..a383fae805f8 100644 --- a/pkg/sql/logictest/BUILD.bazel +++ b/pkg/sql/logictest/BUILD.bazel @@ -16,7 +16,7 @@ go_library( "//pkg/build", "//pkg/clusterversion", "//pkg/kv/kvserver", - "//pkg/migration/migrationmanager", + "//pkg/migration", "//pkg/roachpb:with-mocks", "//pkg/security", "//pkg/server", diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index 2e4506c26fcc..70df64dc22a7 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -42,7 +42,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" - "github.com/cockroachdb/cockroach/pkg/migration/migrationmanager" + "github.com/cockroachdb/cockroach/pkg/migration" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server" @@ -1414,9 +1414,9 @@ func (t *logicTest) newCluster(serverArgs TestServerArgs) { from := clusterversion.ClusterVersion{Version: cfg.bootstrapVersion} to := clusterversion.ClusterVersion{Version: cfg.binaryVersion} if len(clusterversion.ListBetween(from, to)) == 0 { - mm, ok := nodeParams.Knobs.MigrationManager.(*migrationmanager.TestingKnobs) + mm, ok := nodeParams.Knobs.MigrationManager.(*migration.TestingKnobs) if !ok { - mm = &migrationmanager.TestingKnobs{} + mm = &migration.TestingKnobs{} nodeParams.Knobs.MigrationManager = mm } mm.ListBetweenOverride = func( diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index eea802b8339c..fbe8eab51cc3 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -1946,6 +1946,8 @@ system public jobs created system public jobs created_by_id 7 system public jobs created_by_type 6 system public jobs id 1 +system public jobs last_run 11 +system public jobs num_runs 10 system public jobs payload 4 system public jobs progress 5 system public jobs status 2 diff --git a/pkg/sql/logictest/testdata/logic_test/mem_limit b/pkg/sql/logictest/testdata/logic_test/mem_limit index fb627f3ad9a3..5562c0652422 100644 --- a/pkg/sql/logictest/testdata/logic_test/mem_limit +++ b/pkg/sql/logictest/testdata/logic_test/mem_limit @@ -44,4 +44,3 @@ vectorized: true query error scan with start key .* memory budget exceeded SELECT * FROM foo@foo_attribute_idx WHERE attribute=10 AND blob LIKE 'blah%' - diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 78e89c5f2e8e..4f106f71ac0c 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -1007,54 +1007,55 @@ SELECT * FROM pg_catalog.pg_index ORDER BY indexrelid ---- -indexrelid indrelid indnatts indisunique indisprimary indisexclusion indimmediate indisclustered indisvalid indcheckxmin indisready indislive indisreplident indkey indcollation indclass indoption indexprs indpred indnkeyatts -144368028 32 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 -404104299 39 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 -543291288 23 1 false false false false false true false false true false 1 3403232968 0 2 NULL NULL 1 -543291289 23 1 false false false false false true false false true false 2 3403232968 0 2 NULL NULL 1 -543291291 23 2 true true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL 2 -663840565 42 5 false false false false false true false false true false 2 1 3 4 5 0 0 0 3403232968 0 0 0 0 0 0 2 2 2 2 2 NULL NULL 5 -663840566 42 5 true true false true false true false false true false 1 2 3 4 5 0 0 0 3403232968 0 0 0 0 0 0 2 2 2 2 2 NULL NULL 5 -803027558 26 3 true true false true false true false false true false 1 2 3 0 0 3403232968 0 0 0 2 2 2 NULL NULL 3 -923576837 41 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 -1062763829 25 4 true true false true false true false false true false 1 2 3 4 0 0 3403232968 3403232968 0 0 0 0 2 2 2 2 NULL NULL 4 -1183313104 44 2 true true false true false true false false true false 1 2 0 3403232968 0 0 2 2 NULL NULL 2 -1276104432 12 2 true true false true false true false false true false 1 6 0 0 0 0 2 2 NULL NULL 2 -1322500096 28 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 -1489445036 35 3 false false false false false true false false true false 2 1 3 0 0 0 0 2 2 NULL NULL 2 -1489445039 35 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 -1582236367 3 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 -1628632026 19 1 false false false false false true false false true false 6 0 0 2 NULL NULL 1 -1628632027 19 1 false false false false false true false false true false 7 0 0 2 NULL NULL 1 -1628632028 19 1 false false false false false true false false true false 5 0 0 2 NULL NULL 1 -1628632029 19 1 false false false false false true false false true false 4 0 0 2 NULL NULL 1 -1628632031 19 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 -1841972634 6 1 true true false true false true false false true false 1 3403232968 0 2 NULL NULL 1 -2008917577 37 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 -2008917578 37 1 false false false false false true false false true false 5 0 0 2 NULL NULL 1 -2101708905 5 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 -2148104569 21 2 true true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL 2 -2268653844 40 4 true true false true false true false false true false 1 2 3 4 0 0 0 0 0 0 0 0 2 2 2 2 NULL NULL 4 -2361445172 8 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 -2407840836 24 3 true true false true false true false false true false 1 2 3 0 0 0 0 0 0 2 2 2 NULL NULL 3 -2621181440 15 2 false false false false false true false false true false 2 3 3403232968 0 0 0 2 2 NULL NULL 2 -2621181441 15 3 false false false false false true false false true false 6 7 2 3403232968 0 0 0 2 2 NULL NULL 2 -2621181443 15 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 -2667577107 31 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 -2834522046 34 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 -3094258317 33 2 true true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL 2 -3353994584 36 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 -3446785912 4 1 true true false true false true false false true false 1 3403232968 0 2 NULL NULL 1 -3493181576 20 2 true true false true false true false false true false 1 2 0 0 0 0 2 2 NULL NULL 2 -3613730852 43 4 false false false false false true false false true false 2 1 3 4 0 0 3403232968 0 0 0 0 0 2 2 2 2 NULL NULL 4 -3613730855 43 4 true true false true false true false false true false 1 2 3 4 0 0 3403232968 0 0 0 0 0 2 2 2 2 NULL NULL 4 -3706522183 11 4 true true false true false true false false true false 1 2 4 3 0 0 0 0 0 0 0 0 2 2 2 2 NULL NULL 4 -3752917847 27 2 true true false true false true false false true false 1 2 0 0 0 0 2 2 NULL NULL 2 -3873467122 46 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 -3966258450 14 1 true true false true false true false false true false 1 3403232968 0 2 NULL NULL 1 -4012654114 30 3 true true false true false true false false true false 1 2 3 0 0 3403232968 0 0 0 2 2 2 NULL NULL 3 -4133203393 45 2 true true false true false true false false true false 1 2 0 0 0 0 2 2 NULL NULL 2 -4225994721 13 2 true true false true false true false false true false 1 7 0 0 0 0 2 2 NULL NULL 2 +indexrelid indrelid indnatts indisunique indisprimary indisexclusion indimmediate indisclustered indisvalid indcheckxmin indisready indislive indisreplident indkey indcollation indclass indoption indexprs indpred indnkeyatts +144368028 32 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 +404104299 39 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 +543291288 23 1 false false false false false true false false true false 1 3403232968 0 2 NULL NULL 1 +543291289 23 1 false false false false false true false false true false 2 3403232968 0 2 NULL NULL 1 +543291291 23 2 true true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL 2 +663840565 42 5 false false false false false true false false true false 2 1 3 4 5 0 0 0 3403232968 0 0 0 0 0 0 2 2 2 2 2 NULL NULL 5 +663840566 42 5 true true false true false true false false true false 1 2 3 4 5 0 0 0 3403232968 0 0 0 0 0 0 2 2 2 2 2 NULL NULL 5 +803027558 26 3 true true false true false true false false true false 1 2 3 0 0 3403232968 0 0 0 2 2 2 NULL NULL 3 +923576837 41 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 +1062763829 25 4 true true false true false true false false true false 1 2 3 4 0 0 3403232968 3403232968 0 0 0 0 2 2 2 2 NULL NULL 4 +1183313104 44 2 true true false true false true false false true false 1 2 0 3403232968 0 0 2 2 NULL NULL 2 +1276104432 12 2 true true false true false true false false true false 1 6 0 0 0 0 2 2 NULL NULL 2 +1322500096 28 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 +1489445036 35 3 false false false false false true false false true false 2 1 3 0 0 0 0 2 2 NULL NULL 2 +1489445039 35 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 +1582236367 3 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 +1628632026 19 1 false false false false false true false false true false 6 0 0 2 NULL NULL 1 +1628632027 19 1 false false false false false true false false true false 7 0 0 2 NULL NULL 1 +1628632028 19 1 false false false false false true false false true false 5 0 0 2 NULL NULL 1 +1628632029 19 1 false false false false false true false false true false 4 0 0 2 NULL NULL 1 +1628632031 19 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 +1841972634 6 1 true true false true false true false false true false 1 3403232968 0 2 NULL NULL 1 +2008917577 37 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 +2008917578 37 1 false false false false false true false false true false 5 0 0 2 NULL NULL 1 +2101708905 5 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 +2148104569 21 2 true true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL 2 +2268653844 40 4 true true false true false true false false true false 1 2 3 4 0 0 0 0 0 0 0 0 2 2 2 2 NULL NULL 4 +2361445172 8 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 +2407840836 24 3 true true false true false true false false true false 1 2 3 0 0 0 0 0 0 2 2 2 NULL NULL 3 +2621181440 15 2 false false false false false true false false true false 2 3 3403232968 0 0 0 2 2 NULL NULL 2 +2621181441 15 3 false false false false false true false false true false 6 7 2 3403232968 0 0 0 2 2 NULL NULL 2 +2621181443 15 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 +2621181446 15 6 false false false false false true false false true false 8 2 3 11 10 9 0 3403232968 0 0 0 0 2 2 2 NULL NULL 3 +2667577107 31 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 +2834522046 34 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 +3094258317 33 2 true true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL 2 +3353994584 36 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 +3446785912 4 1 true true false true false true false false true false 1 3403232968 0 2 NULL NULL 1 +3493181576 20 2 true true false true false true false false true false 1 2 0 0 0 0 2 2 NULL NULL 2 +3613730852 43 4 false false false false false true false false true false 2 1 3 4 0 0 3403232968 0 0 0 0 0 2 2 2 2 NULL NULL 4 +3613730855 43 4 true true false true false true false false true false 1 2 3 4 0 0 3403232968 0 0 0 0 0 2 2 2 2 NULL NULL 4 +3706522183 11 4 true true false true false true false false true false 1 2 4 3 0 0 0 0 0 0 0 0 2 2 2 2 NULL NULL 4 +3752917847 27 2 true true false true false true false false true false 1 2 0 0 0 0 2 2 NULL NULL 2 +3873467122 46 1 true true false true false true false false true false 1 0 0 2 NULL NULL 1 +3966258450 14 1 true true false true false true false false true false 1 3403232968 0 2 NULL NULL 1 +4012654114 30 3 true true false true false true false false true false 1 2 3 0 0 3403232968 0 0 0 2 2 2 NULL NULL 3 +4133203393 45 2 true true false true false true false false true false 1 2 0 0 0 0 2 2 NULL NULL 2 +4225994721 13 2 true true false true false true false false true false 1 7 0 0 0 0 2 2 NULL NULL 2 # From #26504 query OOI colnames @@ -1122,6 +1123,9 @@ indexrelid operator_argument_type_oid operator_argument_position 2621181441 0 1 2621181441 0 2 2621181443 0 1 +2621181446 0 1 +2621181446 0 2 +2621181446 0 3 2667577107 0 1 2834522046 0 1 3094258317 0 1 diff --git a/pkg/sql/logictest/testdata/logic_test/system b/pkg/sql/logictest/testdata/logic_test/system index 4ec4cb74727f..27a300066560 100644 --- a/pkg/sql/logictest/testdata/logic_test/system +++ b/pkg/sql/logictest/testdata/logic_test/system @@ -165,15 +165,17 @@ lastUpdated TIMESTAMP false NULL · {primary} false query TTBTTTB SHOW COLUMNS FROM system.jobs ---- -id INT8 false unique_rowid() · {jobs_created_by_type_created_by_id_idx,jobs_status_created_idx,primary} false -status STRING false NULL · {jobs_created_by_type_created_by_id_idx,jobs_status_created_idx,primary} false -created TIMESTAMP false now():::TIMESTAMP · {jobs_status_created_idx,primary} false -payload BYTES false NULL · {primary} false -progress BYTES true NULL · {primary} false -created_by_type STRING true NULL · {jobs_created_by_type_created_by_id_idx,primary} false -created_by_id INT8 true NULL · {jobs_created_by_type_created_by_id_idx,primary} false -claim_session_id BYTES true NULL · {primary} false -claim_instance_id INT8 true NULL · {primary} false +id INT8 false unique_rowid() · {jobs_created_by_type_created_by_id_idx,jobs_run_stats_idx,jobs_status_created_idx,primary} false +status STRING false NULL · {jobs_created_by_type_created_by_id_idx,jobs_run_stats_idx,jobs_status_created_idx,primary} false +created TIMESTAMP false now():::TIMESTAMP · {jobs_run_stats_idx,jobs_status_created_idx,primary} false +payload BYTES false NULL · {primary} false +progress BYTES true NULL · {primary} false +created_by_type STRING true NULL · {jobs_created_by_type_created_by_id_idx,primary} false +created_by_id INT8 true NULL · {jobs_created_by_type_created_by_id_idx,primary} false +claim_session_id BYTES true NULL · {jobs_run_stats_idx,primary} false +claim_instance_id INT8 true NULL · {jobs_run_stats_idx,primary} false +num_runs INT8 true NULL · {jobs_run_stats_idx,primary} false +last_run TIMESTAMP true NULL · {jobs_run_stats_idx,primary} false query TTBTTTB SHOW COLUMNS FROM system.settings diff --git a/pkg/sql/memsize/constants.go b/pkg/sql/memsize/constants.go index f6e02d391cca..109bd43ea9ed 100644 --- a/pkg/sql/memsize/constants.go +++ b/pkg/sql/memsize/constants.go @@ -34,6 +34,9 @@ const ( // Int32 is the in-memory size of an int32 in bytes. Int32 = int64(unsafe.Sizeof(int32(0))) + // Uint32 is the in-memory size of a uint32 in bytes. + Uint32 = int64(unsafe.Sizeof(uint32(0))) + // Int64 is the in-memory size of an int64 in bytes. Int64 = int64(unsafe.Sizeof(int64(0))) diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index ba9acec15a96..f4e5d7760dcf 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -2472,7 +2472,6 @@ var charts = []sectionDescription{ { Title: "Round", Metrics: []string{ - "schedules.round.schedules-ready-to-run", "schedules.round.reschedule-skip", "schedules.round.reschedule-wait", @@ -2671,6 +2670,20 @@ var charts = []sectionDescription{ }, }, }, + { + Organization: [][]string{{Jobs, "Registry"}}, + Charts: []chartDescription{ + { + Title: "Jobs Registry Stats", + Metrics: []string{ + "jobs.adopt_iterations", + "jobs.claimed_jobs", + "jobs.resumed_claimed_jobs", + }, + AxisLabel: "Count", + }, + }, + }, { Organization: [][]string{{Process, "Node", "Admission"}}, Charts: []chartDescription{