github.com/matrixorigin/matrixone@v1.2.0/pkg/sql/colexec/aggexec/exec_test.go (about)

     1  // Copyright 2024 Matrix Origin
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //      http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    12  // See the License for the specific language governing permissions and
    13  // limitations under the License.
    14  
    15  package aggexec
    16  
    17  import (
    18  	"github.com/matrixorigin/matrixone/pkg/common/mpool"
    19  	"github.com/matrixorigin/matrixone/pkg/container/types"
    20  	"github.com/matrixorigin/matrixone/pkg/container/vector"
    21  	"github.com/stretchr/testify/require"
    22  	"sync"
    23  	"testing"
    24  )
    25  
    26  var uniqueAggIdForTest int64 = 100
    27  var uniqueAggIdForTestLock sync.RWMutex
    28  
    29  func gUniqueAggIdForTest() int64 {
    30  	uniqueAggIdForTestLock.Lock()
    31  	uniqueAggIdForTest++
    32  	ret := uniqueAggIdForTest
    33  	uniqueAggIdForTestLock.Unlock()
    34  
    35  	return ret
    36  }
    37  
    38  type testAggMemoryManager struct {
    39  	mp *mpool.MPool
    40  }
    41  
    42  func (m *testAggMemoryManager) Mp() *mpool.MPool {
    43  	return m.mp
    44  }
    45  func (m *testAggMemoryManager) GetVector(typ types.Type) *vector.Vector {
    46  	return vector.NewVec(typ)
    47  }
    48  func (m *testAggMemoryManager) PutVector(v *vector.Vector) {
    49  	v.Free(m.mp)
    50  }
    51  func newTestAggMemoryManager() AggMemoryManager {
    52  	return &testAggMemoryManager{mp: mpool.MustNewNoFixed("test_agg_exec")}
    53  }
    54  
    55  // testSingleAggPrivate1 is a structure that implements SingleAggFromFixedRetFixed.
    56  // it counts the number of input values (including NULLs) and it returns the count.
    57  // just for testing purposes.
    58  type testSingleAggPrivate1 struct{}
    59  
    60  func gTesSingleAggPrivate1() SingleAggFromFixedRetFixed[int32, int64] {
    61  	return &testSingleAggPrivate1{}
    62  }
    63  func (t *testSingleAggPrivate1) Init(AggSetter[int64], types.Type, types.Type) error {
    64  	return nil
    65  }
    66  
    67  func tSinglePrivate1Ret(_ []types.Type) types.Type {
    68  	return types.T_int64.ToType()
    69  }
    70  func fillSinglePrivate1(
    71  	exec SingleAggFromFixedRetFixed[int32, int64], value int32, getter AggGetter[int64], setter AggSetter[int64]) error {
    72  	setter(getter() + 1)
    73  	return nil
    74  }
    75  func fillNullSinglePrivate1(
    76  	exec SingleAggFromFixedRetFixed[int32, int64], getter AggGetter[int64], setter AggSetter[int64]) error {
    77  	setter(getter() + 1)
    78  	return nil
    79  }
    80  func fillsSinglePrivate1(
    81  	exec SingleAggFromFixedRetFixed[int32, int64], value int32, isNull bool, count int, getter AggGetter[int64], setter AggSetter[int64]) error {
    82  	setter(getter() + int64(count))
    83  	return nil
    84  }
    85  func mergeSinglePrivate1(
    86  	exec SingleAggFromFixedRetFixed[int32, int64], other SingleAggFromFixedRetFixed[int32, int64], getter1 AggGetter[int64], getter2 AggGetter[int64], setter AggSetter[int64]) error {
    87  	setter(getter1() + getter2())
    88  	return nil
    89  }
    90  func (t *testSingleAggPrivate1) Marshal() []byte {
    91  	return nil
    92  }
    93  func (t *testSingleAggPrivate1) Unmarshal([]byte) {}
    94  
    95  func TestSingleAggFuncExec1(t *testing.T) {
    96  	mg := newTestAggMemoryManager()
    97  
    98  	info := singleAggInfo{
    99  		aggID:     gUniqueAggIdForTest(),
   100  		distinct:  false,
   101  		argType:   types.T_int32.ToType(),
   102  		retType:   types.T_int64.ToType(),
   103  		emptyNull: false,
   104  	}
   105  	RegisterSingleAggFromFixedToFixed(
   106  		MakeSingleAgg1RegisteredInfo(
   107  			MakeSingleColumnAggInformation(info.aggID, info.argType, tSinglePrivate1Ret, true, info.emptyNull),
   108  			gTesSingleAggPrivate1,
   109  			nil, fillSinglePrivate1, fillNullSinglePrivate1, fillsSinglePrivate1, mergeSinglePrivate1, nil))
   110  	executor := MakeAgg(
   111  		mg,
   112  		info.aggID, info.distinct, info.argType)
   113  
   114  	// input first row of [3, null, 4, 5] - count 1
   115  	// input second row of [3, null, 4, 5] - count 1
   116  	// input [null * 2] - count 2
   117  	// input [1 * 3] - count 3
   118  	// input [1, 2, 3, 4] - count 4
   119  	// and the total count is 1+1+2+3+4 = 11
   120  	inputType := info.argType
   121  	inputs := make([]*vector.Vector, 5)
   122  	{
   123  		// prepare the input data.
   124  		var err error
   125  
   126  		vec := vector.NewVec(inputType)
   127  		require.NoError(t, vector.AppendFixedList[int32](vec, []int32{3, 0, 4, 5}, []bool{false, true, false, false}, mg.Mp()))
   128  		inputs[0] = vec
   129  		inputs[1] = vec
   130  		inputs[2] = vector.NewConstNull(inputType, 2, mg.Mp())
   131  		inputs[3], err = vector.NewConstFixed[int32](inputType, 1, 3, mg.Mp())
   132  		require.NoError(t, err)
   133  		inputs[4] = vector.NewVec(inputType)
   134  		require.NoError(t, vector.AppendFixedList[int32](inputs[4], []int32{1, 2, 3, 4}, nil, mg.Mp()))
   135  	}
   136  	{
   137  		require.NoError(t, executor.GroupGrow(1))
   138  		// data Fill.
   139  		require.NoError(t, executor.Fill(0, 0, []*vector.Vector{inputs[0]}))
   140  		require.NoError(t, executor.Fill(0, 1, []*vector.Vector{inputs[1]}))
   141  		require.NoError(t, executor.BulkFill(0, []*vector.Vector{inputs[2]}))
   142  		require.NoError(t, executor.BulkFill(0, []*vector.Vector{inputs[3]}))
   143  		require.NoError(t, executor.BulkFill(0, []*vector.Vector{inputs[4]}))
   144  	}
   145  	{
   146  		// result check.
   147  		v, err := executor.Flush()
   148  		require.NoError(t, err)
   149  		{
   150  			require.NotNil(t, v)
   151  			require.Equal(t, 1, v.Length())
   152  			require.Equal(t, int64(11), vector.MustFixedCol[int64](v)[0])
   153  		}
   154  		v.Free(mg.Mp())
   155  	}
   156  	{
   157  		executor.Free()
   158  		// memory check.
   159  		for _, v := range inputs {
   160  			v.Free(mg.Mp())
   161  		}
   162  		require.Equal(t, int64(0), mg.Mp().CurrNB())
   163  	}
   164  }
   165  
   166  // testMultiAggPrivate1 is a structure that implements MultiAggRetFixed.
   167  // it counts the amount of the second input value (including NULLs) when the first value was not null.
   168  type testMultiAggPrivate1 struct {
   169  	firstIsNull bool
   170  }
   171  
   172  func gTesMultiAggPrivate1() MultiAggRetFixed[int64] {
   173  	return &testMultiAggPrivate1{}
   174  }
   175  
   176  var mPrivate1Args = []types.Type{types.T_int64.ToType(), types.T_bool.ToType()}
   177  
   178  func mPrivate1Ret(_ []types.Type) types.Type {
   179  	return types.T_int64.ToType()
   180  }
   181  func fillWhich0(exec MultiAggRetFixed[int64], value int64) error {
   182  	exec.(*testMultiAggPrivate1).firstIsNull = false
   183  	return nil
   184  }
   185  func fillWhich1(exec MultiAggRetFixed[int64], value bool) error {
   186  	return nil
   187  }
   188  func fillNullWhich0(exec MultiAggRetFixed[int64]) error {
   189  	exec.(*testMultiAggPrivate1).firstIsNull = true
   190  	return nil
   191  }
   192  func fillNullWhich1(exec MultiAggRetFixed[int64]) error {
   193  	return nil
   194  }
   195  func validMPrivate1(exec MultiAggRetFixed[int64]) bool {
   196  	return !exec.(*testMultiAggPrivate1).firstIsNull
   197  }
   198  func evalMPrivate1(exec MultiAggRetFixed[int64], getter AggGetter[int64], setter AggSetter[int64]) error {
   199  	setter(getter() + 1)
   200  	return nil
   201  }
   202  func mergeMPrivate1(exec1, exec2 MultiAggRetFixed[int64], getter1 AggGetter[int64], getter2 AggGetter[int64], setter AggSetter[int64]) error {
   203  	setter(getter1() + getter2())
   204  	return nil
   205  }
   206  
   207  var mPrivate1FillWhich = []any{fillWhich0, fillWhich1}
   208  var mPrivate1FillNullWhich = []MultiAggFillNull1[int64]{fillNullWhich0, fillNullWhich1}
   209  
   210  func (t *testMultiAggPrivate1) Marshal() []byte {
   211  	return nil
   212  }
   213  func (t *testMultiAggPrivate1) Unmarshal([]byte) {}
   214  
   215  func TestMultiAggFuncExec1(t *testing.T) {
   216  	mg := newTestAggMemoryManager()
   217  
   218  	info := multiAggInfo{
   219  		aggID:     gUniqueAggIdForTest(),
   220  		distinct:  false,
   221  		argTypes:  mPrivate1Args,
   222  		retType:   types.T_int64.ToType(),
   223  		emptyNull: false,
   224  	}
   225  
   226  	RegisterMultiAggRetFixed(
   227  		MakeMultiAggRetFixedRegisteredInfo(
   228  			MakeMultiColumnAggInformation(info.aggID, mPrivate1Args, mPrivate1Ret, info.emptyNull),
   229  			gTesMultiAggPrivate1,
   230  			nil,
   231  			mPrivate1FillWhich,
   232  			mPrivate1FillNullWhich,
   233  			validMPrivate1,
   234  			evalMPrivate1,
   235  			mergeMPrivate1,
   236  			nil,
   237  		))
   238  
   239  	executor := MakeAgg(
   240  		mg,
   241  		info.aggID, info.distinct, info.argTypes...)
   242  
   243  	// input first row of [{null, false}, {1, true}] - count 0
   244  	// input second row of [{null, false}, {1, true}] - count 1
   245  	// input [{null, false} * 2] - count 0
   246  	// input [{1, true} * 3] - count 3
   247  	// input [{1, null}, {null, false}, {3, true}, {null, false}] - count 2
   248  	// and the total count is 0+1+0+3+2 = 6
   249  	inputs := make([][2]*vector.Vector, 5)
   250  	inputType1, inputType2 := info.argTypes[0], info.argTypes[1]
   251  	{
   252  		var err error
   253  
   254  		// prepare the input data.
   255  		vec1 := vector.NewVec(inputType1)
   256  		require.NoError(t, vector.AppendFixedList[int64](vec1, []int64{0, 1}, []bool{true, false}, mg.Mp()))
   257  		vec2 := vector.NewVec(inputType2)
   258  		require.NoError(t, vector.AppendFixedList[bool](vec2, []bool{false, true}, nil, mg.Mp()))
   259  		inputs[0] = [2]*vector.Vector{vec1, vec2}
   260  		inputs[1] = [2]*vector.Vector{vec1, vec2}
   261  
   262  		inputs[2] = [2]*vector.Vector{nil, nil}
   263  		inputs[2][0] = vector.NewConstNull(inputType1, 2, mg.Mp())
   264  		inputs[2][1], err = vector.NewConstFixed[bool](inputType2, false, 2, mg.Mp())
   265  		require.NoError(t, err)
   266  
   267  		inputs[3] = [2]*vector.Vector{nil, nil}
   268  		inputs[3][0], err = vector.NewConstFixed[int64](inputType1, 1, 3, mg.Mp())
   269  		require.NoError(t, err)
   270  		inputs[3][1], err = vector.NewConstFixed[bool](inputType2, true, 3, mg.Mp())
   271  		require.NoError(t, err)
   272  
   273  		inputs[4][0] = vector.NewVec(inputType1)
   274  		require.NoError(t, vector.AppendFixedList[int64](inputs[4][0], []int64{1, 0, 3, 0}, []bool{false, true, false, true}, mg.Mp()))
   275  		inputs[4][1] = vector.NewVec(inputType2)
   276  		require.NoError(t, vector.AppendFixedList[bool](inputs[4][1], []bool{true, false, true, false}, []bool{true, false, false, false}, mg.Mp()))
   277  	}
   278  	{
   279  		require.NoError(t, executor.GroupGrow(1))
   280  		// data Fill.
   281  		require.NoError(t, executor.Fill(0, 0, inputs[0][:]))
   282  		require.NoError(t, executor.Fill(0, 1, inputs[1][:]))
   283  		require.NoError(t, executor.BulkFill(0, inputs[2][:]))
   284  		require.NoError(t, executor.BulkFill(0, inputs[3][:]))
   285  		require.NoError(t, executor.BulkFill(0, inputs[4][:]))
   286  	}
   287  	{
   288  		// result check.
   289  		v, err := executor.Flush()
   290  		require.NoError(t, err)
   291  		{
   292  			require.NotNil(t, v)
   293  			require.Equal(t, 1, v.Length())
   294  			require.Equal(t, int64(6), vector.MustFixedCol[int64](v)[0])
   295  		}
   296  		v.Free(mg.Mp())
   297  	}
   298  	{
   299  		executor.Free()
   300  		// memory check.
   301  		for _, v := range inputs {
   302  			for _, vv := range v {
   303  				vv.Free(mg.Mp())
   304  			}
   305  		}
   306  		require.Equal(t, int64(0), mg.Mp().CurrNB())
   307  	}
   308  }
   309  
   310  func TestGroupConcatExec(t *testing.T) {
   311  	mg := newTestAggMemoryManager()
   312  	info := multiAggInfo{
   313  		aggID:    gUniqueAggIdForTest(),
   314  		distinct: false,
   315  		argTypes: []types.Type{
   316  			types.T_varchar.ToType(),
   317  			types.T_char.ToType(),
   318  		},
   319  		retType:   types.T_text.ToType(),
   320  		emptyNull: false,
   321  	}
   322  	RegisterGroupConcatAgg(info.aggID, ",")
   323  
   324  	executor := MakeAgg(mg, info.aggID, info.distinct, info.argTypes...)
   325  
   326  	// group concat the vector1 and vector2.
   327  	// vector1: ["a", "b", "c", "d"].
   328  	// vector2: ["d", "c", "b", "a"].
   329  	// the result is ["ad,bc,cb,da"].
   330  	inputs := make([]*vector.Vector, 2)
   331  	inputs[0] = vector.NewVec(info.argTypes[0])
   332  	inputs[1] = vector.NewVec(info.argTypes[1])
   333  	require.NoError(t, vector.AppendStringList(inputs[0], []string{"a", "b", "c", "d"}, nil, mg.Mp()))
   334  	require.NoError(t, vector.AppendStringList(inputs[1], []string{"d", "c", "b", "a"}, nil, mg.Mp()))
   335  	{
   336  		require.NoError(t, executor.GroupGrow(1))
   337  		// data Fill.
   338  		require.NoError(t, executor.Fill(0, 0, inputs))
   339  		require.NoError(t, executor.Fill(0, 1, inputs))
   340  		require.NoError(t, executor.Fill(0, 2, inputs))
   341  		require.NoError(t, executor.Fill(0, 3, inputs))
   342  	}
   343  	{
   344  		// result check.
   345  		v, err := executor.Flush()
   346  		require.NoError(t, err)
   347  		{
   348  			require.NotNil(t, v)
   349  			require.Equal(t, 1, v.Length())
   350  			bs := vector.MustStrCol(v)
   351  			require.Equal(t, 1, len(bs))
   352  			require.Equal(t, "ad,bc,cb,da", bs[0])
   353  		}
   354  		v.Free(mg.Mp())
   355  	}
   356  	{
   357  		executor.Free()
   358  		// memory check.
   359  		for _, v := range inputs {
   360  			v.Free(mg.Mp())
   361  		}
   362  		require.Equal(t, int64(0), mg.Mp().CurrNB())
   363  	}
   364  }
   365  
   366  // TestEmptyNullFlag test if the emptyNull flag is working.
   367  // the emptyNull flag is used to determine whether the NULL value is included in the aggregation result.
   368  // if the emptyNull flag is true, the NULL value will be returned for empty groups.
   369  func TestEmptyNullFlag(t *testing.T) {
   370  	mg := newTestAggMemoryManager()
   371  	{
   372  		id := gUniqueAggIdForTest()
   373  		RegisterSingleAggFromFixedToFixed(
   374  			MakeSingleAgg1RegisteredInfo(
   375  				MakeSingleColumnAggInformation(id, types.T_int32.ToType(), tSinglePrivate1Ret, false, true),
   376  				gTesSingleAggPrivate1,
   377  				nil, fillSinglePrivate1, fillNullSinglePrivate1, fillsSinglePrivate1, mergeSinglePrivate1, nil))
   378  		executor := MakeAgg(
   379  			mg,
   380  			id, false, types.T_int32.ToType())
   381  		require.NoError(t, executor.GroupGrow(1))
   382  		v, err := executor.Flush()
   383  		require.NoError(t, err)
   384  		require.True(t, v.IsNull(0))
   385  		v.Free(mg.Mp())
   386  		executor.Free()
   387  	}
   388  	{
   389  		id := gUniqueAggIdForTest()
   390  		RegisterSingleAggFromFixedToFixed(
   391  			MakeSingleAgg1RegisteredInfo(
   392  				MakeSingleColumnAggInformation(id, types.T_int32.ToType(), tSinglePrivate1Ret, false, false),
   393  				gTesSingleAggPrivate1,
   394  				nil, fillSinglePrivate1, fillNullSinglePrivate1, fillsSinglePrivate1, mergeSinglePrivate1, nil))
   395  		executor := MakeAgg(
   396  			mg,
   397  			id, false, types.T_int32.ToType())
   398  		require.NoError(t, executor.GroupGrow(1))
   399  		v, err := executor.Flush()
   400  		require.NoError(t, err)
   401  		require.False(t, v.IsNull(0))
   402  		require.Equal(t, int64(0), vector.MustFixedCol[int64](v)[0])
   403  		v.Free(mg.Mp())
   404  		executor.Free()
   405  	}
   406  	{
   407  		id := gUniqueAggIdForTest()
   408  		RegisterMultiAggRetFixed(
   409  			MakeMultiAggRetFixedRegisteredInfo(
   410  				MakeMultiColumnAggInformation(id, mPrivate1Args, mPrivate1Ret, true),
   411  				gTesMultiAggPrivate1,
   412  				nil,
   413  				mPrivate1FillWhich,
   414  				mPrivate1FillNullWhich,
   415  				validMPrivate1,
   416  				evalMPrivate1,
   417  				mergeMPrivate1,
   418  				nil,
   419  			))
   420  		executor := MakeAgg(
   421  			mg,
   422  			id, false, []types.Type{types.T_int64.ToType(), types.T_bool.ToType()}...)
   423  		require.NoError(t, executor.GroupGrow(1))
   424  		v, err := executor.Flush()
   425  		require.NoError(t, err)
   426  		require.True(t, v.IsNull(0))
   427  		v.Free(mg.Mp())
   428  		executor.Free()
   429  	}
   430  	{
   431  		id := gUniqueAggIdForTest()
   432  		RegisterMultiAggRetFixed(
   433  			MakeMultiAggRetFixedRegisteredInfo(
   434  				MakeMultiColumnAggInformation(id, mPrivate1Args, mPrivate1Ret, false),
   435  				gTesMultiAggPrivate1,
   436  				nil,
   437  				mPrivate1FillWhich,
   438  				mPrivate1FillNullWhich,
   439  				validMPrivate1,
   440  				evalMPrivate1,
   441  				mergeMPrivate1,
   442  				nil,
   443  			))
   444  		executor := MakeAgg(
   445  			mg,
   446  			id, false, []types.Type{types.T_int64.ToType(), types.T_bool.ToType()}...)
   447  		require.NoError(t, executor.GroupGrow(1))
   448  		v, err := executor.Flush()
   449  		require.NoError(t, err)
   450  		require.False(t, v.IsNull(0))
   451  		require.Equal(t, int64(0), vector.MustFixedCol[int64](v)[0])
   452  		v.Free(mg.Mp())
   453  		executor.Free()
   454  	}
   455  	require.Equal(t, int64(0), mg.Mp().CurrNB())
   456  }