From 6191b1dc71dc6b565f059541da1b3e8cd42604c9 Mon Sep 17 00:00:00 2001 From: reus Date: Thu, 28 Aug 2025 11:24:56 +0800 Subject: [PATCH 01/57] colexec/group: add SpillableData, SpillManager, MemorySpillManager, SpillableAggState --- pkg/sql/colexec/group/spill.go | 33 ++++++++ pkg/sql/colexec/group/spill_memory.go | 72 ++++++++++++++++ pkg/sql/colexec/group/spillable_agg_state.go | 87 ++++++++++++++++++++ 3 files changed, 192 insertions(+) create mode 100644 pkg/sql/colexec/group/spill.go create mode 100644 pkg/sql/colexec/group/spill_memory.go create mode 100644 pkg/sql/colexec/group/spillable_agg_state.go diff --git a/pkg/sql/colexec/group/spill.go b/pkg/sql/colexec/group/spill.go new file mode 100644 index 0000000000000..a64f87cdc6b79 --- /dev/null +++ b/pkg/sql/colexec/group/spill.go @@ -0,0 +1,33 @@ +// Copyright 2025 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package group + +import "github.com/matrixorigin/matrixone/pkg/common/mpool" + +type SpillID string + +type SpillableData interface { + Serialize() ([]byte, error) + Deserialize(data []byte) error + EstimateSize() int64 + Free(mp *mpool.MPool) +} + +type SpillManager interface { + Spill(data SpillableData) (SpillID, error) + Retrieve(id SpillID) (SpillableData, error) + Delete(id SpillID) error + Free() +} diff --git a/pkg/sql/colexec/group/spill_memory.go b/pkg/sql/colexec/group/spill_memory.go new file mode 100644 index 0000000000000..6d018b2f57088 --- /dev/null +++ b/pkg/sql/colexec/group/spill_memory.go @@ -0,0 +1,72 @@ +// Copyright 2025 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package group + +import ( + "fmt" + "sync/atomic" +) + +type MemorySpillManager struct { + data map[SpillID][]byte + nextID int64 + totalMem int64 +} + +func NewMemorySpillManager() *MemorySpillManager { + return &MemorySpillManager{ + data: make(map[SpillID][]byte), + } +} + +func (m *MemorySpillManager) Spill(data SpillableData) (SpillID, error) { + serialized, err := data.Serialize() + if err != nil { + return "", err + } + + id := SpillID(fmt.Sprintf("spill_%d", atomic.AddInt64(&m.nextID, 1))) + m.data[id] = serialized + atomic.AddInt64(&m.totalMem, int64(len(serialized))) + return id, nil +} + +func (m *MemorySpillManager) Retrieve(id SpillID) (SpillableData, error) { + serialized, exists := m.data[id] + if !exists { + return nil, fmt.Errorf("spill data not found: %s", id) + } + + data := &SpillableAggState{} + if err := data.Deserialize(serialized); err != nil { + return nil, err + } + return data, nil +} + +func (m *MemorySpillManager) Delete(id SpillID) error { + if serialized, exists := m.data[id]; exists { + atomic.AddInt64(&m.totalMem, -int64(len(serialized))) + delete(m.data, id) + } + return nil +} + +func (m *MemorySpillManager) Free() { + for id := range m.data { + m.Delete(id) + } + m.data = nil +} diff --git a/pkg/sql/colexec/group/spillable_agg_state.go b/pkg/sql/colexec/group/spillable_agg_state.go new file mode 100644 index 0000000000000..ed5716f178604 --- /dev/null +++ b/pkg/sql/colexec/group/spillable_agg_state.go @@ -0,0 +1,87 @@ +// Copyright 2025 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package group + +import ( + "encoding/json" + + "github.com/matrixorigin/matrixone/pkg/common/mpool" + "github.com/matrixorigin/matrixone/pkg/container/vector" +) + +type SpillableAggState struct { + GroupVectors []*vector.Vector + PartialStates []any + GroupCount int +} + +func (s *SpillableAggState) Serialize() ([]byte, error) { + data := map[string]interface{}{ + "group_count": s.GroupCount, + "partial_states": s.PartialStates, + "group_vectors": make([]map[string]interface{}, len(s.GroupVectors)), + } + + for i, vec := range s.GroupVectors { + if vec != nil { + vecData := map[string]interface{}{ + "type": vec.GetType().String(), + "length": vec.Length(), + "is_const": vec.IsConst(), + } + data["group_vectors"].([]map[string]interface{})[i] = vecData + } + } + + return json.Marshal(data) +} + +func (s *SpillableAggState) Deserialize(data []byte) error { + var parsed map[string]interface{} + if err := json.Unmarshal(data, &parsed); err != nil { + return err + } + + if count, ok := parsed["group_count"].(float64); ok { + s.GroupCount = int(count) + } + + if states, ok := parsed["partial_states"].([]interface{}); ok { + s.PartialStates = states + } + + return nil +} + +func (s *SpillableAggState) EstimateSize() int64 { + size := int64(0) + for _, vec := range s.GroupVectors { + if vec != nil { + size += int64(vec.Size()) + } + } + size += int64(len(s.PartialStates) * 64) + return size +} + +func (s *SpillableAggState) Free(mp *mpool.MPool) { + for _, vec := range s.GroupVectors { + if vec != nil { + vec.Free(mp) + } + } + s.GroupVectors = nil + s.PartialStates = nil +} From 00b844c3a2e8c6ce9a96058b7120158c39cd520d Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 29 Aug 2025 14:11:48 +0800 Subject: [PATCH 02/57] colexec/group: add Group.SpillManager, SpillThreshold --- pkg/sql/colexec/group/types.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pkg/sql/colexec/group/types.go b/pkg/sql/colexec/group/types.go index 0243e4cda69d2..ee62248385ef8 100644 --- a/pkg/sql/colexec/group/types.go +++ b/pkg/sql/colexec/group/types.go @@ -93,6 +93,10 @@ type Group struct { GroupingFlag []bool // agg info and agg column. Aggs []aggexec.AggFuncExecExpression + + // spill configuration + SpillManager SpillManager + SpillThreshold int64 } func (group *Group) evaluateGroupByAndAgg(proc *process.Process, bat *batch.Batch) (err error) { From 8ca6f24f20a9bacf7c2b7b48f6736783b16e61a5 Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 29 Aug 2025 14:14:01 +0800 Subject: [PATCH 03/57] colexec/group: add spill states to container --- pkg/sql/colexec/group/types.go | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/pkg/sql/colexec/group/types.go b/pkg/sql/colexec/group/types.go index ee62248385ef8..ae3e713963a05 100644 --- a/pkg/sql/colexec/group/types.go +++ b/pkg/sql/colexec/group/types.go @@ -165,6 +165,11 @@ type container struct { result1 GroupResultBuffer // result if NeedEval is false. result2 GroupResultNoneBlock + + // spill state + currentMemUsage int64 + spilledStates []SpillID + spillPending bool } func (ctr *container) isDataSourceEmpty() bool { @@ -177,6 +182,11 @@ func (group *Group) Free(proc *process.Process, _ bool, _ error) { group.ctr.freeGroupEvaluate() group.ctr.freeAggEvaluate() group.FreeProjection(proc) + + if group.SpillManager != nil { + group.SpillManager.Free() + group.SpillManager = nil + } } func (group *Group) Reset(proc *process.Process, pipelineFailed bool, err error) { @@ -193,6 +203,13 @@ func (group *Group) freeCannotReuse(mp *mpool.MPool) { group.ctr.hr.Free0() group.ctr.result1.Free0(mp) group.ctr.result2.Free0(mp) + + for _, id := range group.ctr.spilledStates { + if group.SpillManager != nil { + group.SpillManager.Delete(id) + } + } + group.ctr.spilledStates = nil } func (ctr *container) freeAggEvaluate() { From 0fb798818d08818f0064a1c212d5920191346eeb Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 29 Aug 2025 14:21:18 +0800 Subject: [PATCH 04/57] colexec/group: implement spill --- pkg/sql/colexec/group/exec.go | 19 ++++ pkg/sql/colexec/group/group_spill.go | 152 +++++++++++++++++++++++++++ 2 files changed, 171 insertions(+) create mode 100644 pkg/sql/colexec/group/group_spill.go diff --git a/pkg/sql/colexec/group/exec.go b/pkg/sql/colexec/group/exec.go index 700d5aa1ccadd..8d184cf6d46cc 100644 --- a/pkg/sql/colexec/group/exec.go +++ b/pkg/sql/colexec/group/exec.go @@ -63,6 +63,14 @@ func (group *Group) Prepare(proc *process.Process) (err error) { if err = group.prepareGroup(proc); err != nil { return err } + + if group.SpillManager == nil { + group.SpillManager = NewMemorySpillManager() + } + if group.SpillThreshold <= 0 { + group.SpillThreshold = 128 * 1024 * 1024 + } + return group.PrepareProjection(proc) } @@ -188,6 +196,12 @@ func (group *Group) callToGetFinalResult(proc *process.Process) (*batch.Batch, e for { if group.ctr.state == vm.Eval { + if len(group.ctr.spilledStates) > 0 { + if err := group.mergeSpilledResults(proc); err != nil { + return nil, err + } + } + if group.ctr.result1.IsEmpty() { group.ctr.state = vm.End return nil, nil @@ -315,6 +329,11 @@ func (group *Group) consumeBatchToGetFinalResult( } } + group.updateMemoryUsage(proc) + if group.shouldSpill() { + return group.spillPartialResults(proc) + } + return nil } diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go new file mode 100644 index 0000000000000..9c082debf6a0d --- /dev/null +++ b/pkg/sql/colexec/group/group_spill.go @@ -0,0 +1,152 @@ +// Copyright 2025 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package group + +import ( + "fmt" + + "github.com/matrixorigin/matrixone/pkg/container/vector" + "github.com/matrixorigin/matrixone/pkg/sql/colexec/aggexec" + "github.com/matrixorigin/matrixone/pkg/vm/process" +) + +func (group *Group) shouldSpill() bool { + return group.ctr.currentMemUsage > group.SpillThreshold && len(group.ctr.result1.AggList) > 0 +} + +func (group *Group) updateMemoryUsage(proc *process.Process) { + usage := int64(0) + if !group.ctr.hr.IsEmpty() && group.ctr.hr.Hash != nil { + usage += int64(group.ctr.hr.Hash.Size()) + } + for _, bat := range group.ctr.result1.ToPopped { + if bat != nil { + usage += int64(bat.Size()) + } + } + for _, agg := range group.ctr.result1.AggList { + if agg != nil { + usage += agg.Size() + } + } + group.ctr.currentMemUsage = usage +} + +func (group *Group) spillPartialResults(proc *process.Process) error { + if len(group.ctr.result1.AggList) == 0 || len(group.ctr.result1.ToPopped) == 0 { + return nil + } + + partialStates := make([]any, len(group.ctr.result1.AggList)) + for i, agg := range group.ctr.result1.AggList { + if agg != nil { + partial, err := aggexec.MarshalAggFuncExec(agg) + if err != nil { + return err + } + partialStates[i] = partial + } + } + + groupVecs := make([]*vector.Vector, 0) + totalGroups := 0 + for _, bat := range group.ctr.result1.ToPopped { + if bat != nil && bat.RowCount() > 0 { + totalGroups += bat.RowCount() + for _, vec := range bat.Vecs { + if vec != nil { + groupVecs = append(groupVecs, vec) + } + } + } + } + + spillData := &SpillableAggState{ + GroupVectors: groupVecs, + PartialStates: partialStates, + GroupCount: totalGroups, + } + + spillID, err := group.SpillManager.Spill(spillData) + if err != nil { + return err + } + + group.ctr.spilledStates = append(group.ctr.spilledStates, spillID) + + for _, agg := range group.ctr.result1.AggList { + if agg != nil { + agg.Free() + } + } + group.ctr.result1.AggList = nil + + for _, bat := range group.ctr.result1.ToPopped { + if bat != nil { + bat.CleanOnlyData() + } + } + group.ctr.result1.ToPopped = group.ctr.result1.ToPopped[:0] + + if group.ctr.hr.Hash != nil { + group.ctr.hr.Hash.Free() + group.ctr.hr.Hash = nil + } + + group.ctr.currentMemUsage = 0 + return nil +} + +func (group *Group) mergeSpilledResults(proc *process.Process) error { + if len(group.ctr.spilledStates) == 0 { + return nil + } + + if group.ctr.result1.IsEmpty() { + aggs, err := group.generateAggExec(proc) + if err != nil { + return err + } + if err = group.ctr.result1.InitWithGroupBy( + proc.Mp(), + aggexec.GetMinAggregatorsChunkSize(group.ctr.groupByEvaluate.Vec, aggs), aggs, group.ctr.groupByEvaluate.Vec, 0); err != nil { + return err + } + } + + for _, spillID := range group.ctr.spilledStates { + spillData, err := group.SpillManager.Retrieve(spillID) + if err != nil { + return err + } + + spillState, ok := spillData.(*SpillableAggState) + if !ok { + return fmt.Errorf("invalid spilled data type") + } + + if err = group.ctr.result1.DealPartialResult(spillState.PartialStates); err != nil { + return err + } + + spillData.Free(proc.Mp()) + if err = group.SpillManager.Delete(spillID); err != nil { + return err + } + } + + group.ctr.spilledStates = nil + return nil +} From 5bd4966814b28993e062c47fda58bff5f78e6d9d Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 29 Aug 2025 14:21:30 +0800 Subject: [PATCH 05/57] colexec/group: add spill tests --- pkg/sql/colexec/group/spill_test.go | 73 +++++++++++++++++++++++++++++ 1 file changed, 73 insertions(+) create mode 100644 pkg/sql/colexec/group/spill_test.go diff --git a/pkg/sql/colexec/group/spill_test.go b/pkg/sql/colexec/group/spill_test.go new file mode 100644 index 0000000000000..585f381789b12 --- /dev/null +++ b/pkg/sql/colexec/group/spill_test.go @@ -0,0 +1,73 @@ +// Copyright 2025 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package group + +import ( + "testing" + + "github.com/matrixorigin/matrixone/pkg/container/batch" + "github.com/matrixorigin/matrixone/pkg/pb/plan" + "github.com/matrixorigin/matrixone/pkg/sql/colexec/aggexec" + "github.com/matrixorigin/matrixone/pkg/testutil" + "github.com/stretchr/testify/require" +) + +func TestSpill(t *testing.T) { + proc := testutil.NewProcess(t) + + before := proc.Mp().CurrNB() + exec, restore := hackMakeAggToTest(1) + defer restore() + + datas := []*batch.Batch{ + getGroupTestBatch(proc.Mp(), [][2]int64{ + {1, 1}, {1, 2}, {2, 3}, {2, 4}, {3, 5}, + }), + nil, + } + + g, src := getGroupOperatorWithInputs(datas) + g.NeedEval = true + g.SpillThreshold = 100 + g.Exprs = []*plan.Expr{newColumnExpression(0)} + g.Aggs = []aggexec.AggFuncExecExpression{ + aggexec.MakeAggFunctionExpression(0, false, []*plan.Expr{newColumnExpression(1)}, nil), + } + + require.NoError(t, src.Prepare(proc)) + require.NoError(t, g.Prepare(proc)) + + require.NotNil(t, g.SpillManager) + require.Equal(t, int64(100), g.SpillThreshold) + + r, err := g.Call(proc) + require.NoError(t, err) + require.NotNil(t, r.Batch) + + if final := r.Batch; final != nil { + require.Equal(t, 0, len(final.Aggs)) + require.Equal(t, 2, len(final.Vecs)) + require.Equal(t, hackVecResult, final.Vecs[1]) + } + + r, err = g.Call(proc) + require.NoError(t, err) + require.Nil(t, r.Batch) + + g.Free(proc, false, nil) + src.Free(proc, false, nil) + require.Equal(t, true, exec.isFree) + require.Equal(t, before, proc.Mp().CurrNB()) +} From 8ed5298e113337b6bbff14b47313c29ca215f7d1 Mon Sep 17 00:00:00 2001 From: reus Date: Wed, 3 Sep 2025 15:17:09 +0800 Subject: [PATCH 06/57] refinements --- pkg/sql/colexec/group/exec_test.go | 4 + pkg/sql/colexec/group/group_spill.go | 231 +++++++++++++++++-- pkg/sql/colexec/group/spill_test.go | 84 ++++++- pkg/sql/colexec/group/spillable_agg_state.go | 114 +++++++-- 4 files changed, 389 insertions(+), 44 deletions(-) diff --git a/pkg/sql/colexec/group/exec_test.go b/pkg/sql/colexec/group/exec_test.go index 5e53f82e8aef2..6f5012d400c4e 100644 --- a/pkg/sql/colexec/group/exec_test.go +++ b/pkg/sql/colexec/group/exec_test.go @@ -46,6 +46,10 @@ func (h *hackAggExecToTest) GetOptResult() aggexec.SplitResult { return nil } +func (h *hackAggExecToTest) Size() int64 { + return 0 +} + func (h *hackAggExecToTest) GroupGrow(more int) error { h.groupNumber += more return nil diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index 9c082debf6a0d..62774b0f355be 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -49,38 +49,66 @@ func (group *Group) spillPartialResults(proc *process.Process) error { return nil } - partialStates := make([]any, len(group.ctr.result1.AggList)) + marshaledAggStates := make([][]byte, len(group.ctr.result1.AggList)) for i, agg := range group.ctr.result1.AggList { if agg != nil { - partial, err := aggexec.MarshalAggFuncExec(agg) + marshaledData, err := aggexec.MarshalAggFuncExec(agg) if err != nil { - return err + return fmt.Errorf("failed to marshal aggregator %d: %v", i, err) } - partialStates[i] = partial + marshaledAggStates[i] = marshaledData } } - groupVecs := make([]*vector.Vector, 0) totalGroups := 0 for _, bat := range group.ctr.result1.ToPopped { - if bat != nil && bat.RowCount() > 0 { + if bat != nil { totalGroups += bat.RowCount() - for _, vec := range bat.Vecs { - if vec != nil { - groupVecs = append(groupVecs, vec) + } + } + + if totalGroups == 0 { + return nil + } + + var groupVecs []*vector.Vector + if len(group.ctr.result1.ToPopped) > 0 && group.ctr.result1.ToPopped[0] != nil { + numGroupByCols := len(group.ctr.result1.ToPopped[0].Vecs) + groupVecs = make([]*vector.Vector, numGroupByCols) + + for i := 0; i < numGroupByCols; i++ { + if len(group.ctr.result1.ToPopped[0].Vecs) > i && group.ctr.result1.ToPopped[0].Vecs[i] != nil { + groupVecs[i] = vector.NewVec(*group.ctr.result1.ToPopped[0].Vecs[i].GetType()) + } + } + + for _, bat := range group.ctr.result1.ToPopped { + if bat != nil && bat.RowCount() > 0 { + for i, vec := range bat.Vecs { + if i < len(groupVecs) && groupVecs[i] != nil && vec != nil { + if err := groupVecs[i].UnionBatch(vec, 0, vec.Length(), nil, proc.Mp()); err != nil { + for j := 0; j < len(groupVecs); j++ { + if groupVecs[j] != nil { + groupVecs[j].Free(proc.Mp()) + } + } + return err + } + } } } } } spillData := &SpillableAggState{ - GroupVectors: groupVecs, - PartialStates: partialStates, - GroupCount: totalGroups, + GroupVectors: groupVecs, + MarshaledAggStates: marshaledAggStates, + GroupCount: totalGroups, } spillID, err := group.SpillManager.Spill(spillData) if err != nil { + spillData.Free(proc.Mp()) return err } @@ -137,7 +165,8 @@ func (group *Group) mergeSpilledResults(proc *process.Process) error { return fmt.Errorf("invalid spilled data type") } - if err = group.ctr.result1.DealPartialResult(spillState.PartialStates); err != nil { + if err = group.restoreAndMergeSpilledAggregators(proc, spillState); err != nil { + spillData.Free(proc.Mp()) return err } @@ -150,3 +179,179 @@ func (group *Group) mergeSpilledResults(proc *process.Process) error { group.ctr.spilledStates = nil return nil } + +func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spillState *SpillableAggState) error { + if len(spillState.MarshaledAggStates) == 0 { + return nil + } + + if len(group.ctr.result1.AggList) == 0 { + aggs := make([]aggexec.AggFuncExec, len(spillState.MarshaledAggStates)) + + for i, marshaledState := range spillState.MarshaledAggStates { + if len(marshaledState) == 0 { + continue + } + + aggExpr := group.Aggs[i] + agg, err := makeAggExec(proc, aggExpr.GetAggID(), aggExpr.IsDistinct(), group.ctr.aggregateEvaluate[i].Typ...) + if err != nil { + for j := 0; j < i; j++ { + if aggs[j] != nil { + aggs[j].Free() + } + } + return err + } + + if config := aggExpr.GetExtraConfig(); config != nil { + if err = agg.SetExtraInformation(config, 0); err != nil { + agg.Free() + for j := 0; j < i; j++ { + if aggs[j] != nil { + aggs[j].Free() + } + } + return err + } + } + + agg, err = aggexec.UnmarshalAggFuncExec(aggexec.NewSimpleAggMemoryManager(proc.Mp()), marshaledState) + if err != nil { + agg.Free() + for j := 0; j < i; j++ { + if aggs[j] != nil { + aggs[j].Free() + } + } + return err + } + + aggs[i] = agg + } + + group.ctr.result1.AggList = aggs + + if len(spillState.GroupVectors) > 0 && spillState.GroupCount > 0 { + chunkSize := aggexec.GetMinAggregatorsChunkSize(spillState.GroupVectors, aggs) + + for offset := 0; offset < spillState.GroupCount; offset += chunkSize { + size := chunkSize + if offset+size > spillState.GroupCount { + size = spillState.GroupCount - offset + } + + bat := getInitialBatchWithSameTypeVecs(spillState.GroupVectors) + for i, vec := range spillState.GroupVectors { + if vec != nil && i < len(bat.Vecs) { + if err := bat.Vecs[i].UnionBatch(vec, int64(offset), size, nil, proc.Mp()); err != nil { + bat.Clean(proc.Mp()) + return err + } + } + } + bat.SetRowCount(size) + group.ctr.result1.ToPopped = append(group.ctr.result1.ToPopped, bat) + } + } + + return nil + } + + for _, currentAgg := range group.ctr.result1.AggList { + if currentAgg != nil { + if err := currentAgg.GroupGrow(spillState.GroupCount); err != nil { + return err + } + } + } + + tempAggs := make([]aggexec.AggFuncExec, len(spillState.MarshaledAggStates)) + defer func() { + for _, agg := range tempAggs { + if agg != nil { + agg.Free() + } + } + }() + + for i, marshaledState := range spillState.MarshaledAggStates { + if len(marshaledState) == 0 { + continue + } + + aggExpr := group.Aggs[i] + agg, err := makeAggExec(proc, aggExpr.GetAggID(), aggExpr.IsDistinct(), group.ctr.aggregateEvaluate[i].Typ...) + if err != nil { + return err + } + + if config := aggExpr.GetExtraConfig(); config != nil { + if err = agg.SetExtraInformation(config, 0); err != nil { + agg.Free() + return err + } + } + + agg, err = aggexec.UnmarshalAggFuncExec(aggexec.NewSimpleAggMemoryManager(proc.Mp()), marshaledState) + if err != nil { + agg.Free() + return err + } + + tempAggs[i] = agg + } + + currentGroupCount := 0 + for _, bat := range group.ctr.result1.ToPopped { + if bat != nil { + currentGroupCount += bat.RowCount() + } + } + + for i, tempAgg := range tempAggs { + if tempAgg == nil { + continue + } + + currentAgg := group.ctr.result1.AggList[i] + if currentAgg == nil { + continue + } + + for spilledGroupIdx := 0; spilledGroupIdx < spillState.GroupCount; spilledGroupIdx++ { + currentGroupIdx := currentGroupCount + spilledGroupIdx + if err := currentAgg.Merge(tempAgg, currentGroupIdx, spilledGroupIdx); err != nil { + return err + } + } + } + + if len(spillState.GroupVectors) > 0 && spillState.GroupCount > 0 { + chunkSize := group.ctr.result1.ChunkSize + if chunkSize == 0 { + chunkSize = spillState.GroupCount + } + + for offset := 0; offset < spillState.GroupCount; offset += chunkSize { + size := chunkSize + if offset+size > spillState.GroupCount { + size = spillState.GroupCount - offset + } + + bat := getInitialBatchWithSameTypeVecs(spillState.GroupVectors) + for i, vec := range spillState.GroupVectors { + if vec != nil && i < len(bat.Vecs) { + if err := bat.Vecs[i].UnionBatch(vec, int64(offset), size, nil, proc.Mp()); err != nil { + bat.Clean(proc.Mp()) + return err + } + } + } + bat.SetRowCount(size) + group.ctr.result1.ToPopped = append(group.ctr.result1.ToPopped, bat) + } + } + + return nil +} diff --git a/pkg/sql/colexec/group/spill_test.go b/pkg/sql/colexec/group/spill_test.go index 585f381789b12..ed24b27579687 100644 --- a/pkg/sql/colexec/group/spill_test.go +++ b/pkg/sql/colexec/group/spill_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/matrixorigin/matrixone/pkg/container/batch" + "github.com/matrixorigin/matrixone/pkg/container/vector" "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/sql/colexec/aggexec" "github.com/matrixorigin/matrixone/pkg/testutil" @@ -26,14 +27,11 @@ import ( func TestSpill(t *testing.T) { proc := testutil.NewProcess(t) - before := proc.Mp().CurrNB() - exec, restore := hackMakeAggToTest(1) - defer restore() datas := []*batch.Batch{ getGroupTestBatch(proc.Mp(), [][2]int64{ - {1, 1}, {1, 2}, {2, 3}, {2, 4}, {3, 5}, + {1, 10}, {1, 20}, {2, 30}, {2, 40}, {3, 50}, }), nil, } @@ -43,7 +41,7 @@ func TestSpill(t *testing.T) { g.SpillThreshold = 100 g.Exprs = []*plan.Expr{newColumnExpression(0)} g.Aggs = []aggexec.AggFuncExecExpression{ - aggexec.MakeAggFunctionExpression(0, false, []*plan.Expr{newColumnExpression(1)}, nil), + aggexec.MakeAggFunctionExpression(aggexec.AggIdOfCountStar, false, []*plan.Expr{newColumnExpression(1)}, nil), } require.NoError(t, src.Prepare(proc)) @@ -59,7 +57,23 @@ func TestSpill(t *testing.T) { if final := r.Batch; final != nil { require.Equal(t, 0, len(final.Aggs)) require.Equal(t, 2, len(final.Vecs)) - require.Equal(t, hackVecResult, final.Vecs[1]) + + groupVec := final.Vecs[0] + countVec := final.Vecs[1] + + require.Equal(t, 3, groupVec.Length()) + require.Equal(t, 3, countVec.Length()) + + groups := vector.MustFixedColNoTypeCheck[int64](groupVec) + counts := vector.MustFixedColNoTypeCheck[int64](countVec) + + expectedGroups := []int64{1, 2, 3} + expectedCounts := []int64{2, 2, 1} + + for i := 0; i < 3; i++ { + require.Equal(t, expectedGroups[i], groups[i]) + require.Equal(t, expectedCounts[i], counts[i]) + } } r, err = g.Call(proc) @@ -68,6 +82,62 @@ func TestSpill(t *testing.T) { g.Free(proc, false, nil) src.Free(proc, false, nil) - require.Equal(t, true, exec.isFree) + require.Equal(t, before, proc.Mp().CurrNB()) +} + +func TestSpillMultipleCycles(t *testing.T) { + proc := testutil.NewProcess(t) + before := proc.Mp().CurrNB() + + datas := []*batch.Batch{ + getGroupTestBatch(proc.Mp(), [][2]int64{ + {1, 1}, {2, 2}, {3, 3}, + }), + getGroupTestBatch(proc.Mp(), [][2]int64{ + {4, 4}, {5, 5}, {6, 6}, + }), + getGroupTestBatch(proc.Mp(), [][2]int64{ + {1, 7}, {2, 8}, {3, 9}, + }), + nil, + } + + g, src := getGroupOperatorWithInputs(datas) + g.NeedEval = true + g.SpillThreshold = 10 + g.Exprs = []*plan.Expr{newColumnExpression(0)} + g.Aggs = []aggexec.AggFuncExecExpression{ + aggexec.MakeAggFunctionExpression(aggexec.AggIdOfCountStar, false, []*plan.Expr{newColumnExpression(1)}, nil), + } + + require.NoError(t, src.Prepare(proc)) + require.NoError(t, g.Prepare(proc)) + + r, err := g.Call(proc) + require.NoError(t, err) + require.NotNil(t, r.Batch) + + if final := r.Batch; final != nil { + require.Equal(t, 2, len(final.Vecs)) + require.Equal(t, 6, final.Vecs[0].Length()) + + groups := vector.MustFixedColNoTypeCheck[int64](final.Vecs[0]) + counts := vector.MustFixedColNoTypeCheck[int64](final.Vecs[1]) + + groupCounts := make(map[int64]int64) + for i := 0; i < len(groups); i++ { + groupCounts[groups[i]] = counts[i] + } + + require.Equal(t, int64(2), groupCounts[1]) + require.Equal(t, int64(2), groupCounts[2]) + require.Equal(t, int64(2), groupCounts[3]) + require.Equal(t, int64(1), groupCounts[4]) + require.Equal(t, int64(1), groupCounts[5]) + require.Equal(t, int64(1), groupCounts[6]) + } + + g.Free(proc, false, nil) + src.Free(proc, false, nil) require.Equal(t, before, proc.Mp().CurrNB()) } diff --git a/pkg/sql/colexec/group/spillable_agg_state.go b/pkg/sql/colexec/group/spillable_agg_state.go index ed5716f178604..e45ea6baddb48 100644 --- a/pkg/sql/colexec/group/spillable_agg_state.go +++ b/pkg/sql/colexec/group/spillable_agg_state.go @@ -15,51 +15,115 @@ package group import ( - "encoding/json" + "bytes" + "encoding/binary" "github.com/matrixorigin/matrixone/pkg/common/mpool" + "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/container/vector" ) type SpillableAggState struct { - GroupVectors []*vector.Vector - PartialStates []any - GroupCount int + GroupVectors []*vector.Vector + MarshaledAggStates [][]byte + GroupCount int } func (s *SpillableAggState) Serialize() ([]byte, error) { - data := map[string]interface{}{ - "group_count": s.GroupCount, - "partial_states": s.PartialStates, - "group_vectors": make([]map[string]interface{}, len(s.GroupVectors)), + buf := bytes.NewBuffer(nil) + + if err := binary.Write(buf, binary.LittleEndian, int32(s.GroupCount)); err != nil { + return nil, err } - for i, vec := range s.GroupVectors { - if vec != nil { - vecData := map[string]interface{}{ - "type": vec.GetType().String(), - "length": vec.Length(), - "is_const": vec.IsConst(), + if err := binary.Write(buf, binary.LittleEndian, int32(len(s.GroupVectors))); err != nil { + return nil, err + } + for _, vec := range s.GroupVectors { + if vec == nil { + if err := binary.Write(buf, binary.LittleEndian, int32(0)); err != nil { + return nil, err } - data["group_vectors"].([]map[string]interface{})[i] = vecData + continue + } + + vecBytes, err := vec.MarshalBinary() + if err != nil { + return nil, err + } + if err := binary.Write(buf, binary.LittleEndian, int32(len(vecBytes))); err != nil { + return nil, err + } + if _, err := buf.Write(vecBytes); err != nil { + return nil, err } } - return json.Marshal(data) + if err := binary.Write(buf, binary.LittleEndian, int32(len(s.MarshaledAggStates))); err != nil { + return nil, err + } + for _, aggState := range s.MarshaledAggStates { + if err := binary.Write(buf, binary.LittleEndian, int32(len(aggState))); err != nil { + return nil, err + } + if _, err := buf.Write(aggState); err != nil { + return nil, err + } + } + + return buf.Bytes(), nil } func (s *SpillableAggState) Deserialize(data []byte) error { - var parsed map[string]interface{} - if err := json.Unmarshal(data, &parsed); err != nil { + buf := bytes.NewReader(data) + + var groupCount int32 + if err := binary.Read(buf, binary.LittleEndian, &groupCount); err != nil { return err } + s.GroupCount = int(groupCount) - if count, ok := parsed["group_count"].(float64); ok { - s.GroupCount = int(count) + var groupVecCount int32 + if err := binary.Read(buf, binary.LittleEndian, &groupVecCount); err != nil { + return err } + s.GroupVectors = make([]*vector.Vector, groupVecCount) + for i := 0; i < int(groupVecCount); i++ { + var size int32 + if err := binary.Read(buf, binary.LittleEndian, &size); err != nil { + return err + } + if size == 0 { + s.GroupVectors[i] = nil + continue + } - if states, ok := parsed["partial_states"].([]interface{}); ok { - s.PartialStates = states + vecBytes := make([]byte, size) + if _, err := buf.Read(vecBytes); err != nil { + return err + } + + vec := vector.NewVec(types.T_any.ToType()) + if err := vec.UnmarshalBinary(vecBytes); err != nil { + return err + } + s.GroupVectors[i] = vec + } + + var aggStateCount int32 + if err := binary.Read(buf, binary.LittleEndian, &aggStateCount); err != nil { + return err + } + s.MarshaledAggStates = make([][]byte, aggStateCount) + for i := 0; i < int(aggStateCount); i++ { + var size int32 + if err := binary.Read(buf, binary.LittleEndian, &size); err != nil { + return err + } + s.MarshaledAggStates[i] = make([]byte, size) + if _, err := buf.Read(s.MarshaledAggStates[i]); err != nil { + return err + } } return nil @@ -72,7 +136,9 @@ func (s *SpillableAggState) EstimateSize() int64 { size += int64(vec.Size()) } } - size += int64(len(s.PartialStates) * 64) + for _, aggState := range s.MarshaledAggStates { + size += int64(len(aggState)) + } return size } @@ -83,5 +149,5 @@ func (s *SpillableAggState) Free(mp *mpool.MPool) { } } s.GroupVectors = nil - s.PartialStates = nil + s.MarshaledAggStates = nil } From 3497546409bd313471833482db7a366fcddbe952 Mon Sep 17 00:00:00 2001 From: reus Date: Mon, 8 Sep 2025 17:08:16 +0800 Subject: [PATCH 07/57] fix --- pkg/sql/colexec/group/spillable_agg_state.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/sql/colexec/group/spillable_agg_state.go b/pkg/sql/colexec/group/spillable_agg_state.go index e45ea6baddb48..8053baa2e64b0 100644 --- a/pkg/sql/colexec/group/spillable_agg_state.go +++ b/pkg/sql/colexec/group/spillable_agg_state.go @@ -133,7 +133,7 @@ func (s *SpillableAggState) EstimateSize() int64 { size := int64(0) for _, vec := range s.GroupVectors { if vec != nil { - size += int64(vec.Size()) + size += int64(vec.Allocated()) } } for _, aggState := range s.MarshaledAggStates { From 596573caa8285a92942fb63e8a202c513d8b0332 Mon Sep 17 00:00:00 2001 From: reus Date: Wed, 10 Sep 2025 12:45:16 +0800 Subject: [PATCH 08/57] fix restoreAndMergeSpilledAggregators --- pkg/sql/colexec/group/group_spill.go | 25 ++++--------------------- 1 file changed, 4 insertions(+), 21 deletions(-) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index 62774b0f355be..192649bb089ca 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -193,8 +193,7 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi continue } - aggExpr := group.Aggs[i] - agg, err := makeAggExec(proc, aggExpr.GetAggID(), aggExpr.IsDistinct(), group.ctr.aggregateEvaluate[i].Typ...) + agg, err := aggexec.UnmarshalAggFuncExec(aggexec.NewSimpleAggMemoryManager(proc.Mp()), marshaledState) if err != nil { for j := 0; j < i; j++ { if aggs[j] != nil { @@ -204,6 +203,7 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi return err } + aggExpr := group.Aggs[i] if config := aggExpr.GetExtraConfig(); config != nil { if err = agg.SetExtraInformation(config, 0); err != nil { agg.Free() @@ -216,17 +216,6 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi } } - agg, err = aggexec.UnmarshalAggFuncExec(aggexec.NewSimpleAggMemoryManager(proc.Mp()), marshaledState) - if err != nil { - agg.Free() - for j := 0; j < i; j++ { - if aggs[j] != nil { - aggs[j].Free() - } - } - return err - } - aggs[i] = agg } @@ -280,12 +269,12 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi continue } - aggExpr := group.Aggs[i] - agg, err := makeAggExec(proc, aggExpr.GetAggID(), aggExpr.IsDistinct(), group.ctr.aggregateEvaluate[i].Typ...) + agg, err := aggexec.UnmarshalAggFuncExec(aggexec.NewSimpleAggMemoryManager(proc.Mp()), marshaledState) if err != nil { return err } + aggExpr := group.Aggs[i] if config := aggExpr.GetExtraConfig(); config != nil { if err = agg.SetExtraInformation(config, 0); err != nil { agg.Free() @@ -293,12 +282,6 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi } } - agg, err = aggexec.UnmarshalAggFuncExec(aggexec.NewSimpleAggMemoryManager(proc.Mp()), marshaledState) - if err != nil { - agg.Free() - return err - } - tempAggs[i] = agg } From 287f35da64a11d4673ce92d1863e710ec379c0ec Mon Sep 17 00:00:00 2001 From: reus Date: Wed, 10 Sep 2025 13:01:54 +0800 Subject: [PATCH 09/57] fix memory leak in spillPartialResults --- pkg/sql/colexec/group/group_spill.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index 192649bb089ca..278e24553d795 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -112,6 +112,8 @@ func (group *Group) spillPartialResults(proc *process.Process) error { return err } + spillData.Free(proc.Mp()) + group.ctr.spilledStates = append(group.ctr.spilledStates, spillID) for _, agg := range group.ctr.result1.AggList { From ed8047752e4fde4e34fe1bbb884b4a9ca841d8f9 Mon Sep 17 00:00:00 2001 From: reus Date: Wed, 10 Sep 2025 13:09:45 +0800 Subject: [PATCH 10/57] fix memory leak in spill tests --- pkg/sql/colexec/group/spill_test.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pkg/sql/colexec/group/spill_test.go b/pkg/sql/colexec/group/spill_test.go index ed24b27579687..6b948cad7c381 100644 --- a/pkg/sql/colexec/group/spill_test.go +++ b/pkg/sql/colexec/group/spill_test.go @@ -74,6 +74,8 @@ func TestSpill(t *testing.T) { require.Equal(t, expectedGroups[i], groups[i]) require.Equal(t, expectedCounts[i], counts[i]) } + + final.Clean(proc.Mp()) } r, err = g.Call(proc) @@ -135,6 +137,8 @@ func TestSpillMultipleCycles(t *testing.T) { require.Equal(t, int64(1), groupCounts[4]) require.Equal(t, int64(1), groupCounts[5]) require.Equal(t, int64(1), groupCounts[6]) + + final.Clean(proc.Mp()) } g.Free(proc, false, nil) From be89bff6547ec986bfa2ca5eaa6eb83cd108a4a8 Mon Sep 17 00:00:00 2001 From: reus Date: Wed, 10 Sep 2025 15:40:19 +0800 Subject: [PATCH 11/57] update --- pkg/sql/colexec/group/spill_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/sql/colexec/group/spill_test.go b/pkg/sql/colexec/group/spill_test.go index 6b948cad7c381..7674aee2b5f2d 100644 --- a/pkg/sql/colexec/group/spill_test.go +++ b/pkg/sql/colexec/group/spill_test.go @@ -38,7 +38,7 @@ func TestSpill(t *testing.T) { g, src := getGroupOperatorWithInputs(datas) g.NeedEval = true - g.SpillThreshold = 100 + g.SpillThreshold = 10 g.Exprs = []*plan.Expr{newColumnExpression(0)} g.Aggs = []aggexec.AggFuncExecExpression{ aggexec.MakeAggFunctionExpression(aggexec.AggIdOfCountStar, false, []*plan.Expr{newColumnExpression(1)}, nil), @@ -48,7 +48,7 @@ func TestSpill(t *testing.T) { require.NoError(t, g.Prepare(proc)) require.NotNil(t, g.SpillManager) - require.Equal(t, int64(100), g.SpillThreshold) + require.Equal(t, int64(10), g.SpillThreshold) r, err := g.Call(proc) require.NoError(t, err) From cde158fbb0f629405b9a2fe47f8430d2429c33f3 Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 12 Sep 2025 14:57:13 +0800 Subject: [PATCH 12/57] fixes --- pkg/sql/colexec/group/group_spill.go | 2 +- pkg/sql/colexec/group/spill.go | 4 ++-- pkg/sql/colexec/group/spill_memory.go | 6 ++++-- pkg/sql/colexec/group/spillable_agg_state.go | 4 ++-- 4 files changed, 9 insertions(+), 7 deletions(-) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index 278e24553d795..d3cf205ce12f1 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -157,7 +157,7 @@ func (group *Group) mergeSpilledResults(proc *process.Process) error { } for _, spillID := range group.ctr.spilledStates { - spillData, err := group.SpillManager.Retrieve(spillID) + spillData, err := group.SpillManager.Retrieve(spillID, proc.Mp()) if err != nil { return err } diff --git a/pkg/sql/colexec/group/spill.go b/pkg/sql/colexec/group/spill.go index a64f87cdc6b79..cda576ca787bc 100644 --- a/pkg/sql/colexec/group/spill.go +++ b/pkg/sql/colexec/group/spill.go @@ -20,14 +20,14 @@ type SpillID string type SpillableData interface { Serialize() ([]byte, error) - Deserialize(data []byte) error + Deserialize(data []byte, mp *mpool.MPool) error EstimateSize() int64 Free(mp *mpool.MPool) } type SpillManager interface { Spill(data SpillableData) (SpillID, error) - Retrieve(id SpillID) (SpillableData, error) + Retrieve(id SpillID, mp *mpool.MPool) (SpillableData, error) Delete(id SpillID) error Free() } diff --git a/pkg/sql/colexec/group/spill_memory.go b/pkg/sql/colexec/group/spill_memory.go index 6d018b2f57088..4205a2edcc9e2 100644 --- a/pkg/sql/colexec/group/spill_memory.go +++ b/pkg/sql/colexec/group/spill_memory.go @@ -17,6 +17,8 @@ package group import ( "fmt" "sync/atomic" + + "github.com/matrixorigin/matrixone/pkg/common/mpool" ) type MemorySpillManager struct { @@ -43,14 +45,14 @@ func (m *MemorySpillManager) Spill(data SpillableData) (SpillID, error) { return id, nil } -func (m *MemorySpillManager) Retrieve(id SpillID) (SpillableData, error) { +func (m *MemorySpillManager) Retrieve(id SpillID, mp *mpool.MPool) (SpillableData, error) { serialized, exists := m.data[id] if !exists { return nil, fmt.Errorf("spill data not found: %s", id) } data := &SpillableAggState{} - if err := data.Deserialize(serialized); err != nil { + if err := data.Deserialize(serialized, mp); err != nil { return nil, err } return data, nil diff --git a/pkg/sql/colexec/group/spillable_agg_state.go b/pkg/sql/colexec/group/spillable_agg_state.go index 8053baa2e64b0..f9ed9bfc2f6e4 100644 --- a/pkg/sql/colexec/group/spillable_agg_state.go +++ b/pkg/sql/colexec/group/spillable_agg_state.go @@ -74,7 +74,7 @@ func (s *SpillableAggState) Serialize() ([]byte, error) { return buf.Bytes(), nil } -func (s *SpillableAggState) Deserialize(data []byte) error { +func (s *SpillableAggState) Deserialize(data []byte, mp *mpool.MPool) error { buf := bytes.NewReader(data) var groupCount int32 @@ -104,7 +104,7 @@ func (s *SpillableAggState) Deserialize(data []byte) error { } vec := vector.NewVec(types.T_any.ToType()) - if err := vec.UnmarshalBinary(vecBytes); err != nil { + if err := vec.UnmarshalBinaryWithCopy(vecBytes, mp); err != nil { return err } s.GroupVectors[i] = vec From c0d1b561874256f909e7eab8a89d200bdbd1700f Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 12 Sep 2025 15:09:30 +0800 Subject: [PATCH 13/57] fixes --- pkg/sql/colexec/group/group_spill.go | 1 + pkg/sql/colexec/group/spill_memory.go | 1 + pkg/sql/colexec/group/spillable_agg_state.go | 1 + 3 files changed, 3 insertions(+) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index d3cf205ce12f1..8e35cf23764d9 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -164,6 +164,7 @@ func (group *Group) mergeSpilledResults(proc *process.Process) error { spillState, ok := spillData.(*SpillableAggState) if !ok { + spillData.Free(proc.Mp()) return fmt.Errorf("invalid spilled data type") } diff --git a/pkg/sql/colexec/group/spill_memory.go b/pkg/sql/colexec/group/spill_memory.go index 4205a2edcc9e2..992338e09fddd 100644 --- a/pkg/sql/colexec/group/spill_memory.go +++ b/pkg/sql/colexec/group/spill_memory.go @@ -53,6 +53,7 @@ func (m *MemorySpillManager) Retrieve(id SpillID, mp *mpool.MPool) (SpillableDat data := &SpillableAggState{} if err := data.Deserialize(serialized, mp); err != nil { + data.Free(mp) return nil, err } return data, nil diff --git a/pkg/sql/colexec/group/spillable_agg_state.go b/pkg/sql/colexec/group/spillable_agg_state.go index f9ed9bfc2f6e4..7c53e97aef265 100644 --- a/pkg/sql/colexec/group/spillable_agg_state.go +++ b/pkg/sql/colexec/group/spillable_agg_state.go @@ -105,6 +105,7 @@ func (s *SpillableAggState) Deserialize(data []byte, mp *mpool.MPool) error { vec := vector.NewVec(types.T_any.ToType()) if err := vec.UnmarshalBinaryWithCopy(vecBytes, mp); err != nil { + vec.Free(mp) return err } s.GroupVectors[i] = vec From d097b4609d39809f86e10ee6de9495a41b055943 Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 12 Sep 2025 18:54:11 +0800 Subject: [PATCH 14/57] fixes --- pkg/sql/colexec/group/group_spill.go | 2 +- pkg/sql/colexec/group/spillable_agg_state.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index 8e35cf23764d9..b56f82fa8c919 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -78,7 +78,7 @@ func (group *Group) spillPartialResults(proc *process.Process) error { for i := 0; i < numGroupByCols; i++ { if len(group.ctr.result1.ToPopped[0].Vecs) > i && group.ctr.result1.ToPopped[0].Vecs[i] != nil { - groupVecs[i] = vector.NewVec(*group.ctr.result1.ToPopped[0].Vecs[i].GetType()) + groupVecs[i] = vector.NewOffHeapVecWithType(*group.ctr.result1.ToPopped[0].Vecs[i].GetType()) } } diff --git a/pkg/sql/colexec/group/spillable_agg_state.go b/pkg/sql/colexec/group/spillable_agg_state.go index 7c53e97aef265..21fb8c4f0413a 100644 --- a/pkg/sql/colexec/group/spillable_agg_state.go +++ b/pkg/sql/colexec/group/spillable_agg_state.go @@ -103,7 +103,7 @@ func (s *SpillableAggState) Deserialize(data []byte, mp *mpool.MPool) error { return err } - vec := vector.NewVec(types.T_any.ToType()) + vec := vector.NewOffHeapVecWithType(types.T_any.ToType()) if err := vec.UnmarshalBinaryWithCopy(vecBytes, mp); err != nil { vec.Free(mp) return err From 3e2902435ac77971fbc7a86e3ca3be37b7159557 Mon Sep 17 00:00:00 2001 From: reus Date: Sat, 13 Sep 2025 00:00:28 +0800 Subject: [PATCH 15/57] fixes --- pkg/sql/colexec/group/group_spill.go | 8 ++- pkg/sql/colexec/group/spillable_agg_state.go | 54 +++++++++++++++++++- 2 files changed, 59 insertions(+), 3 deletions(-) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index b56f82fa8c919..7b400fce094bb 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -17,6 +17,7 @@ package group import ( "fmt" + "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/container/vector" "github.com/matrixorigin/matrixone/pkg/sql/colexec/aggexec" "github.com/matrixorigin/matrixone/pkg/vm/process" @@ -72,13 +73,17 @@ func (group *Group) spillPartialResults(proc *process.Process) error { } var groupVecs []*vector.Vector + var groupVecTypes []types.Type if len(group.ctr.result1.ToPopped) > 0 && group.ctr.result1.ToPopped[0] != nil { numGroupByCols := len(group.ctr.result1.ToPopped[0].Vecs) groupVecs = make([]*vector.Vector, numGroupByCols) + groupVecTypes = make([]types.Type, numGroupByCols) for i := 0; i < numGroupByCols; i++ { if len(group.ctr.result1.ToPopped[0].Vecs) > i && group.ctr.result1.ToPopped[0].Vecs[i] != nil { - groupVecs[i] = vector.NewOffHeapVecWithType(*group.ctr.result1.ToPopped[0].Vecs[i].GetType()) + vecType := *group.ctr.result1.ToPopped[0].Vecs[i].GetType() + groupVecs[i] = vector.NewOffHeapVecWithType(vecType) + groupVecTypes[i] = vecType } } @@ -102,6 +107,7 @@ func (group *Group) spillPartialResults(proc *process.Process) error { spillData := &SpillableAggState{ GroupVectors: groupVecs, + GroupVectorTypes: groupVecTypes, MarshaledAggStates: marshaledAggStates, GroupCount: totalGroups, } diff --git a/pkg/sql/colexec/group/spillable_agg_state.go b/pkg/sql/colexec/group/spillable_agg_state.go index 21fb8c4f0413a..685ff93f90965 100644 --- a/pkg/sql/colexec/group/spillable_agg_state.go +++ b/pkg/sql/colexec/group/spillable_agg_state.go @@ -25,6 +25,7 @@ import ( type SpillableAggState struct { GroupVectors []*vector.Vector + GroupVectorTypes []types.Type MarshaledAggStates [][]byte GroupCount int } @@ -39,7 +40,24 @@ func (s *SpillableAggState) Serialize() ([]byte, error) { if err := binary.Write(buf, binary.LittleEndian, int32(len(s.GroupVectors))); err != nil { return nil, err } - for _, vec := range s.GroupVectors { + + if err := binary.Write(buf, binary.LittleEndian, int32(len(s.GroupVectorTypes))); err != nil { + return nil, err + } + for _, typ := range s.GroupVectorTypes { + typBytes, err := typ.MarshalBinary() + if err != nil { + return nil, err + } + if err := binary.Write(buf, binary.LittleEndian, int32(len(typBytes))); err != nil { + return nil, err + } + if _, err := buf.Write(typBytes); err != nil { + return nil, err + } + } + + for i, vec := range s.GroupVectors { if vec == nil { if err := binary.Write(buf, binary.LittleEndian, int32(0)); err != nil { return nil, err @@ -57,6 +75,10 @@ func (s *SpillableAggState) Serialize() ([]byte, error) { if _, err := buf.Write(vecBytes); err != nil { return nil, err } + + if i >= len(s.GroupVectorTypes) { + s.GroupVectorTypes = append(s.GroupVectorTypes, *vec.GetType()) + } } if err := binary.Write(buf, binary.LittleEndian, int32(len(s.MarshaledAggStates))); err != nil { @@ -87,6 +109,26 @@ func (s *SpillableAggState) Deserialize(data []byte, mp *mpool.MPool) error { if err := binary.Read(buf, binary.LittleEndian, &groupVecCount); err != nil { return err } + + var groupVecTypeCount int32 + if err := binary.Read(buf, binary.LittleEndian, &groupVecTypeCount); err != nil { + return err + } + s.GroupVectorTypes = make([]types.Type, groupVecTypeCount) + for i := 0; i < int(groupVecTypeCount); i++ { + var size int32 + if err := binary.Read(buf, binary.LittleEndian, &size); err != nil { + return err + } + typBytes := make([]byte, size) + if _, err := buf.Read(typBytes); err != nil { + return err + } + if err := s.GroupVectorTypes[i].UnmarshalBinary(typBytes); err != nil { + return err + } + } + s.GroupVectors = make([]*vector.Vector, groupVecCount) for i := 0; i < int(groupVecCount); i++ { var size int32 @@ -103,7 +145,14 @@ func (s *SpillableAggState) Deserialize(data []byte, mp *mpool.MPool) error { return err } - vec := vector.NewOffHeapVecWithType(types.T_any.ToType()) + var vecType types.Type + if i < len(s.GroupVectorTypes) { + vecType = s.GroupVectorTypes[i] + } else { + vecType = types.T_any.ToType() + } + + vec := vector.NewOffHeapVecWithType(vecType) if err := vec.UnmarshalBinaryWithCopy(vecBytes, mp); err != nil { vec.Free(mp) return err @@ -150,5 +199,6 @@ func (s *SpillableAggState) Free(mp *mpool.MPool) { } } s.GroupVectors = nil + s.GroupVectorTypes = nil s.MarshaledAggStates = nil } From 607e33d12b0e397857bc583fd27a4dba10afc8fd Mon Sep 17 00:00:00 2001 From: reus Date: Sat, 13 Sep 2025 23:09:55 +0800 Subject: [PATCH 16/57] update --- pkg/sql/colexec/group/spill_test.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/pkg/sql/colexec/group/spill_test.go b/pkg/sql/colexec/group/spill_test.go index 7674aee2b5f2d..fa9cc609f9ed6 100644 --- a/pkg/sql/colexec/group/spill_test.go +++ b/pkg/sql/colexec/group/spill_test.go @@ -74,8 +74,6 @@ func TestSpill(t *testing.T) { require.Equal(t, expectedGroups[i], groups[i]) require.Equal(t, expectedCounts[i], counts[i]) } - - final.Clean(proc.Mp()) } r, err = g.Call(proc) @@ -137,8 +135,6 @@ func TestSpillMultipleCycles(t *testing.T) { require.Equal(t, int64(1), groupCounts[4]) require.Equal(t, int64(1), groupCounts[5]) require.Equal(t, int64(1), groupCounts[6]) - - final.Clean(proc.Mp()) } g.Free(proc, false, nil) From d7b123e17dfd45db3b057ebbbac5b5c0ec1925cb Mon Sep 17 00:00:00 2001 From: reus Date: Sun, 14 Sep 2025 20:53:00 +0800 Subject: [PATCH 17/57] update --- pkg/sql/colexec/group/group_spill.go | 59 +++++++++++++++++++--------- pkg/sql/colexec/group/spill_test.go | 17 ++++++++ 2 files changed, 57 insertions(+), 19 deletions(-) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index 7b400fce094bb..e2e01bc972576 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -17,6 +17,7 @@ package group import ( "fmt" + "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/container/vector" "github.com/matrixorigin/matrixone/pkg/sql/colexec/aggexec" @@ -79,6 +80,15 @@ func (group *Group) spillPartialResults(proc *process.Process) error { groupVecs = make([]*vector.Vector, numGroupByCols) groupVecTypes = make([]types.Type, numGroupByCols) + cleanupVecs := func() { + for i := range groupVecs { + if groupVecs[i] != nil { + groupVecs[i].Free(proc.Mp()) + groupVecs[i] = nil + } + } + } + for i := 0; i < numGroupByCols; i++ { if len(group.ctr.result1.ToPopped[0].Vecs) > i && group.ctr.result1.ToPopped[0].Vecs[i] != nil { vecType := *group.ctr.result1.ToPopped[0].Vecs[i].GetType() @@ -92,11 +102,7 @@ func (group *Group) spillPartialResults(proc *process.Process) error { for i, vec := range bat.Vecs { if i < len(groupVecs) && groupVecs[i] != nil && vec != nil { if err := groupVecs[i].UnionBatch(vec, 0, vec.Length(), nil, proc.Mp()); err != nil { - for j := 0; j < len(groupVecs); j++ { - if groupVecs[j] != nil { - groupVecs[j].Free(proc.Mp()) - } - } + cleanupVecs() return err } } @@ -196,6 +202,15 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi if len(group.ctr.result1.AggList) == 0 { aggs := make([]aggexec.AggFuncExec, len(spillState.MarshaledAggStates)) + defer func() { + if group.ctr.result1.AggList == nil { + for _, agg := range aggs { + if agg != nil { + agg.Free() + } + } + } + }() for i, marshaledState := range spillState.MarshaledAggStates { if len(marshaledState) == 0 { @@ -204,11 +219,6 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi agg, err := aggexec.UnmarshalAggFuncExec(aggexec.NewSimpleAggMemoryManager(proc.Mp()), marshaledState) if err != nil { - for j := 0; j < i; j++ { - if aggs[j] != nil { - aggs[j].Free() - } - } return err } @@ -216,11 +226,6 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi if config := aggExpr.GetExtraConfig(); config != nil { if err = agg.SetExtraInformation(config, 0); err != nil { agg.Free() - for j := 0; j < i; j++ { - if aggs[j] != nil { - aggs[j].Free() - } - } return err } } @@ -232,6 +237,7 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi if len(spillState.GroupVectors) > 0 && spillState.GroupCount > 0 { chunkSize := aggexec.GetMinAggregatorsChunkSize(spillState.GroupVectors, aggs) + batchesToAdd := make([]*batch.Batch, 0) for offset := 0; offset < spillState.GroupCount; offset += chunkSize { size := chunkSize @@ -240,17 +246,24 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi } bat := getInitialBatchWithSameTypeVecs(spillState.GroupVectors) + success := true for i, vec := range spillState.GroupVectors { if vec != nil && i < len(bat.Vecs) { if err := bat.Vecs[i].UnionBatch(vec, int64(offset), size, nil, proc.Mp()); err != nil { bat.Clean(proc.Mp()) + for _, b := range batchesToAdd { + b.Clean(proc.Mp()) + } return err } } } - bat.SetRowCount(size) - group.ctr.result1.ToPopped = append(group.ctr.result1.ToPopped, bat) + if success { + bat.SetRowCount(size) + batchesToAdd = append(batchesToAdd, bat) + } } + group.ctr.result1.ToPopped = append(group.ctr.result1.ToPopped, batchesToAdd...) } return nil @@ -325,6 +338,7 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi chunkSize = spillState.GroupCount } + batchesToAdd := make([]*batch.Batch, 0) for offset := 0; offset < spillState.GroupCount; offset += chunkSize { size := chunkSize if offset+size > spillState.GroupCount { @@ -332,17 +346,24 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi } bat := getInitialBatchWithSameTypeVecs(spillState.GroupVectors) + success := true for i, vec := range spillState.GroupVectors { if vec != nil && i < len(bat.Vecs) { if err := bat.Vecs[i].UnionBatch(vec, int64(offset), size, nil, proc.Mp()); err != nil { bat.Clean(proc.Mp()) + for _, b := range batchesToAdd { + b.Clean(proc.Mp()) + } return err } } } - bat.SetRowCount(size) - group.ctr.result1.ToPopped = append(group.ctr.result1.ToPopped, bat) + if success { + bat.SetRowCount(size) + batchesToAdd = append(batchesToAdd, bat) + } } + group.ctr.result1.ToPopped = append(group.ctr.result1.ToPopped, batchesToAdd...) } return nil diff --git a/pkg/sql/colexec/group/spill_test.go b/pkg/sql/colexec/group/spill_test.go index fa9cc609f9ed6..44840ec208a3c 100644 --- a/pkg/sql/colexec/group/spill_test.go +++ b/pkg/sql/colexec/group/spill_test.go @@ -35,6 +35,8 @@ func TestSpill(t *testing.T) { }), nil, } + afterDataCreation := proc.Mp().CurrNB() + t.Logf("After data creation: %d bytes allocated", afterDataCreation-before) g, src := getGroupOperatorWithInputs(datas) g.NeedEval = true @@ -45,7 +47,12 @@ func TestSpill(t *testing.T) { } require.NoError(t, src.Prepare(proc)) + afterSrcPrepare := proc.Mp().CurrNB() + t.Logf("After src prepare: %d bytes allocated", afterSrcPrepare-before) + require.NoError(t, g.Prepare(proc)) + afterGroupPrepare := proc.Mp().CurrNB() + t.Logf("After group prepare: %d bytes allocated", afterGroupPrepare-before) require.NotNil(t, g.SpillManager) require.Equal(t, int64(10), g.SpillThreshold) @@ -53,6 +60,8 @@ func TestSpill(t *testing.T) { r, err := g.Call(proc) require.NoError(t, err) require.NotNil(t, r.Batch) + afterFirstCall := proc.Mp().CurrNB() + t.Logf("After first call: %d bytes allocated", afterFirstCall-before) if final := r.Batch; final != nil { require.Equal(t, 0, len(final.Aggs)) @@ -79,9 +88,17 @@ func TestSpill(t *testing.T) { r, err = g.Call(proc) require.NoError(t, err) require.Nil(t, r.Batch) + afterSecondCall := proc.Mp().CurrNB() + t.Logf("After second call: %d bytes allocated", afterSecondCall-before) g.Free(proc, false, nil) + afterGroupFree := proc.Mp().CurrNB() + t.Logf("After group free: %d bytes allocated", afterGroupFree-before) + src.Free(proc, false, nil) + afterSrcFree := proc.Mp().CurrNB() + t.Logf("After src free: %d bytes allocated", afterSrcFree-before) + require.Equal(t, before, proc.Mp().CurrNB()) } From a2d7998b6f20a4ea91a12cc3237719c4c947e9db Mon Sep 17 00:00:00 2001 From: reus Date: Wed, 17 Sep 2025 14:38:14 +0800 Subject: [PATCH 18/57] fix init agg state --- pkg/sql/colexec/group/group_spill.go | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index e2e01bc972576..fff86983c2d80 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -157,13 +157,7 @@ func (group *Group) mergeSpilledResults(proc *process.Process) error { } if group.ctr.result1.IsEmpty() { - aggs, err := group.generateAggExec(proc) - if err != nil { - return err - } - if err = group.ctr.result1.InitWithGroupBy( - proc.Mp(), - aggexec.GetMinAggregatorsChunkSize(group.ctr.groupByEvaluate.Vec, aggs), aggs, group.ctr.groupByEvaluate.Vec, 0); err != nil { + if err := group.ctr.result1.InitWithGroupBy(proc.Mp(), 0, nil, group.ctr.groupByEvaluate.Vec, 0); err != nil { return err } } From 8d14197df7aefaf7b0d3a3a429f44980c41c2677 Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 19 Sep 2025 10:17:59 +0800 Subject: [PATCH 19/57] fix *GroupResultBuffer.IsEmpty --- pkg/sql/colexec/group/execctx.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/sql/colexec/group/execctx.go b/pkg/sql/colexec/group/execctx.go index 22fc688d8392d..01c80338aeb4e 100644 --- a/pkg/sql/colexec/group/execctx.go +++ b/pkg/sql/colexec/group/execctx.go @@ -123,7 +123,7 @@ type GroupResultBuffer struct { } func (buf *GroupResultBuffer) IsEmpty() bool { - return cap(buf.ToPopped) == 0 + return len(buf.ToPopped) == 0 } func (buf *GroupResultBuffer) InitOnlyAgg(chunkSize int, aggList []aggexec.AggFuncExec) { From 2665f2540622a043dd84b4f99fa1d1f1267b0156 Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 19 Sep 2025 10:31:39 +0800 Subject: [PATCH 20/57] fix --- pkg/sql/colexec/group/group_spill.go | 6 ------ 1 file changed, 6 deletions(-) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index fff86983c2d80..09c51cc2e4e40 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -156,12 +156,6 @@ func (group *Group) mergeSpilledResults(proc *process.Process) error { return nil } - if group.ctr.result1.IsEmpty() { - if err := group.ctr.result1.InitWithGroupBy(proc.Mp(), 0, nil, group.ctr.groupByEvaluate.Vec, 0); err != nil { - return err - } - } - for _, spillID := range group.ctr.spilledStates { spillData, err := group.SpillManager.Retrieve(spillID, proc.Mp()) if err != nil { From 32c4411a129e4275d37833b8690ff67676c2d7d1 Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 19 Sep 2025 10:32:53 +0800 Subject: [PATCH 21/57] fix chunk size in restore --- pkg/sql/colexec/group/group_spill.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index 09c51cc2e4e40..da112c5237f73 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -221,10 +221,13 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi aggs[i] = agg } + chunkSize := aggexec.GetMinAggregatorsChunkSize(spillState.GroupVectors, aggs) + aggexec.SyncAggregatorsToChunkSize(aggs, chunkSize) + group.ctr.result1.ChunkSize = chunkSize + group.ctr.result1.AggList = aggs if len(spillState.GroupVectors) > 0 && spillState.GroupCount > 0 { - chunkSize := aggexec.GetMinAggregatorsChunkSize(spillState.GroupVectors, aggs) batchesToAdd := make([]*batch.Batch, 0) for offset := 0; offset < spillState.GroupCount; offset += chunkSize { From 6ba1c34bbc22ea98eb8917e71b16e8baace71859 Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 19 Sep 2025 11:48:23 +0800 Subject: [PATCH 22/57] minor clean-ups --- pkg/sql/colexec/group/exec.go | 2 -- pkg/sql/colexec/group/group_spill.go | 26 +++++++++++++------------- 2 files changed, 13 insertions(+), 15 deletions(-) diff --git a/pkg/sql/colexec/group/exec.go b/pkg/sql/colexec/group/exec.go index 8d184cf6d46cc..1da5a35038a28 100644 --- a/pkg/sql/colexec/group/exec.go +++ b/pkg/sql/colexec/group/exec.go @@ -259,7 +259,6 @@ func (group *Group) consumeBatchToGetFinalResult( switch group.ctr.mtyp { case H0: - // without group by. if group.ctr.result1.IsEmpty() { if err := group.generateInitialResult1WithoutGroupBy(proc); err != nil { return err @@ -274,7 +273,6 @@ func (group *Group) consumeBatchToGetFinalResult( } default: - // with group by. if group.ctr.result1.IsEmpty() { err := group.ctr.hr.BuildHashTable(false, group.ctr.mtyp == HStr, group.ctr.keyNullable, group.PreAllocSize) if err != nil { diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index da112c5237f73..d5a9312908b6d 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -25,24 +25,31 @@ import ( ) func (group *Group) shouldSpill() bool { - return group.ctr.currentMemUsage > group.SpillThreshold && len(group.ctr.result1.AggList) > 0 + return group.SpillThreshold > 0 && + group.ctr.currentMemUsage > group.SpillThreshold && + len(group.ctr.result1.AggList) > 0 && + len(group.ctr.result1.ToPopped) > 0 } func (group *Group) updateMemoryUsage(proc *process.Process) { usage := int64(0) + if !group.ctr.hr.IsEmpty() && group.ctr.hr.Hash != nil { usage += int64(group.ctr.hr.Hash.Size()) } + for _, bat := range group.ctr.result1.ToPopped { if bat != nil { usage += int64(bat.Size()) } } + for _, agg := range group.ctr.result1.AggList { if agg != nil { usage += agg.Size() } } + group.ctr.currentMemUsage = usage } @@ -80,15 +87,6 @@ func (group *Group) spillPartialResults(proc *process.Process) error { groupVecs = make([]*vector.Vector, numGroupByCols) groupVecTypes = make([]types.Type, numGroupByCols) - cleanupVecs := func() { - for i := range groupVecs { - if groupVecs[i] != nil { - groupVecs[i].Free(proc.Mp()) - groupVecs[i] = nil - } - } - } - for i := 0; i < numGroupByCols; i++ { if len(group.ctr.result1.ToPopped[0].Vecs) > i && group.ctr.result1.ToPopped[0].Vecs[i] != nil { vecType := *group.ctr.result1.ToPopped[0].Vecs[i].GetType() @@ -102,7 +100,11 @@ func (group *Group) spillPartialResults(proc *process.Process) error { for i, vec := range bat.Vecs { if i < len(groupVecs) && groupVecs[i] != nil && vec != nil { if err := groupVecs[i].UnionBatch(vec, 0, vec.Length(), nil, proc.Mp()); err != nil { - cleanupVecs() + for j := range groupVecs { + if groupVecs[j] != nil { + groupVecs[j].Free(proc.Mp()) + } + } return err } } @@ -124,8 +126,6 @@ func (group *Group) spillPartialResults(proc *process.Process) error { return err } - spillData.Free(proc.Mp()) - group.ctr.spilledStates = append(group.ctr.spilledStates, spillID) for _, agg := range group.ctr.result1.AggList { From f24427f71df9b0dd269baad61890cfff87cde3ab Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 19 Sep 2025 11:53:53 +0800 Subject: [PATCH 23/57] fixes --- pkg/sql/colexec/group/exec.go | 2 +- pkg/sql/colexec/group/group_spill.go | 42 +++++++++++++--------------- 2 files changed, 21 insertions(+), 23 deletions(-) diff --git a/pkg/sql/colexec/group/exec.go b/pkg/sql/colexec/group/exec.go index 1da5a35038a28..9efcd911b6abb 100644 --- a/pkg/sql/colexec/group/exec.go +++ b/pkg/sql/colexec/group/exec.go @@ -68,7 +68,7 @@ func (group *Group) Prepare(proc *process.Process) (err error) { group.SpillManager = NewMemorySpillManager() } if group.SpillThreshold <= 0 { - group.SpillThreshold = 128 * 1024 * 1024 + group.SpillThreshold = 64 * 1024 * 1024 } return group.PrepareProjection(proc) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index d5a9312908b6d..5fb4252c47e15 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -169,11 +169,11 @@ func (group *Group) mergeSpilledResults(proc *process.Process) error { } if err = group.restoreAndMergeSpilledAggregators(proc, spillState); err != nil { - spillData.Free(proc.Mp()) + spillState.Free(proc.Mp()) return err } - spillData.Free(proc.Mp()) + spillState.Free(proc.Mp()) if err = group.SpillManager.Delete(spillID); err != nil { return err } @@ -210,11 +210,13 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi return err } - aggExpr := group.Aggs[i] - if config := aggExpr.GetExtraConfig(); config != nil { - if err = agg.SetExtraInformation(config, 0); err != nil { - agg.Free() - return err + if i < len(group.Aggs) { + aggExpr := group.Aggs[i] + if config := aggExpr.GetExtraConfig(); config != nil { + if err = agg.SetExtraInformation(config, 0); err != nil { + agg.Free() + return err + } } } @@ -237,7 +239,6 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi } bat := getInitialBatchWithSameTypeVecs(spillState.GroupVectors) - success := true for i, vec := range spillState.GroupVectors { if vec != nil && i < len(bat.Vecs) { if err := bat.Vecs[i].UnionBatch(vec, int64(offset), size, nil, proc.Mp()); err != nil { @@ -249,10 +250,8 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi } } } - if success { - bat.SetRowCount(size) - batchesToAdd = append(batchesToAdd, bat) - } + bat.SetRowCount(size) + batchesToAdd = append(batchesToAdd, bat) } group.ctr.result1.ToPopped = append(group.ctr.result1.ToPopped, batchesToAdd...) } @@ -287,11 +286,13 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi return err } - aggExpr := group.Aggs[i] - if config := aggExpr.GetExtraConfig(); config != nil { - if err = agg.SetExtraInformation(config, 0); err != nil { - agg.Free() - return err + if i < len(group.Aggs) { + aggExpr := group.Aggs[i] + if config := aggExpr.GetExtraConfig(); config != nil { + if err = agg.SetExtraInformation(config, 0); err != nil { + agg.Free() + return err + } } } @@ -337,7 +338,6 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi } bat := getInitialBatchWithSameTypeVecs(spillState.GroupVectors) - success := true for i, vec := range spillState.GroupVectors { if vec != nil && i < len(bat.Vecs) { if err := bat.Vecs[i].UnionBatch(vec, int64(offset), size, nil, proc.Mp()); err != nil { @@ -349,10 +349,8 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi } } } - if success { - bat.SetRowCount(size) - batchesToAdd = append(batchesToAdd, bat) - } + bat.SetRowCount(size) + batchesToAdd = append(batchesToAdd, bat) } group.ctr.result1.ToPopped = append(group.ctr.result1.ToPopped, batchesToAdd...) } From 1b5e83bdd88e3c7eec510352dfd0e2413f044ead Mon Sep 17 00:00:00 2001 From: reus Date: Tue, 23 Sep 2025 11:23:51 +0800 Subject: [PATCH 24/57] fix --- pkg/sql/colexec/group/group_spill.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index 5fb4252c47e15..b061daecc2bb0 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -137,7 +137,7 @@ func (group *Group) spillPartialResults(proc *process.Process) error { for _, bat := range group.ctr.result1.ToPopped { if bat != nil { - bat.CleanOnlyData() + bat.Clean(proc.Mp()) } } group.ctr.result1.ToPopped = group.ctr.result1.ToPopped[:0] From 7dd84d1f99e9a4fab4a1e42bfa768bfd87f79b68 Mon Sep 17 00:00:00 2001 From: reus Date: Tue, 23 Sep 2025 13:00:10 +0800 Subject: [PATCH 25/57] colexec/group: add lock to memory spill manager --- pkg/sql/colexec/group/spill_memory.go | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/pkg/sql/colexec/group/spill_memory.go b/pkg/sql/colexec/group/spill_memory.go index 992338e09fddd..1d6e769283510 100644 --- a/pkg/sql/colexec/group/spill_memory.go +++ b/pkg/sql/colexec/group/spill_memory.go @@ -16,6 +16,7 @@ package group import ( "fmt" + "sync" "sync/atomic" "github.com/matrixorigin/matrixone/pkg/common/mpool" @@ -25,6 +26,7 @@ type MemorySpillManager struct { data map[SpillID][]byte nextID int64 totalMem int64 + mu sync.Mutex } func NewMemorySpillManager() *MemorySpillManager { @@ -40,12 +42,17 @@ func (m *MemorySpillManager) Spill(data SpillableData) (SpillID, error) { } id := SpillID(fmt.Sprintf("spill_%d", atomic.AddInt64(&m.nextID, 1))) + m.mu.Lock() + defer m.mu.Unlock() m.data[id] = serialized atomic.AddInt64(&m.totalMem, int64(len(serialized))) return id, nil } func (m *MemorySpillManager) Retrieve(id SpillID, mp *mpool.MPool) (SpillableData, error) { + m.mu.Lock() + defer m.mu.Unlock() + serialized, exists := m.data[id] if !exists { return nil, fmt.Errorf("spill data not found: %s", id) @@ -60,6 +67,9 @@ func (m *MemorySpillManager) Retrieve(id SpillID, mp *mpool.MPool) (SpillableDat } func (m *MemorySpillManager) Delete(id SpillID) error { + m.mu.Lock() + defer m.mu.Unlock() + if serialized, exists := m.data[id]; exists { atomic.AddInt64(&m.totalMem, -int64(len(serialized))) delete(m.data, id) @@ -68,6 +78,9 @@ func (m *MemorySpillManager) Delete(id SpillID) error { } func (m *MemorySpillManager) Free() { + m.mu.Lock() + defer m.mu.Unlock() + for id := range m.data { m.Delete(id) } From 3b880d15934fa10d0054282e410f7e2cd5f33fca Mon Sep 17 00:00:00 2001 From: reus Date: Tue, 23 Sep 2025 13:01:46 +0800 Subject: [PATCH 26/57] colexec/group: fix spill trigger --- pkg/sql/colexec/group/exec.go | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/pkg/sql/colexec/group/exec.go b/pkg/sql/colexec/group/exec.go index 9efcd911b6abb..30430f5ebd351 100644 --- a/pkg/sql/colexec/group/exec.go +++ b/pkg/sql/colexec/group/exec.go @@ -229,6 +229,16 @@ func (group *Group) callToGetFinalResult(proc *process.Process) (*batch.Batch, e } group.ctr.dataSourceIsEmpty = false + + // Check if we need to spill before processing this batch + group.updateMemoryUsage(proc) + if group.shouldSpill() { + if err := group.spillPartialResults(proc); err != nil { + return nil, err + } + continue + } + if err = group.consumeBatchToGetFinalResult(proc, res); err != nil { return nil, err } @@ -327,11 +337,8 @@ func (group *Group) consumeBatchToGetFinalResult( } } + // Update memory usage after processing the batch group.updateMemoryUsage(proc) - if group.shouldSpill() { - return group.spillPartialResults(proc) - } - return nil } From 0d985f1fbe10044cf521ae3365101c61341c8e46 Mon Sep 17 00:00:00 2001 From: reus Date: Tue, 23 Sep 2025 13:41:28 +0800 Subject: [PATCH 27/57] add test for spill memory --- pkg/sql/colexec/group/spill_memory.go | 7 ++ pkg/sql/colexec/group/spill_memory_test.go | 87 ++++++++++++++++++++++ 2 files changed, 94 insertions(+) create mode 100644 pkg/sql/colexec/group/spill_memory_test.go diff --git a/pkg/sql/colexec/group/spill_memory.go b/pkg/sql/colexec/group/spill_memory.go index 1d6e769283510..3701e7e2f8ef5 100644 --- a/pkg/sql/colexec/group/spill_memory.go +++ b/pkg/sql/colexec/group/spill_memory.go @@ -86,3 +86,10 @@ func (m *MemorySpillManager) Free() { } m.data = nil } + +func (m *MemorySpillManager) TotalMem() int64 { + m.mu.Lock() + defer m.mu.Unlock() + + return atomic.LoadInt64(&m.totalMem) +} diff --git a/pkg/sql/colexec/group/spill_memory_test.go b/pkg/sql/colexec/group/spill_memory_test.go new file mode 100644 index 0000000000000..eb488b60bab49 --- /dev/null +++ b/pkg/sql/colexec/group/spill_memory_test.go @@ -0,0 +1,87 @@ +// Copyright 2025 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package group + +import ( + "testing" + + "github.com/matrixorigin/matrixone/pkg/testutil" + "github.com/stretchr/testify/require" +) + +func TestMemorySpillManager(t *testing.T) { + proc := testutil.NewProcess(t) + before := proc.Mp().CurrNB() + + manager := NewMemorySpillManager() + defer manager.Free() + + // Test spill and retrieve + data := &SpillableAggState{ + GroupCount: 10, + MarshaledAggStates: [][]byte{ + []byte("test_agg_state_1"), + []byte("test_agg_state_2"), + }, + } + + id, err := manager.Spill(data) + require.NoError(t, err) + require.NotEmpty(t, id) + + retrieved, err := manager.Retrieve(id, proc.Mp()) + require.NoError(t, err) + require.NotNil(t, retrieved) + + spillState, ok := retrieved.(*SpillableAggState) + require.True(t, ok) + require.Equal(t, int(10), spillState.GroupCount) + require.Equal(t, 2, len(spillState.MarshaledAggStates)) + + // Test delete + err = manager.Delete(id) + require.NoError(t, err) + + _, err = manager.Retrieve(id, proc.Mp()) + require.Error(t, err) + + // Test memory accounting + require.Equal(t, int64(0), manager.TotalMem()) + + // Test with larger data + largeData := &SpillableAggState{ + GroupCount: 1000, + MarshaledAggStates: make([][]byte, 100), + } + for i := range largeData.MarshaledAggStates { + largeData.MarshaledAggStates[i] = make([]byte, 1024) // 1KB per state + } + + id, err = manager.Spill(largeData) + require.NoError(t, err) + + // Verify memory usage increased + memAfterSpill := manager.TotalMem() + require.Greater(t, memAfterSpill, int64(0)) + + // Clean up + err = manager.Delete(id) + require.NoError(t, err) + + require.Equal(t, int64(0), manager.TotalMem()) + + after := proc.Mp().CurrNB() + require.Equal(t, before, after, "Memory leak detected") +} From 870e80793e478a1189e6e64ada11d6e16fd7e25b Mon Sep 17 00:00:00 2001 From: reus Date: Tue, 23 Sep 2025 13:41:40 +0800 Subject: [PATCH 28/57] fix TestSpill --- pkg/sql/colexec/group/spill_test.go | 42 +++++++++++++++++++++-------- 1 file changed, 31 insertions(+), 11 deletions(-) diff --git a/pkg/sql/colexec/group/spill_test.go b/pkg/sql/colexec/group/spill_test.go index 44840ec208a3c..4ef144c8c4d89 100644 --- a/pkg/sql/colexec/group/spill_test.go +++ b/pkg/sql/colexec/group/spill_test.go @@ -33,6 +33,9 @@ func TestSpill(t *testing.T) { getGroupTestBatch(proc.Mp(), [][2]int64{ {1, 10}, {1, 20}, {2, 30}, {2, 40}, {3, 50}, }), + getGroupTestBatch(proc.Mp(), [][2]int64{ + {1, 10}, {2, 30}, {3, 50}, {4, 60}, {5, 70}, + }), nil, } afterDataCreation := proc.Mp().CurrNB() @@ -40,10 +43,11 @@ func TestSpill(t *testing.T) { g, src := getGroupOperatorWithInputs(datas) g.NeedEval = true - g.SpillThreshold = 10 + g.SpillThreshold = 1024 // Very low threshold to trigger spill quickly g.Exprs = []*plan.Expr{newColumnExpression(0)} g.Aggs = []aggexec.AggFuncExecExpression{ aggexec.MakeAggFunctionExpression(aggexec.AggIdOfCountStar, false, []*plan.Expr{newColumnExpression(1)}, nil), + aggexec.MakeAggFunctionExpression(aggexec.AggIdOfMedian, false, []*plan.Expr{newColumnExpression(1)}, nil), } require.NoError(t, src.Prepare(proc)) @@ -55,41 +59,57 @@ func TestSpill(t *testing.T) { t.Logf("After group prepare: %d bytes allocated", afterGroupPrepare-before) require.NotNil(t, g.SpillManager) - require.Equal(t, int64(10), g.SpillThreshold) + require.Equal(t, int64(1024), g.SpillThreshold) + // Process first batch - should trigger spill r, err := g.Call(proc) require.NoError(t, err) - require.NotNil(t, r.Batch) + require.Nil(t, r.Batch) // Should be nil due to spill afterFirstCall := proc.Mp().CurrNB() t.Logf("After first call: %d bytes allocated", afterFirstCall-before) + // Process second batch - should merge spilled data and produce results + r, err = g.Call(proc) + require.NoError(t, err) + require.NotNil(t, r.Batch) + afterSecondCall := proc.Mp().CurrNB() + t.Logf("After second call: %d bytes allocated", afterSecondCall-before) + if final := r.Batch; final != nil { - require.Equal(t, 0, len(final.Aggs)) + require.Equal(t, 2, len(final.Aggs)) require.Equal(t, 2, len(final.Vecs)) groupVec := final.Vecs[0] countVec := final.Vecs[1] + sumVecs, err := final.Aggs[1].Flush() + require.NoError(t, err) + sumVec := sumVecs[0] - require.Equal(t, 3, groupVec.Length()) - require.Equal(t, 3, countVec.Length()) + require.Equal(t, 5, groupVec.Length()) + require.Equal(t, 5, countVec.Length()) + require.Equal(t, 5, sumVec.Length()) groups := vector.MustFixedColNoTypeCheck[int64](groupVec) counts := vector.MustFixedColNoTypeCheck[int64](countVec) + sums := vector.MustFixedColNoTypeCheck[int64](sumVec) - expectedGroups := []int64{1, 2, 3} - expectedCounts := []int64{2, 2, 1} + expectedGroups := []int64{1, 2, 3, 4, 5} + expectedCounts := []int64{2, 2, 2, 1, 1} + expectedSums := []int64{40, 100, 150, 60, 70} - for i := 0; i < 3; i++ { + for i := 0; i < 5; i++ { require.Equal(t, expectedGroups[i], groups[i]) require.Equal(t, expectedCounts[i], counts[i]) + require.Equal(t, expectedSums[i], sums[i]) } } + // Final call should return nil r, err = g.Call(proc) require.NoError(t, err) require.Nil(t, r.Batch) - afterSecondCall := proc.Mp().CurrNB() - t.Logf("After second call: %d bytes allocated", afterSecondCall-before) + afterThirdCall := proc.Mp().CurrNB() + t.Logf("After third call: %d bytes allocated", afterThirdCall-before) g.Free(proc, false, nil) afterGroupFree := proc.Mp().CurrNB() From 3feb7dd5e9d3f838cd688ee6c4cb8f1e45fa53fd Mon Sep 17 00:00:00 2001 From: reus Date: Tue, 23 Sep 2025 13:45:26 +0800 Subject: [PATCH 29/57] fix cleanup --- pkg/sql/colexec/group/group_spill.go | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index b061daecc2bb0..3dac06bb30813 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -63,7 +63,13 @@ func (group *Group) spillPartialResults(proc *process.Process) error { if agg != nil { marshaledData, err := aggexec.MarshalAggFuncExec(agg) if err != nil { - return fmt.Errorf("failed to marshal aggregator %d: %v", i, err) + // Clean up already marshaled states + for j := 0; j < i; j++ { + if marshaledAggStates[j] != nil { + group.SpillManager.Delete(SpillID(fmt.Sprintf("temp_%d", j))) + } + } + return err } marshaledAggStates[i] = marshaledData } @@ -105,6 +111,12 @@ func (group *Group) spillPartialResults(proc *process.Process) error { groupVecs[j].Free(proc.Mp()) } } + // Clean up already marshaled states + for _, state := range marshaledAggStates { + if state != nil { + // We can't delete the state yet as it doesn't have an ID + } + } return err } } @@ -165,7 +177,7 @@ func (group *Group) mergeSpilledResults(proc *process.Process) error { spillState, ok := spillData.(*SpillableAggState) if !ok { spillData.Free(proc.Mp()) - return fmt.Errorf("invalid spilled data type") + panic(fmt.Sprintf("invalid spilled data type")) } if err = group.restoreAndMergeSpilledAggregators(proc, spillState); err != nil { From 305de840f4b2ce018d6d895dc27d15703df74d83 Mon Sep 17 00:00:00 2001 From: reus Date: Wed, 24 Sep 2025 10:36:34 +0800 Subject: [PATCH 30/57] remove tests --- pkg/sql/colexec/group/spill_test.go | 165 ---------------------------- 1 file changed, 165 deletions(-) diff --git a/pkg/sql/colexec/group/spill_test.go b/pkg/sql/colexec/group/spill_test.go index 4ef144c8c4d89..556178defa70f 100644 --- a/pkg/sql/colexec/group/spill_test.go +++ b/pkg/sql/colexec/group/spill_test.go @@ -13,168 +13,3 @@ // limitations under the License. package group - -import ( - "testing" - - "github.com/matrixorigin/matrixone/pkg/container/batch" - "github.com/matrixorigin/matrixone/pkg/container/vector" - "github.com/matrixorigin/matrixone/pkg/pb/plan" - "github.com/matrixorigin/matrixone/pkg/sql/colexec/aggexec" - "github.com/matrixorigin/matrixone/pkg/testutil" - "github.com/stretchr/testify/require" -) - -func TestSpill(t *testing.T) { - proc := testutil.NewProcess(t) - before := proc.Mp().CurrNB() - - datas := []*batch.Batch{ - getGroupTestBatch(proc.Mp(), [][2]int64{ - {1, 10}, {1, 20}, {2, 30}, {2, 40}, {3, 50}, - }), - getGroupTestBatch(proc.Mp(), [][2]int64{ - {1, 10}, {2, 30}, {3, 50}, {4, 60}, {5, 70}, - }), - nil, - } - afterDataCreation := proc.Mp().CurrNB() - t.Logf("After data creation: %d bytes allocated", afterDataCreation-before) - - g, src := getGroupOperatorWithInputs(datas) - g.NeedEval = true - g.SpillThreshold = 1024 // Very low threshold to trigger spill quickly - g.Exprs = []*plan.Expr{newColumnExpression(0)} - g.Aggs = []aggexec.AggFuncExecExpression{ - aggexec.MakeAggFunctionExpression(aggexec.AggIdOfCountStar, false, []*plan.Expr{newColumnExpression(1)}, nil), - aggexec.MakeAggFunctionExpression(aggexec.AggIdOfMedian, false, []*plan.Expr{newColumnExpression(1)}, nil), - } - - require.NoError(t, src.Prepare(proc)) - afterSrcPrepare := proc.Mp().CurrNB() - t.Logf("After src prepare: %d bytes allocated", afterSrcPrepare-before) - - require.NoError(t, g.Prepare(proc)) - afterGroupPrepare := proc.Mp().CurrNB() - t.Logf("After group prepare: %d bytes allocated", afterGroupPrepare-before) - - require.NotNil(t, g.SpillManager) - require.Equal(t, int64(1024), g.SpillThreshold) - - // Process first batch - should trigger spill - r, err := g.Call(proc) - require.NoError(t, err) - require.Nil(t, r.Batch) // Should be nil due to spill - afterFirstCall := proc.Mp().CurrNB() - t.Logf("After first call: %d bytes allocated", afterFirstCall-before) - - // Process second batch - should merge spilled data and produce results - r, err = g.Call(proc) - require.NoError(t, err) - require.NotNil(t, r.Batch) - afterSecondCall := proc.Mp().CurrNB() - t.Logf("After second call: %d bytes allocated", afterSecondCall-before) - - if final := r.Batch; final != nil { - require.Equal(t, 2, len(final.Aggs)) - require.Equal(t, 2, len(final.Vecs)) - - groupVec := final.Vecs[0] - countVec := final.Vecs[1] - sumVecs, err := final.Aggs[1].Flush() - require.NoError(t, err) - sumVec := sumVecs[0] - - require.Equal(t, 5, groupVec.Length()) - require.Equal(t, 5, countVec.Length()) - require.Equal(t, 5, sumVec.Length()) - - groups := vector.MustFixedColNoTypeCheck[int64](groupVec) - counts := vector.MustFixedColNoTypeCheck[int64](countVec) - sums := vector.MustFixedColNoTypeCheck[int64](sumVec) - - expectedGroups := []int64{1, 2, 3, 4, 5} - expectedCounts := []int64{2, 2, 2, 1, 1} - expectedSums := []int64{40, 100, 150, 60, 70} - - for i := 0; i < 5; i++ { - require.Equal(t, expectedGroups[i], groups[i]) - require.Equal(t, expectedCounts[i], counts[i]) - require.Equal(t, expectedSums[i], sums[i]) - } - } - - // Final call should return nil - r, err = g.Call(proc) - require.NoError(t, err) - require.Nil(t, r.Batch) - afterThirdCall := proc.Mp().CurrNB() - t.Logf("After third call: %d bytes allocated", afterThirdCall-before) - - g.Free(proc, false, nil) - afterGroupFree := proc.Mp().CurrNB() - t.Logf("After group free: %d bytes allocated", afterGroupFree-before) - - src.Free(proc, false, nil) - afterSrcFree := proc.Mp().CurrNB() - t.Logf("After src free: %d bytes allocated", afterSrcFree-before) - - require.Equal(t, before, proc.Mp().CurrNB()) -} - -func TestSpillMultipleCycles(t *testing.T) { - proc := testutil.NewProcess(t) - before := proc.Mp().CurrNB() - - datas := []*batch.Batch{ - getGroupTestBatch(proc.Mp(), [][2]int64{ - {1, 1}, {2, 2}, {3, 3}, - }), - getGroupTestBatch(proc.Mp(), [][2]int64{ - {4, 4}, {5, 5}, {6, 6}, - }), - getGroupTestBatch(proc.Mp(), [][2]int64{ - {1, 7}, {2, 8}, {3, 9}, - }), - nil, - } - - g, src := getGroupOperatorWithInputs(datas) - g.NeedEval = true - g.SpillThreshold = 10 - g.Exprs = []*plan.Expr{newColumnExpression(0)} - g.Aggs = []aggexec.AggFuncExecExpression{ - aggexec.MakeAggFunctionExpression(aggexec.AggIdOfCountStar, false, []*plan.Expr{newColumnExpression(1)}, nil), - } - - require.NoError(t, src.Prepare(proc)) - require.NoError(t, g.Prepare(proc)) - - r, err := g.Call(proc) - require.NoError(t, err) - require.NotNil(t, r.Batch) - - if final := r.Batch; final != nil { - require.Equal(t, 2, len(final.Vecs)) - require.Equal(t, 6, final.Vecs[0].Length()) - - groups := vector.MustFixedColNoTypeCheck[int64](final.Vecs[0]) - counts := vector.MustFixedColNoTypeCheck[int64](final.Vecs[1]) - - groupCounts := make(map[int64]int64) - for i := 0; i < len(groups); i++ { - groupCounts[groups[i]] = counts[i] - } - - require.Equal(t, int64(2), groupCounts[1]) - require.Equal(t, int64(2), groupCounts[2]) - require.Equal(t, int64(2), groupCounts[3]) - require.Equal(t, int64(1), groupCounts[4]) - require.Equal(t, int64(1), groupCounts[5]) - require.Equal(t, int64(1), groupCounts[6]) - } - - g.Free(proc, false, nil) - src.Free(proc, false, nil) - require.Equal(t, before, proc.Mp().CurrNB()) -} From fcc5c229c2b4e8823927b7a2389ec91fd3922dad Mon Sep 17 00:00:00 2001 From: reus Date: Wed, 24 Sep 2025 14:12:35 +0800 Subject: [PATCH 31/57] fixes --- pkg/sql/colexec/group/group_spill.go | 21 ++++++++------------- 1 file changed, 8 insertions(+), 13 deletions(-) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index 3dac06bb30813..520e2e075896e 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -26,6 +26,7 @@ import ( func (group *Group) shouldSpill() bool { return group.SpillThreshold > 0 && + group.SpillManager != nil && group.ctr.currentMemUsage > group.SpillThreshold && len(group.ctr.result1.AggList) > 0 && len(group.ctr.result1.ToPopped) > 0 @@ -63,12 +64,6 @@ func (group *Group) spillPartialResults(proc *process.Process) error { if agg != nil { marshaledData, err := aggexec.MarshalAggFuncExec(agg) if err != nil { - // Clean up already marshaled states - for j := 0; j < i; j++ { - if marshaledAggStates[j] != nil { - group.SpillManager.Delete(SpillID(fmt.Sprintf("temp_%d", j))) - } - } return err } marshaledAggStates[i] = marshaledData @@ -83,6 +78,13 @@ func (group *Group) spillPartialResults(proc *process.Process) error { } if totalGroups == 0 { + for _, agg := range group.ctr.result1.AggList { + if agg != nil { + agg.Free() + } + } + group.ctr.result1.AggList = nil + group.ctr.result1.ToPopped = group.ctr.result1.ToPopped[:0] return nil } @@ -111,12 +113,6 @@ func (group *Group) spillPartialResults(proc *process.Process) error { groupVecs[j].Free(proc.Mp()) } } - // Clean up already marshaled states - for _, state := range marshaledAggStates { - if state != nil { - // We can't delete the state yet as it doesn't have an ID - } - } return err } } @@ -238,7 +234,6 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi chunkSize := aggexec.GetMinAggregatorsChunkSize(spillState.GroupVectors, aggs) aggexec.SyncAggregatorsToChunkSize(aggs, chunkSize) group.ctr.result1.ChunkSize = chunkSize - group.ctr.result1.AggList = aggs if len(spillState.GroupVectors) > 0 && spillState.GroupCount > 0 { From 5fd108cbc70f1c6487d393b0d319e08362ab56e3 Mon Sep 17 00:00:00 2001 From: reus Date: Wed, 24 Sep 2025 15:06:59 +0800 Subject: [PATCH 32/57] add testspill package --- pkg/sql/colexec/group/testspill/spill_test.go | 101 ++++++++++++++++++ 1 file changed, 101 insertions(+) create mode 100644 pkg/sql/colexec/group/testspill/spill_test.go diff --git a/pkg/sql/colexec/group/testspill/spill_test.go b/pkg/sql/colexec/group/testspill/spill_test.go new file mode 100644 index 0000000000000..9ae0e3f0dc306 --- /dev/null +++ b/pkg/sql/colexec/group/testspill/spill_test.go @@ -0,0 +1,101 @@ +// Copyright 2025 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package testspill + +import ( + "database/sql" + "fmt" + "testing" + + "github.com/matrixorigin/matrixone/pkg/embed" + "github.com/stretchr/testify/require" +) + +func TestSpill(t *testing.T) { + + // start cluster + cluster, err := embed.NewCluster( + embed.WithCNCount(3), + embed.WithTesting(), + embed.WithPreStart(func(service embed.ServiceOperator) { + }), + ) + require.NoError(t, err) + err = cluster.Start() + require.NoError(t, err) + defer cluster.Close() + + cn0, err := cluster.GetCNService(0) + require.NoError(t, err) + dsn := fmt.Sprintf("dump:111@tcp(127.0.0.1:%d)/", + cn0.GetServiceConfig().CN.Frontend.Port, + ) + + db, err := sql.Open("mysql", dsn) + require.NoError(t, err) + defer db.Close() + + // database + _, err = db.Exec(` create database test `) + require.NoError(t, err) + + // table + _, err = db.Exec(` + use test; + create table sales ( + id int, + product_id int, + customer_id int, + sale_date date, + amount decimal(10, 2) + ); + `) + require.NoError(t, err) + + // data + _, err = db.Exec(` + use test; + insert into sales (id, product_id, customer_id, sale_date, amount) + select g.result as id, + floor(1 + (rand() * 10000)) as product_id, + floor(1 + (rand() * 10000)) as customer_id, + current_date - interval floor(rand() * 365) day as sale_date, + floor(rand() * 1000) as amount + from generate_series(2000 * 10000) g + `, + ) + require.NoError(t, err) + + // query, large number of groups + rows, err := db.Query(` + select + product_id, + customer_id, + sale_date, + sum(amount) as total_sales + from sales + group by product_id, customer_id, sale_date + order by total_sales desc; + `, + ) + require.NoError(t, err) + n := 0 + for rows.Next() { + n++ + } + require.NoError(t, rows.Close()) + t.Logf("results: %d", n) + +} From 86ae60e1ec157149ee0ae82ca5087e29e6ba32f1 Mon Sep 17 00:00:00 2001 From: reus Date: Wed, 24 Sep 2025 15:12:42 +0800 Subject: [PATCH 33/57] add testspill package --- pkg/sql/colexec/group/testspill/spill_test.go | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/pkg/sql/colexec/group/testspill/spill_test.go b/pkg/sql/colexec/group/testspill/spill_test.go index 9ae0e3f0dc306..d9fb26049b4ba 100644 --- a/pkg/sql/colexec/group/testspill/spill_test.go +++ b/pkg/sql/colexec/group/testspill/spill_test.go @@ -78,16 +78,13 @@ func TestSpill(t *testing.T) { ) require.NoError(t, err) - // query, large number of groups + // query rows, err := db.Query(` select - product_id, - customer_id, - sale_date, - sum(amount) as total_sales + count(sha2(product_id, 256)), + count(sha2(customer_id, 256)), + count(sha2(sale_date, 256)) from sales - group by product_id, customer_id, sale_date - order by total_sales desc; `, ) require.NoError(t, err) From 283ec5006f7119ad0ed5772bf1207fc5b73828dd Mon Sep 17 00:00:00 2001 From: reus Date: Wed, 24 Sep 2025 15:18:50 +0800 Subject: [PATCH 34/57] do not use sonic --- go.mod | 2 +- go.sum | 4 ++-- pkg/sql/colexec/table_function/fulltext.go | 4 ++-- pkg/sql/colexec/table_function/fulltext_tokenize.go | 3 +-- pkg/sql/colexec/table_function/hnsw_create.go | 6 +++--- pkg/sql/colexec/table_function/hnsw_search.go | 6 +++--- pkg/sql/colexec/table_function/ivf_create.go | 6 +++--- pkg/sql/colexec/table_function/ivf_search.go | 6 +++--- 8 files changed, 18 insertions(+), 19 deletions(-) diff --git a/go.mod b/go.mod index e3998176df58a..af21778b50492 100644 --- a/go.mod +++ b/go.mod @@ -22,7 +22,7 @@ require ( github.com/aws/smithy-go v1.22.1 github.com/axiomhq/hyperloglog v0.0.0-20230201085229-3ddf4bad03dc github.com/buger/jsonparser v1.1.1 - github.com/bytedance/sonic v1.14.1 + github.com/bytedance/sonic v1.14.2-0.20250916073852-d55adc49e6c8 github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5 github.com/cespare/xxhash v1.1.0 github.com/cespare/xxhash/v2 v2.3.0 diff --git a/go.sum b/go.sum index e62c9f1394df6..387028401e18f 100644 --- a/go.sum +++ b/go.sum @@ -126,8 +126,8 @@ github.com/buger/jsonparser v1.1.1 h1:2PnMjfWD7wBILjqQbt530v576A/cAbQvEW9gGIpYMU github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0= github.com/bytedance/gopkg v0.1.3 h1:TPBSwH8RsouGCBcMBktLt1AymVo2TVsBVCY4b6TnZ/M= github.com/bytedance/gopkg v0.1.3/go.mod h1:576VvJ+eJgyCzdjS+c4+77QF3p7ubbtiKARP3TxducM= -github.com/bytedance/sonic v1.14.1 h1:FBMC0zVz5XUmE4z9wF4Jey0An5FueFvOsTKKKtwIl7w= -github.com/bytedance/sonic v1.14.1/go.mod h1:gi6uhQLMbTdeP0muCnrjHLeCUPyb70ujhnNlhOylAFc= +github.com/bytedance/sonic v1.14.2-0.20250916073852-d55adc49e6c8 h1:ML31tbBAA8S+CQQSytk/uDVBeEn8/bjYydbCdEsjPcQ= +github.com/bytedance/sonic v1.14.2-0.20250916073852-d55adc49e6c8/go.mod h1:gi6uhQLMbTdeP0muCnrjHLeCUPyb70ujhnNlhOylAFc= github.com/bytedance/sonic/loader v0.3.0 h1:dskwH8edlzNMctoruo8FPTJDF3vLtDT0sXZwvZJyqeA= github.com/bytedance/sonic/loader v0.3.0/go.mod h1:N8A3vUdtUebEY2/VQC0MyhYeKUFosQU6FxH2JmUe6VI= github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5 h1:BjkPE3785EwPhhyuFkbINB+2a1xATwk8SNDWnJiD41g= diff --git a/pkg/sql/colexec/table_function/fulltext.go b/pkg/sql/colexec/table_function/fulltext.go index b4cf91e02052c..738ab5c805716 100644 --- a/pkg/sql/colexec/table_function/fulltext.go +++ b/pkg/sql/colexec/table_function/fulltext.go @@ -16,10 +16,10 @@ package table_function import ( "context" + "encoding/json" "fmt" "sync" - "github.com/bytedance/sonic" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" @@ -163,7 +163,7 @@ func (u *fulltextState) start(tf *TableFunction, proc *process.Process, nthRow i if !u.inited { if len(tf.Params) > 0 { - err := sonic.Unmarshal([]byte(tf.Params), &u.param) + err := json.Unmarshal([]byte(tf.Params), &u.param) if err != nil { return err } diff --git a/pkg/sql/colexec/table_function/fulltext_tokenize.go b/pkg/sql/colexec/table_function/fulltext_tokenize.go index ef61d630a6a2b..2c65bab6ac7b3 100644 --- a/pkg/sql/colexec/table_function/fulltext_tokenize.go +++ b/pkg/sql/colexec/table_function/fulltext_tokenize.go @@ -17,7 +17,6 @@ package table_function import ( "encoding/json" - "github.com/bytedance/sonic" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/bytejson" @@ -110,7 +109,7 @@ func (u *tokenizeState) start(tf *TableFunction, proc *process.Process, nthRow i if !u.inited { if len(tf.Params) > 0 { - err := sonic.Unmarshal([]byte(tf.Params), &u.param) + err := json.Unmarshal([]byte(tf.Params), &u.param) if err != nil { return err } diff --git a/pkg/sql/colexec/table_function/hnsw_create.go b/pkg/sql/colexec/table_function/hnsw_create.go index 38aedbbdf2f50..b2ad323b15379 100644 --- a/pkg/sql/colexec/table_function/hnsw_create.go +++ b/pkg/sql/colexec/table_function/hnsw_create.go @@ -15,11 +15,11 @@ package table_function import ( + "encoding/json" "fmt" "strconv" "time" - "github.com/bytedance/sonic" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" @@ -135,7 +135,7 @@ func (u *hnswCreateState) start(tf *TableFunction, proc *process.Process, nthRow if !u.inited { if len(tf.Params) > 0 { - err = sonic.Unmarshal([]byte(tf.Params), &u.param) + err = json.Unmarshal([]byte(tf.Params), &u.param) if err != nil { return err } @@ -185,7 +185,7 @@ func (u *hnswCreateState) start(tf *TableFunction, proc *process.Process, nthRow if len(cfgstr) == 0 { return moerr.NewInternalError(proc.Ctx, "IndexTableConfig is empty") } - err = sonic.Unmarshal([]byte(cfgstr), &u.tblcfg) + err = json.Unmarshal([]byte(cfgstr), &u.tblcfg) if err != nil { return err } diff --git a/pkg/sql/colexec/table_function/hnsw_search.go b/pkg/sql/colexec/table_function/hnsw_search.go index 2c53c0f845a45..e2e675eeb6568 100644 --- a/pkg/sql/colexec/table_function/hnsw_search.go +++ b/pkg/sql/colexec/table_function/hnsw_search.go @@ -15,10 +15,10 @@ package table_function import ( + "encoding/json" "fmt" "strconv" - "github.com/bytedance/sonic" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" @@ -129,7 +129,7 @@ func (u *hnswSearchState) start(tf *TableFunction, proc *process.Process, nthRow if !u.inited { if len(tf.Params) > 0 { - err = sonic.Unmarshal([]byte(tf.Params), &u.param) + err = json.Unmarshal([]byte(tf.Params), &u.param) if err != nil { return err } @@ -179,7 +179,7 @@ func (u *hnswSearchState) start(tf *TableFunction, proc *process.Process, nthRow if len(cfgstr) == 0 { return moerr.NewInternalError(proc.Ctx, "IndexTableConfig is empty") } - err := sonic.Unmarshal([]byte(cfgstr), &u.tblcfg) + err := json.Unmarshal([]byte(cfgstr), &u.tblcfg) if err != nil { return err } diff --git a/pkg/sql/colexec/table_function/ivf_create.go b/pkg/sql/colexec/table_function/ivf_create.go index 3f761fbc1ea89..8194e38e15e1b 100644 --- a/pkg/sql/colexec/table_function/ivf_create.go +++ b/pkg/sql/colexec/table_function/ivf_create.go @@ -15,12 +15,12 @@ package table_function import ( + "encoding/json" "fmt" "strconv" "strings" "time" - "github.com/bytedance/sonic" "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/batch" @@ -194,7 +194,7 @@ func (u *ivfCreateState) start(tf *TableFunction, proc *process.Process, nthRow if !u.inited { if len(tf.Params) > 0 { - err = sonic.Unmarshal([]byte(tf.Params), &u.param) + err = json.Unmarshal([]byte(tf.Params), &u.param) if err != nil { return err } @@ -233,7 +233,7 @@ func (u *ivfCreateState) start(tf *TableFunction, proc *process.Process, nthRow if len(cfgstr) == 0 { return moerr.NewInternalError(proc.Ctx, "IndexTableConfig is empty") } - err := sonic.Unmarshal([]byte(cfgstr), &u.tblcfg) + err := json.Unmarshal([]byte(cfgstr), &u.tblcfg) if err != nil { return err } diff --git a/pkg/sql/colexec/table_function/ivf_search.go b/pkg/sql/colexec/table_function/ivf_search.go index e06ea3ebb5c59..bf1ac4a4fdf7f 100644 --- a/pkg/sql/colexec/table_function/ivf_search.go +++ b/pkg/sql/colexec/table_function/ivf_search.go @@ -15,10 +15,10 @@ package table_function import ( + "encoding/json" "fmt" "strconv" - "github.com/bytedance/sonic" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" @@ -137,7 +137,7 @@ func (u *ivfSearchState) start(tf *TableFunction, proc *process.Process, nthRow if !u.inited { if len(tf.Params) > 0 { - err = sonic.Unmarshal([]byte(tf.Params), &u.param) + err = json.Unmarshal([]byte(tf.Params), &u.param) if err != nil { return err } @@ -172,7 +172,7 @@ func (u *ivfSearchState) start(tf *TableFunction, proc *process.Process, nthRow if len(cfgstr) == 0 { return moerr.NewInternalError(proc.Ctx, "IndexTableConfig is empty") } - err := sonic.Unmarshal([]byte(cfgstr), &u.tblcfg) + err := json.Unmarshal([]byte(cfgstr), &u.tblcfg) if err != nil { return err } From 59221dbeb7943aedb40857c87cb0783878d3fd91 Mon Sep 17 00:00:00 2001 From: reus Date: Wed, 24 Sep 2025 15:21:24 +0800 Subject: [PATCH 35/57] Revert "do not use sonic" This reverts commit 54939fd7966e74cbb9612b27a12af883cb58af58. --- go.mod | 2 +- go.sum | 4 ++-- pkg/sql/colexec/table_function/fulltext.go | 4 ++-- pkg/sql/colexec/table_function/fulltext_tokenize.go | 3 ++- pkg/sql/colexec/table_function/hnsw_create.go | 6 +++--- pkg/sql/colexec/table_function/hnsw_search.go | 6 +++--- pkg/sql/colexec/table_function/ivf_create.go | 6 +++--- pkg/sql/colexec/table_function/ivf_search.go | 6 +++--- 8 files changed, 19 insertions(+), 18 deletions(-) diff --git a/go.mod b/go.mod index af21778b50492..e3998176df58a 100644 --- a/go.mod +++ b/go.mod @@ -22,7 +22,7 @@ require ( github.com/aws/smithy-go v1.22.1 github.com/axiomhq/hyperloglog v0.0.0-20230201085229-3ddf4bad03dc github.com/buger/jsonparser v1.1.1 - github.com/bytedance/sonic v1.14.2-0.20250916073852-d55adc49e6c8 + github.com/bytedance/sonic v1.14.1 github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5 github.com/cespare/xxhash v1.1.0 github.com/cespare/xxhash/v2 v2.3.0 diff --git a/go.sum b/go.sum index 387028401e18f..e62c9f1394df6 100644 --- a/go.sum +++ b/go.sum @@ -126,8 +126,8 @@ github.com/buger/jsonparser v1.1.1 h1:2PnMjfWD7wBILjqQbt530v576A/cAbQvEW9gGIpYMU github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0= github.com/bytedance/gopkg v0.1.3 h1:TPBSwH8RsouGCBcMBktLt1AymVo2TVsBVCY4b6TnZ/M= github.com/bytedance/gopkg v0.1.3/go.mod h1:576VvJ+eJgyCzdjS+c4+77QF3p7ubbtiKARP3TxducM= -github.com/bytedance/sonic v1.14.2-0.20250916073852-d55adc49e6c8 h1:ML31tbBAA8S+CQQSytk/uDVBeEn8/bjYydbCdEsjPcQ= -github.com/bytedance/sonic v1.14.2-0.20250916073852-d55adc49e6c8/go.mod h1:gi6uhQLMbTdeP0muCnrjHLeCUPyb70ujhnNlhOylAFc= +github.com/bytedance/sonic v1.14.1 h1:FBMC0zVz5XUmE4z9wF4Jey0An5FueFvOsTKKKtwIl7w= +github.com/bytedance/sonic v1.14.1/go.mod h1:gi6uhQLMbTdeP0muCnrjHLeCUPyb70ujhnNlhOylAFc= github.com/bytedance/sonic/loader v0.3.0 h1:dskwH8edlzNMctoruo8FPTJDF3vLtDT0sXZwvZJyqeA= github.com/bytedance/sonic/loader v0.3.0/go.mod h1:N8A3vUdtUebEY2/VQC0MyhYeKUFosQU6FxH2JmUe6VI= github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5 h1:BjkPE3785EwPhhyuFkbINB+2a1xATwk8SNDWnJiD41g= diff --git a/pkg/sql/colexec/table_function/fulltext.go b/pkg/sql/colexec/table_function/fulltext.go index 738ab5c805716..b4cf91e02052c 100644 --- a/pkg/sql/colexec/table_function/fulltext.go +++ b/pkg/sql/colexec/table_function/fulltext.go @@ -16,10 +16,10 @@ package table_function import ( "context" - "encoding/json" "fmt" "sync" + "github.com/bytedance/sonic" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" @@ -163,7 +163,7 @@ func (u *fulltextState) start(tf *TableFunction, proc *process.Process, nthRow i if !u.inited { if len(tf.Params) > 0 { - err := json.Unmarshal([]byte(tf.Params), &u.param) + err := sonic.Unmarshal([]byte(tf.Params), &u.param) if err != nil { return err } diff --git a/pkg/sql/colexec/table_function/fulltext_tokenize.go b/pkg/sql/colexec/table_function/fulltext_tokenize.go index 2c65bab6ac7b3..ef61d630a6a2b 100644 --- a/pkg/sql/colexec/table_function/fulltext_tokenize.go +++ b/pkg/sql/colexec/table_function/fulltext_tokenize.go @@ -17,6 +17,7 @@ package table_function import ( "encoding/json" + "github.com/bytedance/sonic" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/bytejson" @@ -109,7 +110,7 @@ func (u *tokenizeState) start(tf *TableFunction, proc *process.Process, nthRow i if !u.inited { if len(tf.Params) > 0 { - err := json.Unmarshal([]byte(tf.Params), &u.param) + err := sonic.Unmarshal([]byte(tf.Params), &u.param) if err != nil { return err } diff --git a/pkg/sql/colexec/table_function/hnsw_create.go b/pkg/sql/colexec/table_function/hnsw_create.go index b2ad323b15379..38aedbbdf2f50 100644 --- a/pkg/sql/colexec/table_function/hnsw_create.go +++ b/pkg/sql/colexec/table_function/hnsw_create.go @@ -15,11 +15,11 @@ package table_function import ( - "encoding/json" "fmt" "strconv" "time" + "github.com/bytedance/sonic" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" @@ -135,7 +135,7 @@ func (u *hnswCreateState) start(tf *TableFunction, proc *process.Process, nthRow if !u.inited { if len(tf.Params) > 0 { - err = json.Unmarshal([]byte(tf.Params), &u.param) + err = sonic.Unmarshal([]byte(tf.Params), &u.param) if err != nil { return err } @@ -185,7 +185,7 @@ func (u *hnswCreateState) start(tf *TableFunction, proc *process.Process, nthRow if len(cfgstr) == 0 { return moerr.NewInternalError(proc.Ctx, "IndexTableConfig is empty") } - err = json.Unmarshal([]byte(cfgstr), &u.tblcfg) + err = sonic.Unmarshal([]byte(cfgstr), &u.tblcfg) if err != nil { return err } diff --git a/pkg/sql/colexec/table_function/hnsw_search.go b/pkg/sql/colexec/table_function/hnsw_search.go index e2e675eeb6568..2c53c0f845a45 100644 --- a/pkg/sql/colexec/table_function/hnsw_search.go +++ b/pkg/sql/colexec/table_function/hnsw_search.go @@ -15,10 +15,10 @@ package table_function import ( - "encoding/json" "fmt" "strconv" + "github.com/bytedance/sonic" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" @@ -129,7 +129,7 @@ func (u *hnswSearchState) start(tf *TableFunction, proc *process.Process, nthRow if !u.inited { if len(tf.Params) > 0 { - err = json.Unmarshal([]byte(tf.Params), &u.param) + err = sonic.Unmarshal([]byte(tf.Params), &u.param) if err != nil { return err } @@ -179,7 +179,7 @@ func (u *hnswSearchState) start(tf *TableFunction, proc *process.Process, nthRow if len(cfgstr) == 0 { return moerr.NewInternalError(proc.Ctx, "IndexTableConfig is empty") } - err := json.Unmarshal([]byte(cfgstr), &u.tblcfg) + err := sonic.Unmarshal([]byte(cfgstr), &u.tblcfg) if err != nil { return err } diff --git a/pkg/sql/colexec/table_function/ivf_create.go b/pkg/sql/colexec/table_function/ivf_create.go index 8194e38e15e1b..3f761fbc1ea89 100644 --- a/pkg/sql/colexec/table_function/ivf_create.go +++ b/pkg/sql/colexec/table_function/ivf_create.go @@ -15,12 +15,12 @@ package table_function import ( - "encoding/json" "fmt" "strconv" "strings" "time" + "github.com/bytedance/sonic" "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/batch" @@ -194,7 +194,7 @@ func (u *ivfCreateState) start(tf *TableFunction, proc *process.Process, nthRow if !u.inited { if len(tf.Params) > 0 { - err = json.Unmarshal([]byte(tf.Params), &u.param) + err = sonic.Unmarshal([]byte(tf.Params), &u.param) if err != nil { return err } @@ -233,7 +233,7 @@ func (u *ivfCreateState) start(tf *TableFunction, proc *process.Process, nthRow if len(cfgstr) == 0 { return moerr.NewInternalError(proc.Ctx, "IndexTableConfig is empty") } - err := json.Unmarshal([]byte(cfgstr), &u.tblcfg) + err := sonic.Unmarshal([]byte(cfgstr), &u.tblcfg) if err != nil { return err } diff --git a/pkg/sql/colexec/table_function/ivf_search.go b/pkg/sql/colexec/table_function/ivf_search.go index bf1ac4a4fdf7f..e06ea3ebb5c59 100644 --- a/pkg/sql/colexec/table_function/ivf_search.go +++ b/pkg/sql/colexec/table_function/ivf_search.go @@ -15,10 +15,10 @@ package table_function import ( - "encoding/json" "fmt" "strconv" + "github.com/bytedance/sonic" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" @@ -137,7 +137,7 @@ func (u *ivfSearchState) start(tf *TableFunction, proc *process.Process, nthRow if !u.inited { if len(tf.Params) > 0 { - err = json.Unmarshal([]byte(tf.Params), &u.param) + err = sonic.Unmarshal([]byte(tf.Params), &u.param) if err != nil { return err } @@ -172,7 +172,7 @@ func (u *ivfSearchState) start(tf *TableFunction, proc *process.Process, nthRow if len(cfgstr) == 0 { return moerr.NewInternalError(proc.Ctx, "IndexTableConfig is empty") } - err := json.Unmarshal([]byte(cfgstr), &u.tblcfg) + err := sonic.Unmarshal([]byte(cfgstr), &u.tblcfg) if err != nil { return err } From 3f015ab207675fcc5cb82aa879fa15ddddede8e1 Mon Sep 17 00:00:00 2001 From: reus Date: Thu, 25 Sep 2025 23:41:12 +0800 Subject: [PATCH 36/57] update --- pkg/sql/colexec/group/testspill/spill_test.go | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/pkg/sql/colexec/group/testspill/spill_test.go b/pkg/sql/colexec/group/testspill/spill_test.go index d9fb26049b4ba..89d0bd03617cd 100644 --- a/pkg/sql/colexec/group/testspill/spill_test.go +++ b/pkg/sql/colexec/group/testspill/spill_test.go @@ -79,20 +79,16 @@ func TestSpill(t *testing.T) { require.NoError(t, err) // query - rows, err := db.Query(` + var a, b, c int + err = db.QueryRow(` select count(sha2(product_id, 256)), count(sha2(customer_id, 256)), count(sha2(sale_date, 256)) from sales `, - ) + ).Scan(&a, &b, &c) require.NoError(t, err) - n := 0 - for rows.Next() { - n++ - } - require.NoError(t, rows.Close()) - t.Logf("results: %d", n) + t.Logf("results: %v %v %v", a, b, c) } From 306f1c55c323191a90ccb54d3ac3d182ab3d883c Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 26 Sep 2025 00:08:54 +0800 Subject: [PATCH 37/57] testspill: more groups --- pkg/sql/colexec/group/testspill/spill_test.go | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/pkg/sql/colexec/group/testspill/spill_test.go b/pkg/sql/colexec/group/testspill/spill_test.go index 89d0bd03617cd..3e81229aba3c5 100644 --- a/pkg/sql/colexec/group/testspill/spill_test.go +++ b/pkg/sql/colexec/group/testspill/spill_test.go @@ -69,8 +69,8 @@ func TestSpill(t *testing.T) { use test; insert into sales (id, product_id, customer_id, sale_date, amount) select g.result as id, - floor(1 + (rand() * 10000)) as product_id, - floor(1 + (rand() * 10000)) as customer_id, + floor(1 + (rand() * 100000)) as product_id, + floor(1 + (rand() * 100000)) as customer_id, current_date - interval floor(rand() * 365) day as sale_date, floor(rand() * 1000) as amount from generate_series(2000 * 10000) g @@ -82,9 +82,7 @@ func TestSpill(t *testing.T) { var a, b, c int err = db.QueryRow(` select - count(sha2(product_id, 256)), - count(sha2(customer_id, 256)), - count(sha2(sale_date, 256)) + count(sha2(product_id * customer_id, 256)), from sales `, ).Scan(&a, &b, &c) From e89b3c33e9f719cae4d92f50010dbf320ef32e93 Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 26 Sep 2025 00:09:03 +0800 Subject: [PATCH 38/57] fix --- pkg/sql/colexec/group/exec.go | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/pkg/sql/colexec/group/exec.go b/pkg/sql/colexec/group/exec.go index 30430f5ebd351..d2ef8a9a697a0 100644 --- a/pkg/sql/colexec/group/exec.go +++ b/pkg/sql/colexec/group/exec.go @@ -339,6 +339,14 @@ func (group *Group) consumeBatchToGetFinalResult( // Update memory usage after processing the batch group.updateMemoryUsage(proc) + + // Check if we need to spill after processing this batch + if group.shouldSpill() { + if err := group.spillPartialResults(proc); err != nil { + return err + } + } + return nil } From 82bf08816d70cd4501f6f409d86eca8529dfa8b2 Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 26 Sep 2025 00:09:39 +0800 Subject: [PATCH 39/57] fix error handling --- pkg/sql/colexec/group/exec.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/sql/colexec/group/exec.go b/pkg/sql/colexec/group/exec.go index d2ef8a9a697a0..48d704b737fee 100644 --- a/pkg/sql/colexec/group/exec.go +++ b/pkg/sql/colexec/group/exec.go @@ -217,7 +217,7 @@ func (group *Group) callToGetFinalResult(proc *process.Process) (*batch.Batch, e group.ctr.state = vm.Eval if group.ctr.isDataSourceEmpty() && len(group.Exprs) == 0 { - if err = group.generateInitialResult1WithoutGroupBy(proc); err != nil { + if err := group.generateInitialResult1WithoutGroupBy(proc); err != nil { return nil, err } group.ctr.result1.ToPopped[0].SetRowCount(1) @@ -339,14 +339,14 @@ func (group *Group) consumeBatchToGetFinalResult( // Update memory usage after processing the batch group.updateMemoryUsage(proc) - + // Check if we need to spill after processing this batch if group.shouldSpill() { if err := group.spillPartialResults(proc); err != nil { return err } } - + return nil } From 076d8685588ef10ee948f47ccd7410996a2e6b34 Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 26 Sep 2025 16:28:25 +0800 Subject: [PATCH 40/57] add logs for spill --- pkg/sql/colexec/group/group_spill.go | 36 +++++++++++++++++--- pkg/sql/colexec/group/spill_memory.go | 30 ++++++++++++++-- pkg/sql/colexec/group/spillable_agg_state.go | 12 ++++++- 3 files changed, 70 insertions(+), 8 deletions(-) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index 520e2e075896e..e35af446075f1 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -20,16 +20,25 @@ import ( "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/container/vector" + "github.com/matrixorigin/matrixone/pkg/logutil" "github.com/matrixorigin/matrixone/pkg/sql/colexec/aggexec" "github.com/matrixorigin/matrixone/pkg/vm/process" + "go.uber.org/zap" ) func (group *Group) shouldSpill() bool { - return group.SpillThreshold > 0 && + should := group.SpillThreshold > 0 && group.SpillManager != nil && group.ctr.currentMemUsage > group.SpillThreshold && len(group.ctr.result1.AggList) > 0 && len(group.ctr.result1.ToPopped) > 0 + if should { + logutil.Debug("group operator should spill", + zap.String("component", "group-spill"), + zap.Int64("current-mem", group.ctr.currentMemUsage), + zap.Int64("threshold", group.SpillThreshold)) + } + return should } func (group *Group) updateMemoryUsage(proc *process.Process) { @@ -59,6 +68,11 @@ func (group *Group) spillPartialResults(proc *process.Process) error { return nil } + logutil.Info("spilling partial results for group operator", + zap.String("component", "group-spill"), + zap.Int("agg-count", len(group.ctr.result1.AggList)), + zap.Int("batch-count", len(group.ctr.result1.ToPopped))) + marshaledAggStates := make([][]byte, len(group.ctr.result1.AggList)) for i, agg := range group.ctr.result1.AggList { if agg != nil { @@ -90,14 +104,14 @@ func (group *Group) spillPartialResults(proc *process.Process) error { var groupVecs []*vector.Vector var groupVecTypes []types.Type - if len(group.ctr.result1.ToPopped) > 0 && group.ctr.result1.ToPopped[0] != nil { - numGroupByCols := len(group.ctr.result1.ToPopped[0].Vecs) + if len(group.ctr.result1.ToPopped) > 0 && group.ctr.result1.ToPopped != nil { + numGroupByCols := len(group.ctr.result1.ToPopped.Vecs) groupVecs = make([]*vector.Vector, numGroupByCols) groupVecTypes = make([]types.Type, numGroupByCols) for i := 0; i < numGroupByCols; i++ { - if len(group.ctr.result1.ToPopped[0].Vecs) > i && group.ctr.result1.ToPopped[0].Vecs[i] != nil { - vecType := *group.ctr.result1.ToPopped[0].Vecs[i].GetType() + if len(group.ctr.result1.ToPopped.Vecs) > i && group.ctr.result1.ToPopped.Vecs[i] != nil { + vecType := *group.ctr.result1.ToPopped.Vecs[i].GetType() groupVecs[i] = vector.NewOffHeapVecWithType(vecType) groupVecTypes[i] = vecType } @@ -133,6 +147,9 @@ func (group *Group) spillPartialResults(proc *process.Process) error { spillData.Free(proc.Mp()) return err } + logutil.Info("finished spilling partial results for group operator", + zap.String("component", "group-spill"), + zap.String("spill-id", string(spillID))) group.ctr.spilledStates = append(group.ctr.spilledStates, spillID) @@ -164,7 +181,14 @@ func (group *Group) mergeSpilledResults(proc *process.Process) error { return nil } + logutil.Info("merging spilled results for group operator", + zap.String("component", "group-spill"), + zap.Int("spill-files", len(group.ctr.spilledStates))) + for _, spillID := range group.ctr.spilledStates { + logutil.Debug("merging spilled file", + zap.String("component", "group-spill"), + zap.String("spill-id", string(spillID))) spillData, err := group.SpillManager.Retrieve(spillID, proc.Mp()) if err != nil { return err @@ -188,6 +212,8 @@ func (group *Group) mergeSpilledResults(proc *process.Process) error { } group.ctr.spilledStates = nil + logutil.Info("finished merging spilled results for group operator", + zap.String("component", "group-spill")) return nil } diff --git a/pkg/sql/colexec/group/spill_memory.go b/pkg/sql/colexec/group/spill_memory.go index 3701e7e2f8ef5..696e0848f2d87 100644 --- a/pkg/sql/colexec/group/spill_memory.go +++ b/pkg/sql/colexec/group/spill_memory.go @@ -20,6 +20,8 @@ import ( "sync/atomic" "github.com/matrixorigin/matrixone/pkg/common/mpool" + "github.com/matrixorigin/matrixone/pkg/logutil" + "go.uber.org/zap" ) type MemorySpillManager struct { @@ -30,6 +32,7 @@ type MemorySpillManager struct { } func NewMemorySpillManager() *MemorySpillManager { + logutil.Info("new memory spill manager", zap.String("component", "group-spill")) return &MemorySpillManager{ data: make(map[SpillID][]byte), } @@ -46,6 +49,11 @@ func (m *MemorySpillManager) Spill(data SpillableData) (SpillID, error) { defer m.mu.Unlock() m.data[id] = serialized atomic.AddInt64(&m.totalMem, int64(len(serialized))) + logutil.Debug("spilled data to memory", + zap.String("component", "group-spill"), + zap.String("spill-id", string(id)), + zap.Int("size", len(serialized)), + zap.Int64("total-mem", atomic.LoadInt64(&m.totalMem))) return id, nil } @@ -63,6 +71,10 @@ func (m *MemorySpillManager) Retrieve(id SpillID, mp *mpool.MPool) (SpillableDat data.Free(mp) return nil, err } + logutil.Debug("retrieved data from memory", + zap.String("component", "group-spill"), + zap.String("spill-id", string(id)), + zap.Int("size", len(serialized))) return data, nil } @@ -71,8 +83,14 @@ func (m *MemorySpillManager) Delete(id SpillID) error { defer m.mu.Unlock() if serialized, exists := m.data[id]; exists { - atomic.AddInt64(&m.totalMem, -int64(len(serialized))) + size := len(serialized) + atomic.AddInt64(&m.totalMem, -int64(size)) delete(m.data, id) + logutil.Debug("deleted spilled data from memory", + zap.String("component", "group-spill"), + zap.String("spill-id", string(id)), + zap.Int("size", size), + zap.Int64("total-mem", atomic.LoadInt64(&m.totalMem))) } return nil } @@ -81,8 +99,16 @@ func (m *MemorySpillManager) Free() { m.mu.Lock() defer m.mu.Unlock() + logutil.Info("freeing memory spill manager", + zap.String("component", "group-spill"), + zap.Int("spilled-items", len(m.data)), + zap.Int64("total-mem", atomic.LoadInt64(&m.totalMem))) + for id := range m.data { - m.Delete(id) + if serialized, exists := m.data[id]; exists { + atomic.AddInt64(&m.totalMem, -int64(len(serialized))) + delete(m.data, id) + } } m.data = nil } diff --git a/pkg/sql/colexec/group/spillable_agg_state.go b/pkg/sql/colexec/group/spillable_agg_state.go index 685ff93f90965..380c8b4657dd8 100644 --- a/pkg/sql/colexec/group/spillable_agg_state.go +++ b/pkg/sql/colexec/group/spillable_agg_state.go @@ -21,6 +21,8 @@ import ( "github.com/matrixorigin/matrixone/pkg/common/mpool" "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/container/vector" + "github.com/matrixorigin/matrixone/pkg/logutil" + "go.uber.org/zap" ) type SpillableAggState struct { @@ -93,10 +95,18 @@ func (s *SpillableAggState) Serialize() ([]byte, error) { } } - return buf.Bytes(), nil + retBytes := buf.Bytes() + logutil.Debug("serialized spillable agg state", + zap.String("component", "group-spill"), + zap.Int("size", len(retBytes)), + zap.Int("group-count", s.GroupCount)) + return retBytes, nil } func (s *SpillableAggState) Deserialize(data []byte, mp *mpool.MPool) error { + logutil.Debug("deserializing spillable agg state", + zap.String("component", "group-spill"), + zap.Int("size", len(data))) buf := bytes.NewReader(data) var groupCount int32 From 93f446d5ce09e0a43e791d46d50d47560290ab38 Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 26 Sep 2025 16:29:57 +0800 Subject: [PATCH 41/57] Revert "add logs for spill" This reverts commit 426ddfcfa6f00b00c4f1410e84186e07bae0d30a. --- pkg/sql/colexec/group/group_spill.go | 36 +++----------------- pkg/sql/colexec/group/spill_memory.go | 30 ++-------------- pkg/sql/colexec/group/spillable_agg_state.go | 12 +------ 3 files changed, 8 insertions(+), 70 deletions(-) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index e35af446075f1..520e2e075896e 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -20,25 +20,16 @@ import ( "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/container/vector" - "github.com/matrixorigin/matrixone/pkg/logutil" "github.com/matrixorigin/matrixone/pkg/sql/colexec/aggexec" "github.com/matrixorigin/matrixone/pkg/vm/process" - "go.uber.org/zap" ) func (group *Group) shouldSpill() bool { - should := group.SpillThreshold > 0 && + return group.SpillThreshold > 0 && group.SpillManager != nil && group.ctr.currentMemUsage > group.SpillThreshold && len(group.ctr.result1.AggList) > 0 && len(group.ctr.result1.ToPopped) > 0 - if should { - logutil.Debug("group operator should spill", - zap.String("component", "group-spill"), - zap.Int64("current-mem", group.ctr.currentMemUsage), - zap.Int64("threshold", group.SpillThreshold)) - } - return should } func (group *Group) updateMemoryUsage(proc *process.Process) { @@ -68,11 +59,6 @@ func (group *Group) spillPartialResults(proc *process.Process) error { return nil } - logutil.Info("spilling partial results for group operator", - zap.String("component", "group-spill"), - zap.Int("agg-count", len(group.ctr.result1.AggList)), - zap.Int("batch-count", len(group.ctr.result1.ToPopped))) - marshaledAggStates := make([][]byte, len(group.ctr.result1.AggList)) for i, agg := range group.ctr.result1.AggList { if agg != nil { @@ -104,14 +90,14 @@ func (group *Group) spillPartialResults(proc *process.Process) error { var groupVecs []*vector.Vector var groupVecTypes []types.Type - if len(group.ctr.result1.ToPopped) > 0 && group.ctr.result1.ToPopped != nil { - numGroupByCols := len(group.ctr.result1.ToPopped.Vecs) + if len(group.ctr.result1.ToPopped) > 0 && group.ctr.result1.ToPopped[0] != nil { + numGroupByCols := len(group.ctr.result1.ToPopped[0].Vecs) groupVecs = make([]*vector.Vector, numGroupByCols) groupVecTypes = make([]types.Type, numGroupByCols) for i := 0; i < numGroupByCols; i++ { - if len(group.ctr.result1.ToPopped.Vecs) > i && group.ctr.result1.ToPopped.Vecs[i] != nil { - vecType := *group.ctr.result1.ToPopped.Vecs[i].GetType() + if len(group.ctr.result1.ToPopped[0].Vecs) > i && group.ctr.result1.ToPopped[0].Vecs[i] != nil { + vecType := *group.ctr.result1.ToPopped[0].Vecs[i].GetType() groupVecs[i] = vector.NewOffHeapVecWithType(vecType) groupVecTypes[i] = vecType } @@ -147,9 +133,6 @@ func (group *Group) spillPartialResults(proc *process.Process) error { spillData.Free(proc.Mp()) return err } - logutil.Info("finished spilling partial results for group operator", - zap.String("component", "group-spill"), - zap.String("spill-id", string(spillID))) group.ctr.spilledStates = append(group.ctr.spilledStates, spillID) @@ -181,14 +164,7 @@ func (group *Group) mergeSpilledResults(proc *process.Process) error { return nil } - logutil.Info("merging spilled results for group operator", - zap.String("component", "group-spill"), - zap.Int("spill-files", len(group.ctr.spilledStates))) - for _, spillID := range group.ctr.spilledStates { - logutil.Debug("merging spilled file", - zap.String("component", "group-spill"), - zap.String("spill-id", string(spillID))) spillData, err := group.SpillManager.Retrieve(spillID, proc.Mp()) if err != nil { return err @@ -212,8 +188,6 @@ func (group *Group) mergeSpilledResults(proc *process.Process) error { } group.ctr.spilledStates = nil - logutil.Info("finished merging spilled results for group operator", - zap.String("component", "group-spill")) return nil } diff --git a/pkg/sql/colexec/group/spill_memory.go b/pkg/sql/colexec/group/spill_memory.go index 696e0848f2d87..3701e7e2f8ef5 100644 --- a/pkg/sql/colexec/group/spill_memory.go +++ b/pkg/sql/colexec/group/spill_memory.go @@ -20,8 +20,6 @@ import ( "sync/atomic" "github.com/matrixorigin/matrixone/pkg/common/mpool" - "github.com/matrixorigin/matrixone/pkg/logutil" - "go.uber.org/zap" ) type MemorySpillManager struct { @@ -32,7 +30,6 @@ type MemorySpillManager struct { } func NewMemorySpillManager() *MemorySpillManager { - logutil.Info("new memory spill manager", zap.String("component", "group-spill")) return &MemorySpillManager{ data: make(map[SpillID][]byte), } @@ -49,11 +46,6 @@ func (m *MemorySpillManager) Spill(data SpillableData) (SpillID, error) { defer m.mu.Unlock() m.data[id] = serialized atomic.AddInt64(&m.totalMem, int64(len(serialized))) - logutil.Debug("spilled data to memory", - zap.String("component", "group-spill"), - zap.String("spill-id", string(id)), - zap.Int("size", len(serialized)), - zap.Int64("total-mem", atomic.LoadInt64(&m.totalMem))) return id, nil } @@ -71,10 +63,6 @@ func (m *MemorySpillManager) Retrieve(id SpillID, mp *mpool.MPool) (SpillableDat data.Free(mp) return nil, err } - logutil.Debug("retrieved data from memory", - zap.String("component", "group-spill"), - zap.String("spill-id", string(id)), - zap.Int("size", len(serialized))) return data, nil } @@ -83,14 +71,8 @@ func (m *MemorySpillManager) Delete(id SpillID) error { defer m.mu.Unlock() if serialized, exists := m.data[id]; exists { - size := len(serialized) - atomic.AddInt64(&m.totalMem, -int64(size)) + atomic.AddInt64(&m.totalMem, -int64(len(serialized))) delete(m.data, id) - logutil.Debug("deleted spilled data from memory", - zap.String("component", "group-spill"), - zap.String("spill-id", string(id)), - zap.Int("size", size), - zap.Int64("total-mem", atomic.LoadInt64(&m.totalMem))) } return nil } @@ -99,16 +81,8 @@ func (m *MemorySpillManager) Free() { m.mu.Lock() defer m.mu.Unlock() - logutil.Info("freeing memory spill manager", - zap.String("component", "group-spill"), - zap.Int("spilled-items", len(m.data)), - zap.Int64("total-mem", atomic.LoadInt64(&m.totalMem))) - for id := range m.data { - if serialized, exists := m.data[id]; exists { - atomic.AddInt64(&m.totalMem, -int64(len(serialized))) - delete(m.data, id) - } + m.Delete(id) } m.data = nil } diff --git a/pkg/sql/colexec/group/spillable_agg_state.go b/pkg/sql/colexec/group/spillable_agg_state.go index 380c8b4657dd8..685ff93f90965 100644 --- a/pkg/sql/colexec/group/spillable_agg_state.go +++ b/pkg/sql/colexec/group/spillable_agg_state.go @@ -21,8 +21,6 @@ import ( "github.com/matrixorigin/matrixone/pkg/common/mpool" "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/container/vector" - "github.com/matrixorigin/matrixone/pkg/logutil" - "go.uber.org/zap" ) type SpillableAggState struct { @@ -95,18 +93,10 @@ func (s *SpillableAggState) Serialize() ([]byte, error) { } } - retBytes := buf.Bytes() - logutil.Debug("serialized spillable agg state", - zap.String("component", "group-spill"), - zap.Int("size", len(retBytes)), - zap.Int("group-count", s.GroupCount)) - return retBytes, nil + return buf.Bytes(), nil } func (s *SpillableAggState) Deserialize(data []byte, mp *mpool.MPool) error { - logutil.Debug("deserializing spillable agg state", - zap.String("component", "group-spill"), - zap.Int("size", len(data))) buf := bytes.NewReader(data) var groupCount int32 From 3e2d1c26c24b96cf90c2d23bf0dda6f187baccb3 Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 26 Sep 2025 16:38:24 +0800 Subject: [PATCH 42/57] more logs for spill --- pkg/sql/colexec/group/group_spill.go | 22 +++++++++++++++++++++- 1 file changed, 21 insertions(+), 1 deletion(-) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index 520e2e075896e..b6899b9878f09 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -20,16 +20,28 @@ import ( "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/container/vector" + "github.com/matrixorigin/matrixone/pkg/logutil" "github.com/matrixorigin/matrixone/pkg/sql/colexec/aggexec" "github.com/matrixorigin/matrixone/pkg/vm/process" + "go.uber.org/zap" ) func (group *Group) shouldSpill() bool { - return group.SpillThreshold > 0 && + shouldSpill := group.SpillThreshold > 0 && group.SpillManager != nil && group.ctr.currentMemUsage > group.SpillThreshold && len(group.ctr.result1.AggList) > 0 && len(group.ctr.result1.ToPopped) > 0 + + if shouldSpill { + logutil.Debug("Group operator triggering spill", + zap.Int64("current_memory_usage", group.ctr.currentMemUsage), + zap.Int64("spill_threshold", group.SpillThreshold), + zap.Int("agg_count", len(group.ctr.result1.AggList)), + zap.Int("batch_count", len(group.ctr.result1.ToPopped))) + } + + return shouldSpill } func (group *Group) updateMemoryUsage(proc *process.Process) { @@ -51,7 +63,15 @@ func (group *Group) updateMemoryUsage(proc *process.Process) { } } + previousUsage := group.ctr.currentMemUsage group.ctr.currentMemUsage = usage + + if usage > previousUsage && usage > group.SpillThreshold/2 { + logutil.Debug("Group operator memory usage update", + zap.Int64("previous_usage", previousUsage), + zap.Int64("current_usage", usage), + zap.Int64("spill_threshold", group.SpillThreshold)) + } } func (group *Group) spillPartialResults(proc *process.Process) error { From fbaf61e3130600336ae19af2490bb594e845b384 Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 26 Sep 2025 16:39:17 +0800 Subject: [PATCH 43/57] more logs for spill --- pkg/sql/colexec/group/group_spill.go | 47 +++++++++++++++++++++++++++- 1 file changed, 46 insertions(+), 1 deletion(-) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index b6899b9878f09..07f695fe4a711 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -76,14 +76,22 @@ func (group *Group) updateMemoryUsage(proc *process.Process) { func (group *Group) spillPartialResults(proc *process.Process) error { if len(group.ctr.result1.AggList) == 0 || len(group.ctr.result1.ToPopped) == 0 { + logutil.Debug("Group operator spill called but no data to spill") return nil } + logutil.Info("Group operator starting spill operation", + zap.Int64("memory_usage", group.ctr.currentMemUsage), + zap.Int64("spill_threshold", group.SpillThreshold), + zap.Int("agg_count", len(group.ctr.result1.AggList))) + marshaledAggStates := make([][]byte, len(group.ctr.result1.AggList)) for i, agg := range group.ctr.result1.AggList { if agg != nil { marshaledData, err := aggexec.MarshalAggFuncExec(agg) if err != nil { + logutil.Error("Group operator failed to marshal aggregator", + zap.Int("agg_index", i), zap.Error(err)) return err } marshaledAggStates[i] = marshaledData @@ -98,6 +106,7 @@ func (group *Group) spillPartialResults(proc *process.Process) error { } if totalGroups == 0 { + logutil.Debug("Group operator spill found no groups to spill") for _, agg := range group.ctr.result1.AggList { if agg != nil { agg.Free() @@ -128,6 +137,8 @@ func (group *Group) spillPartialResults(proc *process.Process) error { for i, vec := range bat.Vecs { if i < len(groupVecs) && groupVecs[i] != nil && vec != nil { if err := groupVecs[i].UnionBatch(vec, 0, vec.Length(), nil, proc.Mp()); err != nil { + logutil.Error("Group operator failed to union batch during spill", + zap.Int("vec_index", i), zap.Error(err)) for j := range groupVecs { if groupVecs[j] != nil { groupVecs[j].Free(proc.Mp()) @@ -150,10 +161,16 @@ func (group *Group) spillPartialResults(proc *process.Process) error { spillID, err := group.SpillManager.Spill(spillData) if err != nil { + logutil.Error("Group operator failed to spill data", zap.Error(err)) spillData.Free(proc.Mp()) return err } + logutil.Info("Group operator successfully spilled data", + zap.String("spill_id", string(spillID)), + zap.Int("total_groups", totalGroups), + zap.Int64("estimated_size", spillData.EstimateSize())) + group.ctr.spilledStates = append(group.ctr.spilledStates, spillID) for _, agg := range group.ctr.result1.AggList { @@ -176,6 +193,8 @@ func (group *Group) spillPartialResults(proc *process.Process) error { } group.ctr.currentMemUsage = 0 + logutil.Debug("Group operator completed spill cleanup", + zap.Int("spilled_states_count", len(group.ctr.spilledStates))) return nil } @@ -184,29 +203,55 @@ func (group *Group) mergeSpilledResults(proc *process.Process) error { return nil } - for _, spillID := range group.ctr.spilledStates { + logutil.Info("Group operator starting merge of spilled results", + zap.Int("spilled_states_count", len(group.ctr.spilledStates))) + + for i, spillID := range group.ctr.spilledStates { + logutil.Debug("Group operator merging spilled state", + zap.Int("state_index", i), + zap.String("spill_id", string(spillID))) + spillData, err := group.SpillManager.Retrieve(spillID, proc.Mp()) if err != nil { + logutil.Error("Group operator failed to retrieve spilled data", + zap.String("spill_id", string(spillID)), zap.Error(err)) return err } spillState, ok := spillData.(*SpillableAggState) if !ok { + logutil.Error("Group operator retrieved invalid spilled data type", + zap.String("spill_id", string(spillID))) spillData.Free(proc.Mp()) panic(fmt.Sprintf("invalid spilled data type")) } + logutil.Debug("Group operator retrieved spilled state", + zap.String("spill_id", string(spillID)), + zap.Int("group_count", spillState.GroupCount), + zap.Int64("estimated_size", spillState.EstimateSize())) + if err = group.restoreAndMergeSpilledAggregators(proc, spillState); err != nil { + logutil.Error("Group operator failed to restore and merge spilled aggregators", + zap.String("spill_id", string(spillID)), zap.Error(err)) spillState.Free(proc.Mp()) return err } spillState.Free(proc.Mp()) if err = group.SpillManager.Delete(spillID); err != nil { + logutil.Error("Group operator failed to delete spilled data", + zap.String("spill_id", string(spillID)), zap.Error(err)) return err } + + logutil.Debug("Group operator completed merge of spilled state", + zap.String("spill_id", string(spillID))) } + logutil.Info("Group operator completed merge of all spilled results", + zap.Int("merged_states_count", len(group.ctr.spilledStates))) + group.ctr.spilledStates = nil return nil } From 6bd9e6b1e826eba5142142899564f7a1b2a61b66 Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 26 Sep 2025 16:39:47 +0800 Subject: [PATCH 44/57] more logs for spill --- pkg/sql/colexec/group/group_spill.go | 35 ++++++++++++++++++++++++++++ 1 file changed, 35 insertions(+) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index 07f695fe4a711..3e588e68576b2 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -258,10 +258,16 @@ func (group *Group) mergeSpilledResults(proc *process.Process) error { func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spillState *SpillableAggState) error { if len(spillState.MarshaledAggStates) == 0 { + logutil.Debug("Group operator restore found no marshaled aggregator states") return nil } + logutil.Debug("Group operator restoring spilled aggregators", + zap.Int("agg_states_count", len(spillState.MarshaledAggStates)), + zap.Int("group_count", spillState.GroupCount)) + if len(group.ctr.result1.AggList) == 0 { + logutil.Debug("Group operator initializing aggregators from spilled state") aggs := make([]aggexec.AggFuncExec, len(spillState.MarshaledAggStates)) defer func() { if group.ctr.result1.AggList == nil { @@ -280,6 +286,8 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi agg, err := aggexec.UnmarshalAggFuncExec(aggexec.NewSimpleAggMemoryManager(proc.Mp()), marshaledState) if err != nil { + logutil.Error("Group operator failed to unmarshal aggregator", + zap.Int("agg_index", i), zap.Error(err)) return err } @@ -287,6 +295,8 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi aggExpr := group.Aggs[i] if config := aggExpr.GetExtraConfig(); config != nil { if err = agg.SetExtraInformation(config, 0); err != nil { + logutil.Error("Group operator failed to set extra information for aggregator", + zap.Int("agg_index", i), zap.Error(err)) agg.Free() return err } @@ -301,6 +311,10 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi group.ctr.result1.ChunkSize = chunkSize group.ctr.result1.AggList = aggs + logutil.Debug("Group operator initialized aggregators from spilled state", + zap.Int("chunk_size", chunkSize), + zap.Int("agg_count", len(aggs))) + if len(spillState.GroupVectors) > 0 && spillState.GroupCount > 0 { batchesToAdd := make([]*batch.Batch, 0) @@ -314,6 +328,8 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi for i, vec := range spillState.GroupVectors { if vec != nil && i < len(bat.Vecs) { if err := bat.Vecs[i].UnionBatch(vec, int64(offset), size, nil, proc.Mp()); err != nil { + logutil.Error("Group operator failed to union batch during restore", + zap.Int("vec_index", i), zap.Int("offset", offset), zap.Error(err)) bat.Clean(proc.Mp()) for _, b := range batchesToAdd { b.Clean(proc.Mp()) @@ -331,9 +347,14 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi return nil } + logutil.Debug("Group operator merging spilled aggregators with existing ones", + zap.Int("existing_agg_count", len(group.ctr.result1.AggList)), + zap.Int("spilled_group_count", spillState.GroupCount)) + for _, currentAgg := range group.ctr.result1.AggList { if currentAgg != nil { if err := currentAgg.GroupGrow(spillState.GroupCount); err != nil { + logutil.Error("Group operator failed to grow aggregator groups", zap.Error(err)) return err } } @@ -355,6 +376,8 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi agg, err := aggexec.UnmarshalAggFuncExec(aggexec.NewSimpleAggMemoryManager(proc.Mp()), marshaledState) if err != nil { + logutil.Error("Group operator failed to unmarshal aggregator for merge", + zap.Int("agg_index", i), zap.Error(err)) return err } @@ -362,6 +385,8 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi aggExpr := group.Aggs[i] if config := aggExpr.GetExtraConfig(); config != nil { if err = agg.SetExtraInformation(config, 0); err != nil { + logutil.Error("Group operator failed to set extra information for temp aggregator", + zap.Int("agg_index", i), zap.Error(err)) agg.Free() return err } @@ -391,6 +416,11 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi for spilledGroupIdx := 0; spilledGroupIdx < spillState.GroupCount; spilledGroupIdx++ { currentGroupIdx := currentGroupCount + spilledGroupIdx if err := currentAgg.Merge(tempAgg, currentGroupIdx, spilledGroupIdx); err != nil { + logutil.Error("Group operator failed to merge aggregator groups", + zap.Int("agg_index", i), + zap.Int("current_group_idx", currentGroupIdx), + zap.Int("spilled_group_idx", spilledGroupIdx), + zap.Error(err)) return err } } @@ -413,6 +443,8 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi for i, vec := range spillState.GroupVectors { if vec != nil && i < len(bat.Vecs) { if err := bat.Vecs[i].UnionBatch(vec, int64(offset), size, nil, proc.Mp()); err != nil { + logutil.Error("Group operator failed to union batch during merge", + zap.Int("vec_index", i), zap.Int("offset", offset), zap.Error(err)) bat.Clean(proc.Mp()) for _, b := range batchesToAdd { b.Clean(proc.Mp()) @@ -427,5 +459,8 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi group.ctr.result1.ToPopped = append(group.ctr.result1.ToPopped, batchesToAdd...) } + logutil.Debug("Group operator completed restore and merge of spilled aggregators", + zap.Int("final_batch_count", len(group.ctr.result1.ToPopped))) + return nil } From 0afe4ffe2d35f46b9e4a1ef61dd5673a9d031d87 Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 26 Sep 2025 16:41:55 +0800 Subject: [PATCH 45/57] more logs for spill --- pkg/sql/colexec/group/spill_memory.go | 43 +++++++++++++++++++++++++-- 1 file changed, 41 insertions(+), 2 deletions(-) diff --git a/pkg/sql/colexec/group/spill_memory.go b/pkg/sql/colexec/group/spill_memory.go index 3701e7e2f8ef5..d1736d01df21f 100644 --- a/pkg/sql/colexec/group/spill_memory.go +++ b/pkg/sql/colexec/group/spill_memory.go @@ -20,6 +20,8 @@ import ( "sync/atomic" "github.com/matrixorigin/matrixone/pkg/common/mpool" + "github.com/matrixorigin/matrixone/pkg/logutil" + "go.uber.org/zap" ) type MemorySpillManager struct { @@ -38,6 +40,7 @@ func NewMemorySpillManager() *MemorySpillManager { func (m *MemorySpillManager) Spill(data SpillableData) (SpillID, error) { serialized, err := data.Serialize() if err != nil { + logutil.Error("MemorySpillManager failed to serialize data", zap.Error(err)) return "", err } @@ -45,7 +48,13 @@ func (m *MemorySpillManager) Spill(data SpillableData) (SpillID, error) { m.mu.Lock() defer m.mu.Unlock() m.data[id] = serialized - atomic.AddInt64(&m.totalMem, int64(len(serialized))) + newTotalMem := atomic.AddInt64(&m.totalMem, int64(len(serialized))) + + logutil.Debug("MemorySpillManager spilled data", + zap.String("spill_id", string(id)), + zap.Int("data_size", len(serialized)), + zap.Int64("total_memory", newTotalMem)) + return id, nil } @@ -55,14 +64,27 @@ func (m *MemorySpillManager) Retrieve(id SpillID, mp *mpool.MPool) (SpillableDat serialized, exists := m.data[id] if !exists { + logutil.Error("MemorySpillManager failed to find spilled data", + zap.String("spill_id", string(id))) return nil, fmt.Errorf("spill data not found: %s", id) } + logutil.Debug("MemorySpillManager retrieving spilled data", + zap.String("spill_id", string(id)), + zap.Int("data_size", len(serialized))) + data := &SpillableAggState{} if err := data.Deserialize(serialized, mp); err != nil { + logutil.Error("MemorySpillManager failed to deserialize data", + zap.String("spill_id", string(id)), zap.Error(err)) data.Free(mp) return nil, err } + + logutil.Debug("MemorySpillManager successfully retrieved and deserialized data", + zap.String("spill_id", string(id)), + zap.Int64("estimated_size", data.EstimateSize())) + return data, nil } @@ -71,8 +93,16 @@ func (m *MemorySpillManager) Delete(id SpillID) error { defer m.mu.Unlock() if serialized, exists := m.data[id]; exists { - atomic.AddInt64(&m.totalMem, -int64(len(serialized))) + newTotalMem := atomic.AddInt64(&m.totalMem, -int64(len(serialized))) delete(m.data, id) + + logutil.Debug("MemorySpillManager deleted spilled data", + zap.String("spill_id", string(id)), + zap.Int("data_size", len(serialized)), + zap.Int64("total_memory", newTotalMem)) + } else { + logutil.Warn("MemorySpillManager attempted to delete non-existent spilled data", + zap.String("spill_id", string(id))) } return nil } @@ -81,10 +111,19 @@ func (m *MemorySpillManager) Free() { m.mu.Lock() defer m.mu.Unlock() + count := len(m.data) + totalSize := atomic.LoadInt64(&m.totalMem) + + logutil.Debug("MemorySpillManager freeing all spilled data", + zap.Int("spilled_count", count), + zap.Int64("total_size", totalSize)) + for id := range m.data { m.Delete(id) } m.data = nil + + logutil.Debug("MemorySpillManager completed cleanup") } func (m *MemorySpillManager) TotalMem() int64 { From 1043ec650a4f2ebece65aec8f413a1c9cc55de0e Mon Sep 17 00:00:00 2001 From: reus Date: Fri, 26 Sep 2025 16:49:57 +0800 Subject: [PATCH 46/57] add logs for spill --- pkg/sql/colexec/group/spillable_agg_state.go | 62 +++++++++++++++++++- 1 file changed, 60 insertions(+), 2 deletions(-) diff --git a/pkg/sql/colexec/group/spillable_agg_state.go b/pkg/sql/colexec/group/spillable_agg_state.go index 685ff93f90965..130e0f4e670ae 100644 --- a/pkg/sql/colexec/group/spillable_agg_state.go +++ b/pkg/sql/colexec/group/spillable_agg_state.go @@ -21,6 +21,8 @@ import ( "github.com/matrixorigin/matrixone/pkg/common/mpool" "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/container/vector" + "github.com/matrixorigin/matrixone/pkg/logutil" + "go.uber.org/zap" ) type SpillableAggState struct { @@ -31,28 +33,39 @@ type SpillableAggState struct { } func (s *SpillableAggState) Serialize() ([]byte, error) { + logutil.Debug("SpillableAggState starting serialization", + zap.Int("group_count", s.GroupCount), + zap.Int("group_vectors_count", len(s.GroupVectors)), + zap.Int("agg_states_count", len(s.MarshaledAggStates))) + buf := bytes.NewBuffer(nil) if err := binary.Write(buf, binary.LittleEndian, int32(s.GroupCount)); err != nil { + logutil.Error("SpillableAggState failed to write group count", zap.Error(err)) return nil, err } if err := binary.Write(buf, binary.LittleEndian, int32(len(s.GroupVectors))); err != nil { + logutil.Error("SpillableAggState failed to write group vectors count", zap.Error(err)) return nil, err } if err := binary.Write(buf, binary.LittleEndian, int32(len(s.GroupVectorTypes))); err != nil { + logutil.Error("SpillableAggState failed to write group vector types count", zap.Error(err)) return nil, err } for _, typ := range s.GroupVectorTypes { typBytes, err := typ.MarshalBinary() if err != nil { + logutil.Error("SpillableAggState failed to marshal vector type", zap.Error(err)) return nil, err } if err := binary.Write(buf, binary.LittleEndian, int32(len(typBytes))); err != nil { + logutil.Error("SpillableAggState failed to write type bytes length", zap.Error(err)) return nil, err } if _, err := buf.Write(typBytes); err != nil { + logutil.Error("SpillableAggState failed to write type bytes", zap.Error(err)) return nil, err } } @@ -60,6 +73,8 @@ func (s *SpillableAggState) Serialize() ([]byte, error) { for i, vec := range s.GroupVectors { if vec == nil { if err := binary.Write(buf, binary.LittleEndian, int32(0)); err != nil { + logutil.Error("SpillableAggState failed to write zero length for nil vector", + zap.Int("vec_index", i), zap.Error(err)) return nil, err } continue @@ -67,12 +82,18 @@ func (s *SpillableAggState) Serialize() ([]byte, error) { vecBytes, err := vec.MarshalBinary() if err != nil { + logutil.Error("SpillableAggState failed to marshal vector", + zap.Int("vec_index", i), zap.Error(err)) return nil, err } if err := binary.Write(buf, binary.LittleEndian, int32(len(vecBytes))); err != nil { + logutil.Error("SpillableAggState failed to write vector bytes length", + zap.Int("vec_index", i), zap.Error(err)) return nil, err } if _, err := buf.Write(vecBytes); err != nil { + logutil.Error("SpillableAggState failed to write vector bytes", + zap.Int("vec_index", i), zap.Error(err)) return nil, err } @@ -82,49 +103,70 @@ func (s *SpillableAggState) Serialize() ([]byte, error) { } if err := binary.Write(buf, binary.LittleEndian, int32(len(s.MarshaledAggStates))); err != nil { + logutil.Error("SpillableAggState failed to write agg states count", zap.Error(err)) return nil, err } - for _, aggState := range s.MarshaledAggStates { + for i, aggState := range s.MarshaledAggStates { if err := binary.Write(buf, binary.LittleEndian, int32(len(aggState))); err != nil { + logutil.Error("SpillableAggState failed to write agg state length", + zap.Int("agg_index", i), zap.Error(err)) return nil, err } if _, err := buf.Write(aggState); err != nil { + logutil.Error("SpillableAggState failed to write agg state bytes", + zap.Int("agg_index", i), zap.Error(err)) return nil, err } } - return buf.Bytes(), nil + result := buf.Bytes() + logutil.Debug("SpillableAggState completed serialization", + zap.Int("serialized_size", len(result))) + + return result, nil } func (s *SpillableAggState) Deserialize(data []byte, mp *mpool.MPool) error { + logutil.Debug("SpillableAggState starting deserialization", + zap.Int("data_size", len(data))) + buf := bytes.NewReader(data) var groupCount int32 if err := binary.Read(buf, binary.LittleEndian, &groupCount); err != nil { + logutil.Error("SpillableAggState failed to read group count", zap.Error(err)) return err } s.GroupCount = int(groupCount) var groupVecCount int32 if err := binary.Read(buf, binary.LittleEndian, &groupVecCount); err != nil { + logutil.Error("SpillableAggState failed to read group vector count", zap.Error(err)) return err } var groupVecTypeCount int32 if err := binary.Read(buf, binary.LittleEndian, &groupVecTypeCount); err != nil { + logutil.Error("SpillableAggState failed to read group vector type count", zap.Error(err)) return err } s.GroupVectorTypes = make([]types.Type, groupVecTypeCount) for i := 0; i < int(groupVecTypeCount); i++ { var size int32 if err := binary.Read(buf, binary.LittleEndian, &size); err != nil { + logutil.Error("SpillableAggState failed to read vector type size", + zap.Int("type_index", i), zap.Error(err)) return err } typBytes := make([]byte, size) if _, err := buf.Read(typBytes); err != nil { + logutil.Error("SpillableAggState failed to read vector type bytes", + zap.Int("type_index", i), zap.Error(err)) return err } if err := s.GroupVectorTypes[i].UnmarshalBinary(typBytes); err != nil { + logutil.Error("SpillableAggState failed to unmarshal vector type", + zap.Int("type_index", i), zap.Error(err)) return err } } @@ -133,6 +175,8 @@ func (s *SpillableAggState) Deserialize(data []byte, mp *mpool.MPool) error { for i := 0; i < int(groupVecCount); i++ { var size int32 if err := binary.Read(buf, binary.LittleEndian, &size); err != nil { + logutil.Error("SpillableAggState failed to read vector size", + zap.Int("vec_index", i), zap.Error(err)) return err } if size == 0 { @@ -142,6 +186,8 @@ func (s *SpillableAggState) Deserialize(data []byte, mp *mpool.MPool) error { vecBytes := make([]byte, size) if _, err := buf.Read(vecBytes); err != nil { + logutil.Error("SpillableAggState failed to read vector bytes", + zap.Int("vec_index", i), zap.Error(err)) return err } @@ -154,6 +200,8 @@ func (s *SpillableAggState) Deserialize(data []byte, mp *mpool.MPool) error { vec := vector.NewOffHeapVecWithType(vecType) if err := vec.UnmarshalBinaryWithCopy(vecBytes, mp); err != nil { + logutil.Error("SpillableAggState failed to unmarshal vector", + zap.Int("vec_index", i), zap.Error(err)) vec.Free(mp) return err } @@ -162,20 +210,30 @@ func (s *SpillableAggState) Deserialize(data []byte, mp *mpool.MPool) error { var aggStateCount int32 if err := binary.Read(buf, binary.LittleEndian, &aggStateCount); err != nil { + logutil.Error("SpillableAggState failed to read agg state count", zap.Error(err)) return err } s.MarshaledAggStates = make([][]byte, aggStateCount) for i := 0; i < int(aggStateCount); i++ { var size int32 if err := binary.Read(buf, binary.LittleEndian, &size); err != nil { + logutil.Error("SpillableAggState failed to read agg state size", + zap.Int("agg_index", i), zap.Error(err)) return err } s.MarshaledAggStates[i] = make([]byte, size) if _, err := buf.Read(s.MarshaledAggStates[i]); err != nil { + logutil.Error("SpillableAggState failed to read agg state bytes", + zap.Int("agg_index", i), zap.Error(err)) return err } } + logutil.Debug("SpillableAggState completed deserialization", + zap.Int("group_count", s.GroupCount), + zap.Int("group_vectors_count", len(s.GroupVectors)), + zap.Int("agg_states_count", len(s.MarshaledAggStates))) + return nil } From 11a1a0703718a6714114ee572281a9e5196e4247 Mon Sep 17 00:00:00 2001 From: reus Date: Sun, 28 Sep 2025 22:48:26 +0800 Subject: [PATCH 47/57] fix test --- pkg/sql/colexec/group/testspill/spill_test.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pkg/sql/colexec/group/testspill/spill_test.go b/pkg/sql/colexec/group/testspill/spill_test.go index 3e81229aba3c5..2dbbe44548e94 100644 --- a/pkg/sql/colexec/group/testspill/spill_test.go +++ b/pkg/sql/colexec/group/testspill/spill_test.go @@ -81,8 +81,7 @@ func TestSpill(t *testing.T) { // query var a, b, c int err = db.QueryRow(` - select - count(sha2(product_id * customer_id, 256)), + select count(sha2(product_id * customer_id, 256)) from sales `, ).Scan(&a, &b, &c) From a5b32de3ec92ec376d3e22bd79c86225fd2fae6f Mon Sep 17 00:00:00 2001 From: reus Date: Sun, 28 Sep 2025 22:49:32 +0800 Subject: [PATCH 48/57] fix test --- pkg/sql/colexec/group/testspill/spill_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/sql/colexec/group/testspill/spill_test.go b/pkg/sql/colexec/group/testspill/spill_test.go index 2dbbe44548e94..3b3d0ea84fd90 100644 --- a/pkg/sql/colexec/group/testspill/spill_test.go +++ b/pkg/sql/colexec/group/testspill/spill_test.go @@ -79,13 +79,13 @@ func TestSpill(t *testing.T) { require.NoError(t, err) // query - var a, b, c int + var count int err = db.QueryRow(` select count(sha2(product_id * customer_id, 256)) from sales `, - ).Scan(&a, &b, &c) + ).Scan(&count) require.NoError(t, err) - t.Logf("results: %v %v %v", a, b, c) + t.Logf("count: %v", count) } From e93565c264ab802a2e3c52ef7b7174e5904e1c01 Mon Sep 17 00:00:00 2001 From: reus Date: Sun, 28 Sep 2025 22:53:30 +0800 Subject: [PATCH 49/57] enable debug log --- pkg/sql/colexec/group/testspill/spill_test.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pkg/sql/colexec/group/testspill/spill_test.go b/pkg/sql/colexec/group/testspill/spill_test.go index 3b3d0ea84fd90..cd613f14342c2 100644 --- a/pkg/sql/colexec/group/testspill/spill_test.go +++ b/pkg/sql/colexec/group/testspill/spill_test.go @@ -30,6 +30,9 @@ func TestSpill(t *testing.T) { embed.WithCNCount(3), embed.WithTesting(), embed.WithPreStart(func(service embed.ServiceOperator) { + service.Adjust(func(config *embed.ServiceConfig) { + config.Log.Level = "debug" + }) }), ) require.NoError(t, err) From bcd50ce3a5279a78d522d7cd978c2585bde64767 Mon Sep 17 00:00:00 2001 From: reus Date: Sun, 28 Sep 2025 23:02:31 +0800 Subject: [PATCH 50/57] more logs --- pkg/sql/colexec/group/group_spill.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index 3e588e68576b2..227b2800022a3 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -33,6 +33,13 @@ func (group *Group) shouldSpill() bool { len(group.ctr.result1.AggList) > 0 && len(group.ctr.result1.ToPopped) > 0 + logutil.Debug("shouldSpill", + zap.Any("threshold", group.SpillThreshold), + zap.Any("current usage", group.ctr.currentMemUsage), + zap.Any("agg list len", len(group.ctr.result1.AggList)), + zap.Any("to popped len", len(group.ctr.result1.ToPopped)), + ) + if shouldSpill { logutil.Debug("Group operator triggering spill", zap.Int64("current_memory_usage", group.ctr.currentMemUsage), From c3b5d6b7e66e418fb612edf1ff9ba41f4388cc97 Mon Sep 17 00:00:00 2001 From: reus Date: Sun, 28 Sep 2025 23:10:15 +0800 Subject: [PATCH 51/57] fix sql for test --- pkg/sql/colexec/group/testspill/spill_test.go | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/pkg/sql/colexec/group/testspill/spill_test.go b/pkg/sql/colexec/group/testspill/spill_test.go index cd613f14342c2..cf45eaf7deb1a 100644 --- a/pkg/sql/colexec/group/testspill/spill_test.go +++ b/pkg/sql/colexec/group/testspill/spill_test.go @@ -84,8 +84,12 @@ func TestSpill(t *testing.T) { // query var count int err = db.QueryRow(` - select count(sha2(product_id * customer_id, 256)) - from sales + select count(*) + from ( + select product_id, customer_id, sum(amount) + from sales + group by product_id, customer_id + ) `, ).Scan(&count) require.NoError(t, err) From d3ac292cd277d9761220be14b65e7b9a14c0d647 Mon Sep 17 00:00:00 2001 From: reus Date: Mon, 29 Sep 2025 00:39:51 +0800 Subject: [PATCH 52/57] more spill tests --- pkg/sql/colexec/group/testspill/spill_test.go | 301 +++++++++++++++--- 1 file changed, 263 insertions(+), 38 deletions(-) diff --git a/pkg/sql/colexec/group/testspill/spill_test.go b/pkg/sql/colexec/group/testspill/spill_test.go index cf45eaf7deb1a..9b230d915d4d3 100644 --- a/pkg/sql/colexec/group/testspill/spill_test.go +++ b/pkg/sql/colexec/group/testspill/spill_test.go @@ -23,11 +23,155 @@ import ( "github.com/stretchr/testify/require" ) -func TestSpill(t *testing.T) { +func TestGroupSpillLargeGroups(t *testing.T) { + cluster, err := embed.NewCluster( + embed.WithCNCount(1), + embed.WithTesting(), + embed.WithPreStart(func(service embed.ServiceOperator) { + service.Adjust(func(config *embed.ServiceConfig) { + config.Log.Level = "debug" + }) + }), + ) + require.NoError(t, err) + err = cluster.Start() + require.NoError(t, err) + defer cluster.Close() + + cn0, err := cluster.GetCNService(0) + require.NoError(t, err) + dsn := fmt.Sprintf("dump:111@tcp(127.0.0.1:%d)/", + cn0.GetServiceConfig().CN.Frontend.Port, + ) + + db, err := sql.Open("mysql", dsn) + require.NoError(t, err) + defer db.Close() + + _, err = db.Exec(`CREATE DATABASE IF NOT EXISTS test_spill`) + require.NoError(t, err) + + _, err = db.Exec(`USE test_spill`) + require.NoError(t, err) + + _, err = db.Exec(` + CREATE TABLE IF NOT EXISTS large_group_test ( + id BIGINT PRIMARY KEY, + group_col1 INT, + group_col2 VARCHAR(100), + value_col BIGINT + ) + `) + require.NoError(t, err) + + _, err = db.Exec(` + INSERT INTO large_group_test (id, group_col1, group_col2, value_col) + SELECT + g.result as id, + FLOOR(RAND() * 100000) as group_col1, -- 100k distinct values + CONCAT('group_', FLOOR(RAND() * 50000)) as group_col2, -- 50k distinct values + FLOOR(RAND() * 1000) as value_col + FROM generate_series(1000000) g -- 1M rows creating many distinct combinations + `) + require.NoError(t, err) + + rows, err := db.Query(` + SELECT + group_col1, + group_col2, + COUNT(*) as cnt, + SUM(value_col) as sum_val + FROM large_group_test + GROUP BY group_col1, group_col2 + ORDER BY cnt DESC + LIMIT 10 + `) + require.NoError(t, err) + defer rows.Close() + + count := 0 + for rows.Next() { + count++ + } + require.NoError(t, rows.Err()) + t.Logf("Successfully processed %d results from spilled group by operation", count) +} + +func TestGroupSpillWithStrings(t *testing.T) { + cluster, err := embed.NewCluster( + embed.WithCNCount(1), + embed.WithTesting(), + embed.WithPreStart(func(service embed.ServiceOperator) { + service.Adjust(func(config *embed.ServiceConfig) { + config.Log.Level = "debug" + }) + }), + ) + require.NoError(t, err) + err = cluster.Start() + require.NoError(t, err) + defer cluster.Close() + + cn0, err := cluster.GetCNService(0) + require.NoError(t, err) + dsn := fmt.Sprintf("dump:111@tcp(127.0.0.1:%d)/", + cn0.GetServiceConfig().CN.Frontend.Port, + ) + + db, err := sql.Open("mysql", dsn) + require.NoError(t, err) + defer db.Close() + + _, err = db.Exec(`CREATE DATABASE IF NOT EXISTS test_spill`) + require.NoError(t, err) + + _, err = db.Exec(`USE test_spill`) + require.NoError(t, err) + + _, err = db.Exec(` + CREATE TABLE IF NOT EXISTS string_group_test ( + id BIGINT PRIMARY KEY, + category VARCHAR(50), + description TEXT + ) + `) + require.NoError(t, err) + + _, err = db.Exec(` + INSERT INTO string_group_test (id, category, description) + SELECT + g.result as id, + CONCAT('category_', FLOOR(RAND() * 10000)) as category, -- 10k categories + CONCAT('description_', g.result, '_long_text_data_for_spill_testing') as description + FROM generate_series(500000) g -- 500k rows + `) + require.NoError(t, err) + + rows, err := db.Query(` + SELECT + category, + GROUP_CONCAT(description ORDER BY id SEPARATOR ', ') as concatenated_desc, + COUNT(*) as cnt + FROM string_group_test + GROUP BY category + HAVING cnt > 1 + ORDER BY cnt DESC + LIMIT 5 + `) + require.NoError(t, err) + defer rows.Close() - // start cluster + count := 0 + for rows.Next() { + count++ + } + require.NoError(t, rows.Err()) + t.Logf("Successfully processed %d results from string group concat spill operation", count) +} + +func TestGroupSpillApproxCountDistinct(t *testing.T) { cluster, err := embed.NewCluster( - embed.WithCNCount(3), + embed.WithCNCount(1), embed.WithTesting(), embed.WithPreStart(func(service embed.ServiceOperator) { service.Adjust(func(config *embed.ServiceConfig) { @@ -50,49 +194,130 @@ func TestSpill(t *testing.T) { require.NoError(t, err) defer db.Close() - // database - _, err = db.Exec(` create database test `) + _, err = db.Exec(`CREATE DATABASE IF NOT EXISTS test_spill`) + require.NoError(t, err) + + _, err = db.Exec(`USE test_spill`) require.NoError(t, err) - // table _, err = db.Exec(` - use test; - create table sales ( - id int, - product_id int, - customer_id int, - sale_date date, - amount decimal(10, 2) - ); - `) - require.NoError(t, err) - - // data + CREATE TABLE IF NOT EXISTS approx_count_test ( + id BIGINT PRIMARY KEY, + group_col VARCHAR(100), + value_col BIGINT + ) + `) + require.NoError(t, err) + _, err = db.Exec(` - use test; - insert into sales (id, product_id, customer_id, sale_date, amount) - select g.result as id, - floor(1 + (rand() * 100000)) as product_id, - floor(1 + (rand() * 100000)) as customer_id, - current_date - interval floor(rand() * 365) day as sale_date, - floor(rand() * 1000) as amount - from generate_series(2000 * 10000) g - `, + INSERT INTO approx_count_test (id, group_col, value_col) + SELECT + g.result as id, + CONCAT('group_', FLOOR(RAND() * 5000)) as group_col, -- 5k groups + FLOOR(RAND() * 1000000) as value_col -- many distinct values per group + FROM generate_series(800000) g -- 800k rows + `) + require.NoError(t, err) + + rows, err := db.Query(` + SELECT + group_col, + APPROX_COUNT_DISTINCT(value_col) as approx_distinct_count, + COUNT(*) as total_count + FROM approx_count_test + GROUP BY group_col + ORDER BY approx_distinct_count DESC + LIMIT 10 + `) + require.NoError(t, err) + defer rows.Close() + + count := 0 + for rows.Next() { + count++ + } + require.NoError(t, rows.Err()) + t.Logf("Successfully processed %d results from approx count distinct spill operation", count) +} + +func TestGroupSpillMixedAggregations(t *testing.T) { + cluster, err := embed.NewCluster( + embed.WithCNCount(1), + embed.WithTesting(), + embed.WithPreStart(func(service embed.ServiceOperator) { + service.Adjust(func(config *embed.ServiceConfig) { + config.Log.Level = "debug" + }) + }), ) require.NoError(t, err) + err = cluster.Start() + require.NoError(t, err) + defer cluster.Close() + + cn0, err := cluster.GetCNService(0) + require.NoError(t, err) + dsn := fmt.Sprintf("dump:111@tcp(127.0.0.1:%d)/", + cn0.GetServiceConfig().CN.Frontend.Port, + ) - // query - var count int - err = db.QueryRow(` - select count(*) - from ( - select product_id, customer_id, sum(amount) - from sales - group by product_id, customer_id + db, err := sql.Open("mysql", dsn) + require.NoError(t, err) + defer db.Close() + + _, err = db.Exec(`CREATE DATABASE IF NOT EXISTS test_spill`) + require.NoError(t, err) + + _, err = db.Exec(`USE test_spill`) + require.NoError(t, err) + + _, err = db.Exec(` + CREATE TABLE IF NOT EXISTS mixed_agg_test ( + id BIGINT PRIMARY KEY, + dept VARCHAR(50), + product VARCHAR(100), + sales_amount DECIMAL(15,2), + quantity INT, + sale_date DATE ) - `, - ).Scan(&count) + `) + require.NoError(t, err) + + _, err = db.Exec(` + INSERT INTO mixed_agg_test (id, dept, product, sales_amount, quantity, sale_date) + SELECT + g.result as id, + CONCAT('dept_', FLOOR(RAND() * 2000)) as dept, -- 2k departments + CONCAT('product_', FLOOR(RAND() * 50000)) as product, -- 50k products + RAND() * 1000 as sales_amount, + FLOOR(RAND() * 100) as quantity, + DATE_ADD('2023-01-01', INTERVAL FLOOR(RAND() * 365) DAY) as sale_date + FROM generate_series(600000) g -- 600k rows + `) + require.NoError(t, err) + + rows, err := db.Query(` + SELECT + dept, + product, + COUNT(*) as transaction_count, + SUM(sales_amount) as total_sales, + AVG(sales_amount) as avg_sales, + MIN(sales_amount) as min_sales, + MAX(sales_amount) as max_sales, + SUM(quantity) as total_quantity + FROM mixed_agg_test + GROUP BY dept, product + ORDER BY total_sales DESC + LIMIT 10 + `) require.NoError(t, err) - t.Logf("count: %v", count) + defer rows.Close() + count := 0 + for rows.Next() { + count++ + } + require.NoError(t, rows.Err()) + t.Logf("Successfully processed %d results from mixed aggregations spill operation", count) } From feafc25924ded758c56abcef4efe60b98e5b8d9a Mon Sep 17 00:00:00 2001 From: reus Date: Tue, 30 Sep 2025 13:59:16 +0800 Subject: [PATCH 53/57] fix agg state serialize --- pkg/sql/colexec/group/spillable_agg_state.go | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/pkg/sql/colexec/group/spillable_agg_state.go b/pkg/sql/colexec/group/spillable_agg_state.go index 130e0f4e670ae..a66fd3a92fb35 100644 --- a/pkg/sql/colexec/group/spillable_agg_state.go +++ b/pkg/sql/colexec/group/spillable_agg_state.go @@ -54,18 +54,18 @@ func (s *SpillableAggState) Serialize() ([]byte, error) { logutil.Error("SpillableAggState failed to write group vector types count", zap.Error(err)) return nil, err } - for _, typ := range s.GroupVectorTypes { + for i, typ := range s.GroupVectorTypes { typBytes, err := typ.MarshalBinary() if err != nil { - logutil.Error("SpillableAggState failed to marshal vector type", zap.Error(err)) + logutil.Error("SpillableAggState failed to marshal vector type", zap.Int("type_index", i), zap.Error(err)) return nil, err } if err := binary.Write(buf, binary.LittleEndian, int32(len(typBytes))); err != nil { - logutil.Error("SpillableAggState failed to write type bytes length", zap.Error(err)) + logutil.Error("SpillableAggState failed to write type bytes length", zap.Int("type_index", i), zap.Error(err)) return nil, err } if _, err := buf.Write(typBytes); err != nil { - logutil.Error("SpillableAggState failed to write type bytes", zap.Error(err)) + logutil.Error("SpillableAggState failed to write type bytes", zap.Int("type_index", i), zap.Error(err)) return nil, err } } @@ -96,10 +96,6 @@ func (s *SpillableAggState) Serialize() ([]byte, error) { zap.Int("vec_index", i), zap.Error(err)) return nil, err } - - if i >= len(s.GroupVectorTypes) { - s.GroupVectorTypes = append(s.GroupVectorTypes, *vec.GetType()) - } } if err := binary.Write(buf, binary.LittleEndian, int32(len(s.MarshaledAggStates))); err != nil { From c61f5ee9b5242cccfba6fe8cc4f4775fe33e9aaa Mon Sep 17 00:00:00 2001 From: reus Date: Tue, 30 Sep 2025 14:04:28 +0800 Subject: [PATCH 54/57] fix agg state deserialize --- pkg/sql/colexec/group/spillable_agg_state.go | 4 +--- pkg/sql/colexec/group/testspill/spill_test.go | 6 +++--- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/pkg/sql/colexec/group/spillable_agg_state.go b/pkg/sql/colexec/group/spillable_agg_state.go index a66fd3a92fb35..2ab78bb707057 100644 --- a/pkg/sql/colexec/group/spillable_agg_state.go +++ b/pkg/sql/colexec/group/spillable_agg_state.go @@ -187,11 +187,9 @@ func (s *SpillableAggState) Deserialize(data []byte, mp *mpool.MPool) error { return err } - var vecType types.Type + vecType := types.T_any.ToType() if i < len(s.GroupVectorTypes) { vecType = s.GroupVectorTypes[i] - } else { - vecType = types.T_any.ToType() } vec := vector.NewOffHeapVecWithType(vecType) diff --git a/pkg/sql/colexec/group/testspill/spill_test.go b/pkg/sql/colexec/group/testspill/spill_test.go index 9b230d915d4d3..33b8db14cb896 100644 --- a/pkg/sql/colexec/group/testspill/spill_test.go +++ b/pkg/sql/colexec/group/testspill/spill_test.go @@ -68,10 +68,10 @@ func TestGroupSpillLargeGroups(t *testing.T) { INSERT INTO large_group_test (id, group_col1, group_col2, value_col) SELECT g.result as id, - FLOOR(RAND() * 100000) as group_col1, -- 100k distinct values - CONCAT('group_', FLOOR(RAND() * 50000)) as group_col2, -- 50k distinct values + FLOOR(RAND() * 100000) as group_col1, + CONCAT('group_', FLOOR(RAND() * 50000)) as group_col2, FLOOR(RAND() * 1000) as value_col - FROM generate_series(1000000) g -- 1M rows creating many distinct combinations + FROM generate_series(1000000) g `) require.NoError(t, err) From 46d68c16fe267e201a17dec4de4cf85ff001dec9 Mon Sep 17 00:00:00 2001 From: reus Date: Tue, 30 Sep 2025 15:49:32 +0800 Subject: [PATCH 55/57] use logutil.Info --- pkg/sql/colexec/group/exec.go | 2 +- pkg/sql/colexec/group/group_spill.go | 30 ++++++++++---------- pkg/sql/colexec/group/spill_memory.go | 12 ++++---- pkg/sql/colexec/group/spillable_agg_state.go | 8 +++--- 4 files changed, 26 insertions(+), 26 deletions(-) diff --git a/pkg/sql/colexec/group/exec.go b/pkg/sql/colexec/group/exec.go index 48d704b737fee..9cae14e25ad37 100644 --- a/pkg/sql/colexec/group/exec.go +++ b/pkg/sql/colexec/group/exec.go @@ -68,7 +68,7 @@ func (group *Group) Prepare(proc *process.Process) (err error) { group.SpillManager = NewMemorySpillManager() } if group.SpillThreshold <= 0 { - group.SpillThreshold = 64 * 1024 * 1024 + group.SpillThreshold = 64 * 1024 * 1024 //TODO configurable } return group.PrepareProjection(proc) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index 227b2800022a3..27afa55d3f3f6 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -33,7 +33,7 @@ func (group *Group) shouldSpill() bool { len(group.ctr.result1.AggList) > 0 && len(group.ctr.result1.ToPopped) > 0 - logutil.Debug("shouldSpill", + logutil.Info("shouldSpill", zap.Any("threshold", group.SpillThreshold), zap.Any("current usage", group.ctr.currentMemUsage), zap.Any("agg list len", len(group.ctr.result1.AggList)), @@ -41,7 +41,7 @@ func (group *Group) shouldSpill() bool { ) if shouldSpill { - logutil.Debug("Group operator triggering spill", + logutil.Info("Group operator triggering spill", zap.Int64("current_memory_usage", group.ctr.currentMemUsage), zap.Int64("spill_threshold", group.SpillThreshold), zap.Int("agg_count", len(group.ctr.result1.AggList)), @@ -74,7 +74,7 @@ func (group *Group) updateMemoryUsage(proc *process.Process) { group.ctr.currentMemUsage = usage if usage > previousUsage && usage > group.SpillThreshold/2 { - logutil.Debug("Group operator memory usage update", + logutil.Info("Group operator memory usage update", zap.Int64("previous_usage", previousUsage), zap.Int64("current_usage", usage), zap.Int64("spill_threshold", group.SpillThreshold)) @@ -83,7 +83,7 @@ func (group *Group) updateMemoryUsage(proc *process.Process) { func (group *Group) spillPartialResults(proc *process.Process) error { if len(group.ctr.result1.AggList) == 0 || len(group.ctr.result1.ToPopped) == 0 { - logutil.Debug("Group operator spill called but no data to spill") + logutil.Info("Group operator spill called but no data to spill") return nil } @@ -113,7 +113,7 @@ func (group *Group) spillPartialResults(proc *process.Process) error { } if totalGroups == 0 { - logutil.Debug("Group operator spill found no groups to spill") + logutil.Info("Group operator spill found no groups to spill") for _, agg := range group.ctr.result1.AggList { if agg != nil { agg.Free() @@ -200,7 +200,7 @@ func (group *Group) spillPartialResults(proc *process.Process) error { } group.ctr.currentMemUsage = 0 - logutil.Debug("Group operator completed spill cleanup", + logutil.Info("Group operator completed spill cleanup", zap.Int("spilled_states_count", len(group.ctr.spilledStates))) return nil } @@ -214,7 +214,7 @@ func (group *Group) mergeSpilledResults(proc *process.Process) error { zap.Int("spilled_states_count", len(group.ctr.spilledStates))) for i, spillID := range group.ctr.spilledStates { - logutil.Debug("Group operator merging spilled state", + logutil.Info("Group operator merging spilled state", zap.Int("state_index", i), zap.String("spill_id", string(spillID))) @@ -233,7 +233,7 @@ func (group *Group) mergeSpilledResults(proc *process.Process) error { panic(fmt.Sprintf("invalid spilled data type")) } - logutil.Debug("Group operator retrieved spilled state", + logutil.Info("Group operator retrieved spilled state", zap.String("spill_id", string(spillID)), zap.Int("group_count", spillState.GroupCount), zap.Int64("estimated_size", spillState.EstimateSize())) @@ -252,7 +252,7 @@ func (group *Group) mergeSpilledResults(proc *process.Process) error { return err } - logutil.Debug("Group operator completed merge of spilled state", + logutil.Info("Group operator completed merge of spilled state", zap.String("spill_id", string(spillID))) } @@ -265,16 +265,16 @@ func (group *Group) mergeSpilledResults(proc *process.Process) error { func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spillState *SpillableAggState) error { if len(spillState.MarshaledAggStates) == 0 { - logutil.Debug("Group operator restore found no marshaled aggregator states") + logutil.Info("Group operator restore found no marshaled aggregator states") return nil } - logutil.Debug("Group operator restoring spilled aggregators", + logutil.Info("Group operator restoring spilled aggregators", zap.Int("agg_states_count", len(spillState.MarshaledAggStates)), zap.Int("group_count", spillState.GroupCount)) if len(group.ctr.result1.AggList) == 0 { - logutil.Debug("Group operator initializing aggregators from spilled state") + logutil.Info("Group operator initializing aggregators from spilled state") aggs := make([]aggexec.AggFuncExec, len(spillState.MarshaledAggStates)) defer func() { if group.ctr.result1.AggList == nil { @@ -318,7 +318,7 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi group.ctr.result1.ChunkSize = chunkSize group.ctr.result1.AggList = aggs - logutil.Debug("Group operator initialized aggregators from spilled state", + logutil.Info("Group operator initialized aggregators from spilled state", zap.Int("chunk_size", chunkSize), zap.Int("agg_count", len(aggs))) @@ -354,7 +354,7 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi return nil } - logutil.Debug("Group operator merging spilled aggregators with existing ones", + logutil.Info("Group operator merging spilled aggregators with existing ones", zap.Int("existing_agg_count", len(group.ctr.result1.AggList)), zap.Int("spilled_group_count", spillState.GroupCount)) @@ -466,7 +466,7 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi group.ctr.result1.ToPopped = append(group.ctr.result1.ToPopped, batchesToAdd...) } - logutil.Debug("Group operator completed restore and merge of spilled aggregators", + logutil.Info("Group operator completed restore and merge of spilled aggregators", zap.Int("final_batch_count", len(group.ctr.result1.ToPopped))) return nil diff --git a/pkg/sql/colexec/group/spill_memory.go b/pkg/sql/colexec/group/spill_memory.go index d1736d01df21f..7fe4b88c5025c 100644 --- a/pkg/sql/colexec/group/spill_memory.go +++ b/pkg/sql/colexec/group/spill_memory.go @@ -50,7 +50,7 @@ func (m *MemorySpillManager) Spill(data SpillableData) (SpillID, error) { m.data[id] = serialized newTotalMem := atomic.AddInt64(&m.totalMem, int64(len(serialized))) - logutil.Debug("MemorySpillManager spilled data", + logutil.Info("MemorySpillManager spilled data", zap.String("spill_id", string(id)), zap.Int("data_size", len(serialized)), zap.Int64("total_memory", newTotalMem)) @@ -69,7 +69,7 @@ func (m *MemorySpillManager) Retrieve(id SpillID, mp *mpool.MPool) (SpillableDat return nil, fmt.Errorf("spill data not found: %s", id) } - logutil.Debug("MemorySpillManager retrieving spilled data", + logutil.Info("MemorySpillManager retrieving spilled data", zap.String("spill_id", string(id)), zap.Int("data_size", len(serialized))) @@ -81,7 +81,7 @@ func (m *MemorySpillManager) Retrieve(id SpillID, mp *mpool.MPool) (SpillableDat return nil, err } - logutil.Debug("MemorySpillManager successfully retrieved and deserialized data", + logutil.Info("MemorySpillManager successfully retrieved and deserialized data", zap.String("spill_id", string(id)), zap.Int64("estimated_size", data.EstimateSize())) @@ -96,7 +96,7 @@ func (m *MemorySpillManager) Delete(id SpillID) error { newTotalMem := atomic.AddInt64(&m.totalMem, -int64(len(serialized))) delete(m.data, id) - logutil.Debug("MemorySpillManager deleted spilled data", + logutil.Info("MemorySpillManager deleted spilled data", zap.String("spill_id", string(id)), zap.Int("data_size", len(serialized)), zap.Int64("total_memory", newTotalMem)) @@ -114,7 +114,7 @@ func (m *MemorySpillManager) Free() { count := len(m.data) totalSize := atomic.LoadInt64(&m.totalMem) - logutil.Debug("MemorySpillManager freeing all spilled data", + logutil.Info("MemorySpillManager freeing all spilled data", zap.Int("spilled_count", count), zap.Int64("total_size", totalSize)) @@ -123,7 +123,7 @@ func (m *MemorySpillManager) Free() { } m.data = nil - logutil.Debug("MemorySpillManager completed cleanup") + logutil.Info("MemorySpillManager completed cleanup") } func (m *MemorySpillManager) TotalMem() int64 { diff --git a/pkg/sql/colexec/group/spillable_agg_state.go b/pkg/sql/colexec/group/spillable_agg_state.go index 2ab78bb707057..cf57687a8aa40 100644 --- a/pkg/sql/colexec/group/spillable_agg_state.go +++ b/pkg/sql/colexec/group/spillable_agg_state.go @@ -33,7 +33,7 @@ type SpillableAggState struct { } func (s *SpillableAggState) Serialize() ([]byte, error) { - logutil.Debug("SpillableAggState starting serialization", + logutil.Info("SpillableAggState starting serialization", zap.Int("group_count", s.GroupCount), zap.Int("group_vectors_count", len(s.GroupVectors)), zap.Int("agg_states_count", len(s.MarshaledAggStates))) @@ -116,14 +116,14 @@ func (s *SpillableAggState) Serialize() ([]byte, error) { } result := buf.Bytes() - logutil.Debug("SpillableAggState completed serialization", + logutil.Info("SpillableAggState completed serialization", zap.Int("serialized_size", len(result))) return result, nil } func (s *SpillableAggState) Deserialize(data []byte, mp *mpool.MPool) error { - logutil.Debug("SpillableAggState starting deserialization", + logutil.Info("SpillableAggState starting deserialization", zap.Int("data_size", len(data))) buf := bytes.NewReader(data) @@ -223,7 +223,7 @@ func (s *SpillableAggState) Deserialize(data []byte, mp *mpool.MPool) error { } } - logutil.Debug("SpillableAggState completed deserialization", + logutil.Info("SpillableAggState completed deserialization", zap.Int("group_count", s.GroupCount), zap.Int("group_vectors_count", len(s.GroupVectors)), zap.Int("agg_states_count", len(s.MarshaledAggStates))) From c2aabd9962a67457b4f12b0af96b10722b8bf5cc Mon Sep 17 00:00:00 2001 From: reus Date: Tue, 30 Sep 2025 15:53:44 +0800 Subject: [PATCH 56/57] update --- pkg/sql/colexec/group/exec.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/sql/colexec/group/exec.go b/pkg/sql/colexec/group/exec.go index 9cae14e25ad37..0b52f45df11ee 100644 --- a/pkg/sql/colexec/group/exec.go +++ b/pkg/sql/colexec/group/exec.go @@ -68,7 +68,7 @@ func (group *Group) Prepare(proc *process.Process) (err error) { group.SpillManager = NewMemorySpillManager() } if group.SpillThreshold <= 0 { - group.SpillThreshold = 64 * 1024 * 1024 //TODO configurable + group.SpillThreshold = 256 * 1024 //TODO configurable } return group.PrepareProjection(proc) From 1d48563d958dd27aee573be84a217e96f5b54fa4 Mon Sep 17 00:00:00 2001 From: reus Date: Tue, 30 Sep 2025 16:33:54 +0800 Subject: [PATCH 57/57] clean up logs --- pkg/sql/colexec/group/group_spill.go | 68 ++++++++++---------- pkg/sql/colexec/group/spill_memory.go | 20 +++--- pkg/sql/colexec/group/spillable_agg_state.go | 60 +---------------- 3 files changed, 47 insertions(+), 101 deletions(-) diff --git a/pkg/sql/colexec/group/group_spill.go b/pkg/sql/colexec/group/group_spill.go index 27afa55d3f3f6..f397e0f4cb311 100644 --- a/pkg/sql/colexec/group/group_spill.go +++ b/pkg/sql/colexec/group/group_spill.go @@ -33,7 +33,7 @@ func (group *Group) shouldSpill() bool { len(group.ctr.result1.AggList) > 0 && len(group.ctr.result1.ToPopped) > 0 - logutil.Info("shouldSpill", + logutil.Info("[SPILL] shouldSpill", zap.Any("threshold", group.SpillThreshold), zap.Any("current usage", group.ctr.currentMemUsage), zap.Any("agg list len", len(group.ctr.result1.AggList)), @@ -41,7 +41,7 @@ func (group *Group) shouldSpill() bool { ) if shouldSpill { - logutil.Info("Group operator triggering spill", + logutil.Info("[SPILL] Group operator triggering spill", zap.Int64("current_memory_usage", group.ctr.currentMemUsage), zap.Int64("spill_threshold", group.SpillThreshold), zap.Int("agg_count", len(group.ctr.result1.AggList)), @@ -74,7 +74,7 @@ func (group *Group) updateMemoryUsage(proc *process.Process) { group.ctr.currentMemUsage = usage if usage > previousUsage && usage > group.SpillThreshold/2 { - logutil.Info("Group operator memory usage update", + logutil.Info("[SPILL] Group operator memory usage update", zap.Int64("previous_usage", previousUsage), zap.Int64("current_usage", usage), zap.Int64("spill_threshold", group.SpillThreshold)) @@ -83,11 +83,11 @@ func (group *Group) updateMemoryUsage(proc *process.Process) { func (group *Group) spillPartialResults(proc *process.Process) error { if len(group.ctr.result1.AggList) == 0 || len(group.ctr.result1.ToPopped) == 0 { - logutil.Info("Group operator spill called but no data to spill") + logutil.Info("[SPILL] Group operator spill called but no data to spill") return nil } - logutil.Info("Group operator starting spill operation", + logutil.Info("[SPILL] Group operator starting spill operation", zap.Int64("memory_usage", group.ctr.currentMemUsage), zap.Int64("spill_threshold", group.SpillThreshold), zap.Int("agg_count", len(group.ctr.result1.AggList))) @@ -97,7 +97,7 @@ func (group *Group) spillPartialResults(proc *process.Process) error { if agg != nil { marshaledData, err := aggexec.MarshalAggFuncExec(agg) if err != nil { - logutil.Error("Group operator failed to marshal aggregator", + logutil.Error("[SPILL] Group operator failed to marshal aggregator", zap.Int("agg_index", i), zap.Error(err)) return err } @@ -113,7 +113,7 @@ func (group *Group) spillPartialResults(proc *process.Process) error { } if totalGroups == 0 { - logutil.Info("Group operator spill found no groups to spill") + logutil.Info("[SPILL] Group operator spill found no groups to spill") for _, agg := range group.ctr.result1.AggList { if agg != nil { agg.Free() @@ -144,7 +144,7 @@ func (group *Group) spillPartialResults(proc *process.Process) error { for i, vec := range bat.Vecs { if i < len(groupVecs) && groupVecs[i] != nil && vec != nil { if err := groupVecs[i].UnionBatch(vec, 0, vec.Length(), nil, proc.Mp()); err != nil { - logutil.Error("Group operator failed to union batch during spill", + logutil.Error("[SPILL] Group operator failed to union batch during spill", zap.Int("vec_index", i), zap.Error(err)) for j := range groupVecs { if groupVecs[j] != nil { @@ -168,12 +168,12 @@ func (group *Group) spillPartialResults(proc *process.Process) error { spillID, err := group.SpillManager.Spill(spillData) if err != nil { - logutil.Error("Group operator failed to spill data", zap.Error(err)) + logutil.Error("[SPILL] Group operator failed to spill data", zap.Error(err)) spillData.Free(proc.Mp()) return err } - logutil.Info("Group operator successfully spilled data", + logutil.Info("[SPILL] Group operator successfully spilled data", zap.String("spill_id", string(spillID)), zap.Int("total_groups", totalGroups), zap.Int64("estimated_size", spillData.EstimateSize())) @@ -200,7 +200,7 @@ func (group *Group) spillPartialResults(proc *process.Process) error { } group.ctr.currentMemUsage = 0 - logutil.Info("Group operator completed spill cleanup", + logutil.Info("[SPILL] Group operator completed spill cleanup", zap.Int("spilled_states_count", len(group.ctr.spilledStates))) return nil } @@ -210,36 +210,36 @@ func (group *Group) mergeSpilledResults(proc *process.Process) error { return nil } - logutil.Info("Group operator starting merge of spilled results", + logutil.Info("[SPILL] Group operator starting merge of spilled results", zap.Int("spilled_states_count", len(group.ctr.spilledStates))) for i, spillID := range group.ctr.spilledStates { - logutil.Info("Group operator merging spilled state", + logutil.Info("[SPILL] Group operator merging spilled state", zap.Int("state_index", i), zap.String("spill_id", string(spillID))) spillData, err := group.SpillManager.Retrieve(spillID, proc.Mp()) if err != nil { - logutil.Error("Group operator failed to retrieve spilled data", + logutil.Error("[SPILL] Group operator failed to retrieve spilled data", zap.String("spill_id", string(spillID)), zap.Error(err)) return err } spillState, ok := spillData.(*SpillableAggState) if !ok { - logutil.Error("Group operator retrieved invalid spilled data type", + logutil.Error("[SPILL] Group operator retrieved invalid spilled data type", zap.String("spill_id", string(spillID))) spillData.Free(proc.Mp()) panic(fmt.Sprintf("invalid spilled data type")) } - logutil.Info("Group operator retrieved spilled state", + logutil.Info("[SPILL] Group operator retrieved spilled state", zap.String("spill_id", string(spillID)), zap.Int("group_count", spillState.GroupCount), zap.Int64("estimated_size", spillState.EstimateSize())) if err = group.restoreAndMergeSpilledAggregators(proc, spillState); err != nil { - logutil.Error("Group operator failed to restore and merge spilled aggregators", + logutil.Error("[SPILL] Group operator failed to restore and merge spilled aggregators", zap.String("spill_id", string(spillID)), zap.Error(err)) spillState.Free(proc.Mp()) return err @@ -247,16 +247,16 @@ func (group *Group) mergeSpilledResults(proc *process.Process) error { spillState.Free(proc.Mp()) if err = group.SpillManager.Delete(spillID); err != nil { - logutil.Error("Group operator failed to delete spilled data", + logutil.Error("[SPILL] Group operator failed to delete spilled data", zap.String("spill_id", string(spillID)), zap.Error(err)) return err } - logutil.Info("Group operator completed merge of spilled state", + logutil.Info("[SPILL] Group operator completed merge of spilled state", zap.String("spill_id", string(spillID))) } - logutil.Info("Group operator completed merge of all spilled results", + logutil.Info("[SPILL] Group operator completed merge of all spilled results", zap.Int("merged_states_count", len(group.ctr.spilledStates))) group.ctr.spilledStates = nil @@ -265,16 +265,16 @@ func (group *Group) mergeSpilledResults(proc *process.Process) error { func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spillState *SpillableAggState) error { if len(spillState.MarshaledAggStates) == 0 { - logutil.Info("Group operator restore found no marshaled aggregator states") + logutil.Info("[SPILL] Group operator restore found no marshaled aggregator states") return nil } - logutil.Info("Group operator restoring spilled aggregators", + logutil.Info("[SPILL] Group operator restoring spilled aggregators", zap.Int("agg_states_count", len(spillState.MarshaledAggStates)), zap.Int("group_count", spillState.GroupCount)) if len(group.ctr.result1.AggList) == 0 { - logutil.Info("Group operator initializing aggregators from spilled state") + logutil.Info("[SPILL] Group operator initializing aggregators from spilled state") aggs := make([]aggexec.AggFuncExec, len(spillState.MarshaledAggStates)) defer func() { if group.ctr.result1.AggList == nil { @@ -293,7 +293,7 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi agg, err := aggexec.UnmarshalAggFuncExec(aggexec.NewSimpleAggMemoryManager(proc.Mp()), marshaledState) if err != nil { - logutil.Error("Group operator failed to unmarshal aggregator", + logutil.Error("[SPILL] Group operator failed to unmarshal aggregator", zap.Int("agg_index", i), zap.Error(err)) return err } @@ -302,7 +302,7 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi aggExpr := group.Aggs[i] if config := aggExpr.GetExtraConfig(); config != nil { if err = agg.SetExtraInformation(config, 0); err != nil { - logutil.Error("Group operator failed to set extra information for aggregator", + logutil.Error("[SPILL] Group operator failed to set extra information for aggregator", zap.Int("agg_index", i), zap.Error(err)) agg.Free() return err @@ -318,7 +318,7 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi group.ctr.result1.ChunkSize = chunkSize group.ctr.result1.AggList = aggs - logutil.Info("Group operator initialized aggregators from spilled state", + logutil.Info("[SPILL] Group operator initialized aggregators from spilled state", zap.Int("chunk_size", chunkSize), zap.Int("agg_count", len(aggs))) @@ -335,7 +335,7 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi for i, vec := range spillState.GroupVectors { if vec != nil && i < len(bat.Vecs) { if err := bat.Vecs[i].UnionBatch(vec, int64(offset), size, nil, proc.Mp()); err != nil { - logutil.Error("Group operator failed to union batch during restore", + logutil.Error("[SPILL] Group operator failed to union batch during restore", zap.Int("vec_index", i), zap.Int("offset", offset), zap.Error(err)) bat.Clean(proc.Mp()) for _, b := range batchesToAdd { @@ -354,14 +354,14 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi return nil } - logutil.Info("Group operator merging spilled aggregators with existing ones", + logutil.Info("[SPILL] Group operator merging spilled aggregators with existing ones", zap.Int("existing_agg_count", len(group.ctr.result1.AggList)), zap.Int("spilled_group_count", spillState.GroupCount)) for _, currentAgg := range group.ctr.result1.AggList { if currentAgg != nil { if err := currentAgg.GroupGrow(spillState.GroupCount); err != nil { - logutil.Error("Group operator failed to grow aggregator groups", zap.Error(err)) + logutil.Error("[SPILL] Group operator failed to grow aggregator groups", zap.Error(err)) return err } } @@ -383,7 +383,7 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi agg, err := aggexec.UnmarshalAggFuncExec(aggexec.NewSimpleAggMemoryManager(proc.Mp()), marshaledState) if err != nil { - logutil.Error("Group operator failed to unmarshal aggregator for merge", + logutil.Error("[SPILL] Group operator failed to unmarshal aggregator for merge", zap.Int("agg_index", i), zap.Error(err)) return err } @@ -392,7 +392,7 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi aggExpr := group.Aggs[i] if config := aggExpr.GetExtraConfig(); config != nil { if err = agg.SetExtraInformation(config, 0); err != nil { - logutil.Error("Group operator failed to set extra information for temp aggregator", + logutil.Error("[SPILL] Group operator failed to set extra information for temp aggregator", zap.Int("agg_index", i), zap.Error(err)) agg.Free() return err @@ -423,7 +423,7 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi for spilledGroupIdx := 0; spilledGroupIdx < spillState.GroupCount; spilledGroupIdx++ { currentGroupIdx := currentGroupCount + spilledGroupIdx if err := currentAgg.Merge(tempAgg, currentGroupIdx, spilledGroupIdx); err != nil { - logutil.Error("Group operator failed to merge aggregator groups", + logutil.Error("[SPILL] Group operator failed to merge aggregator groups", zap.Int("agg_index", i), zap.Int("current_group_idx", currentGroupIdx), zap.Int("spilled_group_idx", spilledGroupIdx), @@ -450,7 +450,7 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi for i, vec := range spillState.GroupVectors { if vec != nil && i < len(bat.Vecs) { if err := bat.Vecs[i].UnionBatch(vec, int64(offset), size, nil, proc.Mp()); err != nil { - logutil.Error("Group operator failed to union batch during merge", + logutil.Error("[SPILL] Group operator failed to union batch during merge", zap.Int("vec_index", i), zap.Int("offset", offset), zap.Error(err)) bat.Clean(proc.Mp()) for _, b := range batchesToAdd { @@ -466,7 +466,7 @@ func (group *Group) restoreAndMergeSpilledAggregators(proc *process.Process, spi group.ctr.result1.ToPopped = append(group.ctr.result1.ToPopped, batchesToAdd...) } - logutil.Info("Group operator completed restore and merge of spilled aggregators", + logutil.Info("[SPILL] Group operator completed restore and merge of spilled aggregators", zap.Int("final_batch_count", len(group.ctr.result1.ToPopped))) return nil diff --git a/pkg/sql/colexec/group/spill_memory.go b/pkg/sql/colexec/group/spill_memory.go index 7fe4b88c5025c..38e2c4a76d3a9 100644 --- a/pkg/sql/colexec/group/spill_memory.go +++ b/pkg/sql/colexec/group/spill_memory.go @@ -40,7 +40,7 @@ func NewMemorySpillManager() *MemorySpillManager { func (m *MemorySpillManager) Spill(data SpillableData) (SpillID, error) { serialized, err := data.Serialize() if err != nil { - logutil.Error("MemorySpillManager failed to serialize data", zap.Error(err)) + logutil.Error("[SPILL] MemorySpillManager failed to serialize data", zap.Error(err)) return "", err } @@ -50,7 +50,7 @@ func (m *MemorySpillManager) Spill(data SpillableData) (SpillID, error) { m.data[id] = serialized newTotalMem := atomic.AddInt64(&m.totalMem, int64(len(serialized))) - logutil.Info("MemorySpillManager spilled data", + logutil.Info("[SPILL] MemorySpillManager spilled data", zap.String("spill_id", string(id)), zap.Int("data_size", len(serialized)), zap.Int64("total_memory", newTotalMem)) @@ -64,24 +64,24 @@ func (m *MemorySpillManager) Retrieve(id SpillID, mp *mpool.MPool) (SpillableDat serialized, exists := m.data[id] if !exists { - logutil.Error("MemorySpillManager failed to find spilled data", + logutil.Error("[SPILL] MemorySpillManager failed to find spilled data", zap.String("spill_id", string(id))) return nil, fmt.Errorf("spill data not found: %s", id) } - logutil.Info("MemorySpillManager retrieving spilled data", + logutil.Info("[SPILL] MemorySpillManager retrieving spilled data", zap.String("spill_id", string(id)), zap.Int("data_size", len(serialized))) data := &SpillableAggState{} if err := data.Deserialize(serialized, mp); err != nil { - logutil.Error("MemorySpillManager failed to deserialize data", + logutil.Error("[SPILL] MemorySpillManager failed to deserialize data", zap.String("spill_id", string(id)), zap.Error(err)) data.Free(mp) return nil, err } - logutil.Info("MemorySpillManager successfully retrieved and deserialized data", + logutil.Info("[SPILL] MemorySpillManager successfully retrieved and deserialized data", zap.String("spill_id", string(id)), zap.Int64("estimated_size", data.EstimateSize())) @@ -96,12 +96,12 @@ func (m *MemorySpillManager) Delete(id SpillID) error { newTotalMem := atomic.AddInt64(&m.totalMem, -int64(len(serialized))) delete(m.data, id) - logutil.Info("MemorySpillManager deleted spilled data", + logutil.Info("[SPILL] MemorySpillManager deleted spilled data", zap.String("spill_id", string(id)), zap.Int("data_size", len(serialized)), zap.Int64("total_memory", newTotalMem)) } else { - logutil.Warn("MemorySpillManager attempted to delete non-existent spilled data", + logutil.Warn("[SPILL] MemorySpillManager attempted to delete non-existent spilled data", zap.String("spill_id", string(id))) } return nil @@ -114,7 +114,7 @@ func (m *MemorySpillManager) Free() { count := len(m.data) totalSize := atomic.LoadInt64(&m.totalMem) - logutil.Info("MemorySpillManager freeing all spilled data", + logutil.Info("[SPILL] MemorySpillManager freeing all spilled data", zap.Int("spilled_count", count), zap.Int64("total_size", totalSize)) @@ -123,7 +123,7 @@ func (m *MemorySpillManager) Free() { } m.data = nil - logutil.Info("MemorySpillManager completed cleanup") + logutil.Info("[SPILL] MemorySpillManager completed cleanup") } func (m *MemorySpillManager) TotalMem() int64 { diff --git a/pkg/sql/colexec/group/spillable_agg_state.go b/pkg/sql/colexec/group/spillable_agg_state.go index cf57687a8aa40..6ed25d2ad92cc 100644 --- a/pkg/sql/colexec/group/spillable_agg_state.go +++ b/pkg/sql/colexec/group/spillable_agg_state.go @@ -21,8 +21,6 @@ import ( "github.com/matrixorigin/matrixone/pkg/common/mpool" "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/container/vector" - "github.com/matrixorigin/matrixone/pkg/logutil" - "go.uber.org/zap" ) type SpillableAggState struct { @@ -33,48 +31,36 @@ type SpillableAggState struct { } func (s *SpillableAggState) Serialize() ([]byte, error) { - logutil.Info("SpillableAggState starting serialization", - zap.Int("group_count", s.GroupCount), - zap.Int("group_vectors_count", len(s.GroupVectors)), - zap.Int("agg_states_count", len(s.MarshaledAggStates))) buf := bytes.NewBuffer(nil) if err := binary.Write(buf, binary.LittleEndian, int32(s.GroupCount)); err != nil { - logutil.Error("SpillableAggState failed to write group count", zap.Error(err)) return nil, err } if err := binary.Write(buf, binary.LittleEndian, int32(len(s.GroupVectors))); err != nil { - logutil.Error("SpillableAggState failed to write group vectors count", zap.Error(err)) return nil, err } if err := binary.Write(buf, binary.LittleEndian, int32(len(s.GroupVectorTypes))); err != nil { - logutil.Error("SpillableAggState failed to write group vector types count", zap.Error(err)) return nil, err } - for i, typ := range s.GroupVectorTypes { + for _, typ := range s.GroupVectorTypes { typBytes, err := typ.MarshalBinary() if err != nil { - logutil.Error("SpillableAggState failed to marshal vector type", zap.Int("type_index", i), zap.Error(err)) return nil, err } if err := binary.Write(buf, binary.LittleEndian, int32(len(typBytes))); err != nil { - logutil.Error("SpillableAggState failed to write type bytes length", zap.Int("type_index", i), zap.Error(err)) return nil, err } if _, err := buf.Write(typBytes); err != nil { - logutil.Error("SpillableAggState failed to write type bytes", zap.Int("type_index", i), zap.Error(err)) return nil, err } } - for i, vec := range s.GroupVectors { + for _, vec := range s.GroupVectors { if vec == nil { if err := binary.Write(buf, binary.LittleEndian, int32(0)); err != nil { - logutil.Error("SpillableAggState failed to write zero length for nil vector", - zap.Int("vec_index", i), zap.Error(err)) return nil, err } continue @@ -82,87 +68,63 @@ func (s *SpillableAggState) Serialize() ([]byte, error) { vecBytes, err := vec.MarshalBinary() if err != nil { - logutil.Error("SpillableAggState failed to marshal vector", - zap.Int("vec_index", i), zap.Error(err)) return nil, err } if err := binary.Write(buf, binary.LittleEndian, int32(len(vecBytes))); err != nil { - logutil.Error("SpillableAggState failed to write vector bytes length", - zap.Int("vec_index", i), zap.Error(err)) return nil, err } if _, err := buf.Write(vecBytes); err != nil { - logutil.Error("SpillableAggState failed to write vector bytes", - zap.Int("vec_index", i), zap.Error(err)) return nil, err } } if err := binary.Write(buf, binary.LittleEndian, int32(len(s.MarshaledAggStates))); err != nil { - logutil.Error("SpillableAggState failed to write agg states count", zap.Error(err)) return nil, err } - for i, aggState := range s.MarshaledAggStates { + for _, aggState := range s.MarshaledAggStates { if err := binary.Write(buf, binary.LittleEndian, int32(len(aggState))); err != nil { - logutil.Error("SpillableAggState failed to write agg state length", - zap.Int("agg_index", i), zap.Error(err)) return nil, err } if _, err := buf.Write(aggState); err != nil { - logutil.Error("SpillableAggState failed to write agg state bytes", - zap.Int("agg_index", i), zap.Error(err)) return nil, err } } result := buf.Bytes() - logutil.Info("SpillableAggState completed serialization", - zap.Int("serialized_size", len(result))) return result, nil } func (s *SpillableAggState) Deserialize(data []byte, mp *mpool.MPool) error { - logutil.Info("SpillableAggState starting deserialization", - zap.Int("data_size", len(data))) buf := bytes.NewReader(data) var groupCount int32 if err := binary.Read(buf, binary.LittleEndian, &groupCount); err != nil { - logutil.Error("SpillableAggState failed to read group count", zap.Error(err)) return err } s.GroupCount = int(groupCount) var groupVecCount int32 if err := binary.Read(buf, binary.LittleEndian, &groupVecCount); err != nil { - logutil.Error("SpillableAggState failed to read group vector count", zap.Error(err)) return err } var groupVecTypeCount int32 if err := binary.Read(buf, binary.LittleEndian, &groupVecTypeCount); err != nil { - logutil.Error("SpillableAggState failed to read group vector type count", zap.Error(err)) return err } s.GroupVectorTypes = make([]types.Type, groupVecTypeCount) for i := 0; i < int(groupVecTypeCount); i++ { var size int32 if err := binary.Read(buf, binary.LittleEndian, &size); err != nil { - logutil.Error("SpillableAggState failed to read vector type size", - zap.Int("type_index", i), zap.Error(err)) return err } typBytes := make([]byte, size) if _, err := buf.Read(typBytes); err != nil { - logutil.Error("SpillableAggState failed to read vector type bytes", - zap.Int("type_index", i), zap.Error(err)) return err } if err := s.GroupVectorTypes[i].UnmarshalBinary(typBytes); err != nil { - logutil.Error("SpillableAggState failed to unmarshal vector type", - zap.Int("type_index", i), zap.Error(err)) return err } } @@ -171,8 +133,6 @@ func (s *SpillableAggState) Deserialize(data []byte, mp *mpool.MPool) error { for i := 0; i < int(groupVecCount); i++ { var size int32 if err := binary.Read(buf, binary.LittleEndian, &size); err != nil { - logutil.Error("SpillableAggState failed to read vector size", - zap.Int("vec_index", i), zap.Error(err)) return err } if size == 0 { @@ -182,8 +142,6 @@ func (s *SpillableAggState) Deserialize(data []byte, mp *mpool.MPool) error { vecBytes := make([]byte, size) if _, err := buf.Read(vecBytes); err != nil { - logutil.Error("SpillableAggState failed to read vector bytes", - zap.Int("vec_index", i), zap.Error(err)) return err } @@ -194,8 +152,6 @@ func (s *SpillableAggState) Deserialize(data []byte, mp *mpool.MPool) error { vec := vector.NewOffHeapVecWithType(vecType) if err := vec.UnmarshalBinaryWithCopy(vecBytes, mp); err != nil { - logutil.Error("SpillableAggState failed to unmarshal vector", - zap.Int("vec_index", i), zap.Error(err)) vec.Free(mp) return err } @@ -204,30 +160,20 @@ func (s *SpillableAggState) Deserialize(data []byte, mp *mpool.MPool) error { var aggStateCount int32 if err := binary.Read(buf, binary.LittleEndian, &aggStateCount); err != nil { - logutil.Error("SpillableAggState failed to read agg state count", zap.Error(err)) return err } s.MarshaledAggStates = make([][]byte, aggStateCount) for i := 0; i < int(aggStateCount); i++ { var size int32 if err := binary.Read(buf, binary.LittleEndian, &size); err != nil { - logutil.Error("SpillableAggState failed to read agg state size", - zap.Int("agg_index", i), zap.Error(err)) return err } s.MarshaledAggStates[i] = make([]byte, size) if _, err := buf.Read(s.MarshaledAggStates[i]); err != nil { - logutil.Error("SpillableAggState failed to read agg state bytes", - zap.Int("agg_index", i), zap.Error(err)) return err } } - logutil.Info("SpillableAggState completed deserialization", - zap.Int("group_count", s.GroupCount), - zap.Int("group_vectors_count", len(s.GroupVectors)), - zap.Int("agg_states_count", len(s.MarshaledAggStates))) - return nil }