From e7c4432c2ec584c0c84e3761a9d66e7196c4b751 Mon Sep 17 00:00:00 2001 From: fagongzi Date: Tue, 10 Dec 2024 09:52:09 +0800 Subject: [PATCH 01/27] fix hung due to deadlock (#20673) fix hung due to deadlock. Approved by: @reusee --- pkg/txn/client/client.go | 27 +++++++++++++++++++++++---- 1 file changed, 23 insertions(+), 4 deletions(-) diff --git a/pkg/txn/client/client.go b/pkg/txn/client/client.go index 12200965806b1..98f2db5580844 100644 --- a/pkg/txn/client/client.go +++ b/pkg/txn/client/client.go @@ -204,7 +204,8 @@ type txnClient struct { // all active txns activeTxns map[string]*txnOperator // FIFO queue for ready to active txn - waitActiveTxns []*txnOperator + waitActiveTxns []*txnOperator + waitMarkAllActiveAbortedC chan struct{} } abortC chan time.Time @@ -499,6 +500,8 @@ func (client *txnClient) openTxn(op *txnOperator) error { client.mu.Unlock() }() + client.waitMarkAllActiveAbortedLocked() + if !op.opts.skipWaitPushClient { for client.mu.state == paused { if client.normalStateNoWait { @@ -695,13 +698,23 @@ func (client *txnClient) handleMarkActiveTxnAborted( case from := <-client.abortC: fn := func() { client.mu.Lock() - defer client.mu.Unlock() - + client.mu.waitMarkAllActiveAbortedC = make(chan struct{}) + ops := make([]*txnOperator, 0, len(client.mu.activeTxns)) for _, op := range client.mu.activeTxns { if op.reset.createAt.Before(from) { - op.addFlag(AbortedFlag) + ops = append(ops, op) } } + client.mu.Unlock() + + for _, op := range ops { + op.addFlag(AbortedFlag) + } + + client.mu.Lock() + close(client.mu.waitMarkAllActiveAbortedC) + client.mu.waitMarkAllActiveAbortedC = nil + client.mu.Unlock() } fn() @@ -727,3 +740,9 @@ func (client *txnClient) removeFromWaitActiveLocked(txnID []byte) bool { client.mu.waitActiveTxns = values return ok } + +func (client *txnClient) waitMarkAllActiveAbortedLocked() { + if client.mu.waitMarkAllActiveAbortedC != nil { + <-client.mu.waitMarkAllActiveAbortedC + } +} From 60428fe6ac6c606bec6b2f0647d669bd1fae308e Mon Sep 17 00:00:00 2001 From: Wei Ziran Date: Tue, 10 Dec 2024 12:11:58 +0800 Subject: [PATCH 02/27] do not merge tombstone when compact (#20551) do not merge tombstone when compact Approved by: @XuPeng-SH --- pkg/vm/engine/tae/db/merge/cnScheduler.go | 2 +- pkg/vm/engine/tae/db/merge/policyCompact.go | 69 ++------- pkg/vm/engine/tae/db/merge/policyOverlap.go | 34 ++--- pkg/vm/engine/tae/db/merge/policy_test.go | 41 ++++-- pkg/vm/engine/tae/db/merge/utils.go | 137 +++++++++++++++++- pkg/vm/engine/tae/db/merge/utils_test.go | 71 +++++++++ .../tae/tables/txnentries/flushTableTail.go | 3 +- .../tae/tables/txnentries/mergeobjects.go | 6 +- 8 files changed, 271 insertions(+), 92 deletions(-) diff --git a/pkg/vm/engine/tae/db/merge/cnScheduler.go b/pkg/vm/engine/tae/db/merge/cnScheduler.go index 2f15d49cb233c..703ecf2fa6ad7 100644 --- a/pkg/vm/engine/tae/db/merge/cnScheduler.go +++ b/pkg/vm/engine/tae/db/merge/cnScheduler.go @@ -54,7 +54,7 @@ func (s *CNMergeScheduler) sendMergeTask(ctx context.Context, task *api.MergeTas if !ok { return taskservice.ErrNotReady } - taskIDPrefix := "Merge:" + task.TableName + taskIDPrefix := "Merge:" + strconv.Itoa(int(task.TblId)) asyncTask, err := ts.QueryAsyncTask(ctx, taskservice.WithTaskMetadataId(taskservice.LIKE, taskIDPrefix+"%"), taskservice.WithTaskStatusCond(taskpb.TaskStatus_Created, taskpb.TaskStatus_Running)) diff --git a/pkg/vm/engine/tae/db/merge/policyCompact.go b/pkg/vm/engine/tae/db/merge/policyCompact.go index ad7258b38690f..1674cf173359b 100644 --- a/pkg/vm/engine/tae/db/merge/policyCompact.go +++ b/pkg/vm/engine/tae/db/merge/policyCompact.go @@ -31,23 +31,13 @@ type objCompactPolicy struct { tblEntry *catalog.TableEntry fs fileservice.FileService - segObjects map[objectio.Segmentid][]*catalog.ObjectEntry + objects []*catalog.ObjectEntry - tombstones []*catalog.ObjectEntry tombstoneMetas []objectio.ObjectDataMeta - - validTombstones map[*catalog.ObjectEntry]struct{} } func newObjCompactPolicy(fs fileservice.FileService) *objCompactPolicy { - return &objCompactPolicy{ - fs: fs, - - segObjects: make(map[objectio.Segmentid][]*catalog.ObjectEntry), - - tombstones: make([]*catalog.ObjectEntry, 0), - validTombstones: make(map[*catalog.ObjectEntry]struct{}), - } + return &objCompactPolicy{fs: fs} } func (o *objCompactPolicy) onObject(entry *catalog.ObjectEntry, config *BasicPolicyConfig) bool { @@ -60,16 +50,15 @@ func (o *objCompactPolicy) onObject(entry *catalog.ObjectEntry, config *BasicPol if entry.OriginSize() < config.ObjectMinOsize { return false } - if len(o.tombstones) == 0 { + if len(o.tombstoneMetas) == 0 { return false } - for i, meta := range o.tombstoneMetas { + for _, meta := range o.tombstoneMetas { if !checkTombstoneMeta(meta, entry.ID()) { continue } - o.validTombstones[o.tombstones[i]] = struct{}{} - o.segObjects[entry.ObjectName().SegmentId()] = append(o.segObjects[entry.ObjectName().SegmentId()], entry) + o.objects = append(o.objects, entry) } return false } @@ -78,28 +67,20 @@ func (o *objCompactPolicy) revise(rc *resourceController) []reviseResult { if o.tblEntry == nil { return nil } - o.filterValidTombstones() - results := make([]reviseResult, 0, len(o.segObjects)+1) - for _, objs := range o.segObjects { - if rc.resourceAvailable(objs) { - rc.reserveResources(objs) - for _, obj := range objs { - results = append(results, reviseResult{[]*catalog.ObjectEntry{obj}, taskHostDN}) - } + results := make([]reviseResult, 0, len(o.objects)) + for _, obj := range o.objects { + if rc.resourceAvailable([]*catalog.ObjectEntry{obj}) { + rc.reserveResources([]*catalog.ObjectEntry{obj}) + results = append(results, reviseResult{[]*catalog.ObjectEntry{obj}, taskHostDN}) } } - if len(o.tombstones) > 0 { - results = append(results, reviseResult{o.tombstones, taskHostDN}) - } return results } func (o *objCompactPolicy) resetForTable(entry *catalog.TableEntry, config *BasicPolicyConfig) { o.tblEntry = entry - o.tombstones = o.tombstones[:0] o.tombstoneMetas = o.tombstoneMetas[:0] - clear(o.segObjects) - clear(o.validTombstones) + o.objects = o.objects[:0] tIter := entry.MakeTombstoneObjectIt() for tIter.Next() { @@ -109,37 +90,19 @@ func (o *objCompactPolicy) resetForTable(entry *catalog.TableEntry, config *Basi continue } - if (entryOutdated(tEntry, config.TombstoneLifetime) && tEntry.OriginSize() > 10*common.Const1MBytes) || - tEntry.OriginSize() > common.DefaultMaxOsizeObjMB*common.Const1MBytes { - - ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) - meta, err := loadTombstoneMeta(ctx, tEntry.GetObjectStats(), o.fs) - cancel() + if tEntry.OriginSize() > common.DefaultMaxOsizeObjMB*common.Const1MBytes { + meta, err := loadTombstoneMeta(tEntry.GetObjectStats(), o.fs) if err != nil { continue } - o.tombstoneMetas = append(o.tombstoneMetas, meta) - o.tombstones = append(o.tombstones, tEntry) - } - } -} - -func (o *objCompactPolicy) filterValidTombstones() { - i := 0 - for _, x := range o.tombstones { - if _, ok := o.validTombstones[x]; !ok { - o.tombstones[i] = x - i++ } } - for j := i; j < len(o.tombstones); j++ { - o.tombstones[j] = nil - } - o.tombstones = o.tombstones[:i] } -func loadTombstoneMeta(ctx context.Context, tombstoneObject *objectio.ObjectStats, fs fileservice.FileService) (objectio.ObjectDataMeta, error) { +func loadTombstoneMeta(tombstoneObject *objectio.ObjectStats, fs fileservice.FileService) (objectio.ObjectDataMeta, error) { + ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) + defer cancel() location := tombstoneObject.ObjectLocation() objMeta, err := objectio.FastLoadObjectMeta( ctx, &location, false, fs, diff --git a/pkg/vm/engine/tae/db/merge/policyOverlap.go b/pkg/vm/engine/tae/db/merge/policyOverlap.go index 1203522ff4871..ef4afd343d44e 100644 --- a/pkg/vm/engine/tae/db/merge/policyOverlap.go +++ b/pkg/vm/engine/tae/db/merge/policyOverlap.go @@ -31,14 +31,12 @@ var levels = [6]int{ type objOverlapPolicy struct { leveledObjects [len(levels)][]*catalog.ObjectEntry - segments map[objectio.Segmentid]map[*catalog.ObjectEntry]struct{} - overlappingObjsSet [][]*catalog.ObjectEntry + segments map[objectio.Segmentid]map[*catalog.ObjectEntry]struct{} } func newObjOverlapPolicy() *objOverlapPolicy { return &objOverlapPolicy{ - overlappingObjsSet: make([][]*catalog.ObjectEntry, 0), - segments: make(map[objectio.Segmentid]map[*catalog.ObjectEntry]struct{}), + segments: make(map[objectio.Segmentid]map[*catalog.ObjectEntry]struct{}), } } @@ -70,20 +68,20 @@ func (m *objOverlapPolicy) revise(rc *resourceController) []reviseResult { continue } - m.overlappingObjsSet = m.overlappingObjsSet[:0] - objs := objectsWithGivenOverlaps(m.leveledObjects[i], 5) - for _, obj := range objs { - result := reviseResult{objs: obj, kind: taskHostDN} - if result.kind == taskHostDN { - if rc.cpuPercent > 80 { - continue - } + for _, objs := range objectsWithGivenOverlaps(m.leveledObjects[i], 5) { + objs = removeOversize(objs) + if len(objs) < 2 || score(objs) < 1.1 { + continue + } + result := reviseResult{objs: objs, kind: taskHostDN} + if rc.cpuPercent > 80 { + continue + } - if rc.resourceAvailable(result.objs) { - rc.reserveResources(result.objs) - } else { - result.kind = taskHostCN - } + if rc.resourceAvailable(result.objs) { + rc.reserveResources(result.objs) + } else { + result.kind = taskHostCN } reviseResults = append(reviseResults, result) } @@ -92,7 +90,6 @@ func (m *objOverlapPolicy) revise(rc *resourceController) []reviseResult { } func (m *objOverlapPolicy) resetForTable(*catalog.TableEntry, *BasicPolicyConfig) { - m.overlappingObjsSet = m.overlappingObjsSet[:0] for i := range m.leveledObjects { m.leveledObjects[i] = m.leveledObjects[i][:0] } @@ -143,6 +140,7 @@ func objectsWithGivenOverlaps(objects []*catalog.ObjectEntry, overlaps int) [][] tmp := make(map[*catalog.ObjectEntry]struct{}) for { objs := make([]*catalog.ObjectEntry, 0, len(points)/2) + clear(tmp) for _, p := range points { if p.s == 1 { tmp[p.obj] = struct{}{} diff --git a/pkg/vm/engine/tae/db/merge/policy_test.go b/pkg/vm/engine/tae/db/merge/policy_test.go index 49d8478835618..dc1eb072de8c7 100644 --- a/pkg/vm/engine/tae/db/merge/policy_test.go +++ b/pkg/vm/engine/tae/db/merge/policy_test.go @@ -16,6 +16,7 @@ package merge import ( "context" + "math" "math/rand/v2" "testing" @@ -74,7 +75,7 @@ func newSortedTombstoneEntryWithTableEntry(t *testing.T, tbl *catalog.TableEntry return entry } -func newSortedTestObjectEntry(t *testing.T, v1, v2 int32, size uint32) *catalog.ObjectEntry { +func newSortedTestObjectEntry(t testing.TB, v1, v2 int32, size uint32) *catalog.ObjectEntry { zm := index.NewZM(types.T_int32, 0) index.UpdateZM(zm, types.EncodeInt32(&v1)) index.UpdateZM(zm, types.EncodeInt32(&v2)) @@ -89,6 +90,21 @@ func newSortedTestObjectEntry(t *testing.T, v1, v2 int32, size uint32) *catalog. } } +func newTestVarcharObjectEntry(t testing.TB, v1, v2 string, size uint32) *catalog.ObjectEntry { + zm := index.NewZM(types.T_varchar, 0) + index.UpdateZM(zm, []byte(v1)) + index.UpdateZM(zm, []byte(v2)) + stats := objectio.NewObjectStats() + objName := objectio.BuildObjectNameWithObjectID(objectio.NewObjectid()) + require.NoError(t, objectio.SetObjectStatsObjectName(stats, objName)) + require.NoError(t, objectio.SetObjectStatsSortKeyZoneMap(stats, zm)) + require.NoError(t, objectio.SetObjectStatsOriginSize(stats, size)) + require.NoError(t, objectio.SetObjectStatsRowCnt(stats, 2)) + return &catalog.ObjectEntry{ + ObjectMVCCNode: catalog.ObjectMVCCNode{ObjectStats: *stats}, + } +} + func newTestObjectEntry(t *testing.T, size uint32, isTombstone bool) *catalog.ObjectEntry { stats := objectio.NewObjectStats() require.NoError(t, objectio.SetObjectStatsOriginSize(stats, size)) @@ -268,8 +284,13 @@ func TestPolicyCompact(t *testing.T) { tbl, err := db.CreateTableEntry(catalog.MockSchema(1, 0), txn1, nil) require.NoError(t, err) require.NoError(t, txn1.Commit(context.Background())) - - p.resetForTable(tbl, nil) + obj := catalog.MockObjEntryWithTbl(tbl, math.MaxUint32, false) + tombstone := catalog.MockObjEntryWithTbl(tbl, math.MaxUint32, true) + require.NoError(t, objectio.SetObjectStatsOriginSize(tombstone.GetObjectStats(), math.MaxUint32)) + tbl.AddEntryLocked(obj) + tbl.AddEntryLocked(tombstone) + p.resetForTable(tbl, &BasicPolicyConfig{}) + p.onObject(obj, &BasicPolicyConfig{}) objs := p.revise(rc) require.Equal(t, 0, len(objs)) @@ -373,7 +394,7 @@ func TestCheckTombstone(t *testing.T) { ss := writer.GetObjectStats() require.Equal(t, rowCnt, int(ss.Rows())) - meta, err := loadTombstoneMeta(context.TODO(), &ss, fs) + meta, err := loadTombstoneMeta(&ss, fs) require.NoError(t, err) metas[i] = meta } @@ -391,12 +412,12 @@ func TestCheckTombstone(t *testing.T) { } func TestObjectsWithMaximumOverlaps(t *testing.T) { - o1 := newSortedTestObjectEntry(t, 0, 50, 0) - o2 := newSortedTestObjectEntry(t, 51, 100, 0) - o3 := newSortedTestObjectEntry(t, 49, 52, 0) - o4 := newSortedTestObjectEntry(t, 1, 52, 0) - o5 := newSortedTestObjectEntry(t, 50, 51, 0) - o6 := newSortedTestObjectEntry(t, 55, 60, 0) + o1 := newSortedTestObjectEntry(t, 0, 50, math.MaxInt32) + o2 := newSortedTestObjectEntry(t, 51, 100, math.MaxInt32) + o3 := newSortedTestObjectEntry(t, 49, 52, math.MaxInt32) + o4 := newSortedTestObjectEntry(t, 1, 52, math.MaxInt32) + o5 := newSortedTestObjectEntry(t, 50, 51, math.MaxInt32) + o6 := newSortedTestObjectEntry(t, 55, 60, math.MaxInt32) res1 := objectsWithGivenOverlaps([]*catalog.ObjectEntry{o1, o2}, 2) require.Equal(t, 0, len(res1)) diff --git a/pkg/vm/engine/tae/db/merge/utils.go b/pkg/vm/engine/tae/db/merge/utils.go index 78edfb5917106..8327646f93978 100644 --- a/pkg/vm/engine/tae/db/merge/utils.go +++ b/pkg/vm/engine/tae/db/merge/utils.go @@ -15,20 +15,24 @@ package merge import ( + "cmp" "context" "math" "os" + "slices" "sync/atomic" "time" "github.com/KimMachineGun/automemlimit/memlimit" "github.com/matrixorigin/matrixone/pkg/common/moerr" + "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/fileservice" "github.com/matrixorigin/matrixone/pkg/logutil" "github.com/matrixorigin/matrixone/pkg/objectio" "github.com/matrixorigin/matrixone/pkg/pb/api" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/catalog" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common" + "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/index" "github.com/shirou/gopsutil/v3/cpu" "github.com/shirou/gopsutil/v3/process" ) @@ -45,19 +49,140 @@ const ( estimateMemUsagePerRow = 30 ) +func score(objs []*catalog.ObjectEntry) float64 { + if len(objs) < 2 { + return 0 + } + totalDiff := float64(0) + minMaxZM := objs[0].SortKeyZoneMap().Clone() + if !minMaxZM.GetType().IsFixedLen() { + return math.MaxFloat64 + } + for _, obj := range objs { + zm := obj.SortKeyZoneMap() + index.UpdateZM(minMaxZM, zm.GetMinBuf()) + index.UpdateZM(minMaxZM, zm.GetMaxBuf()) + w := diff(zm.GetMax(), zm.GetMin(), zm.GetType()) + if w == math.MaxUint64 { + return math.MaxFloat64 + } + totalDiff += float64(w) + } + maxDiff := diff(minMaxZM.GetMax(), minMaxZM.GetMin(), minMaxZM.GetType()) + if maxDiff == math.MaxUint64 { + return math.MaxFloat64 + } + return totalDiff / float64(maxDiff) +} + +func diff(a, b any, t types.T) uint64 { + switch t { + case types.T_bool: + if a == b { + return 0 + } + return 1 + case types.T_bit: + x, y := a.(uint64), b.(uint64) + return max(x, y) - min(x, y) + case types.T_int8: + x, y := a.(int8), b.(int8) + return uint64(max(x, y) - min(x, y)) + case types.T_int16: + x, y := a.(int16), b.(int16) + return uint64(max(x, y) - min(x, y)) + case types.T_int32: + x, y := a.(int32), b.(int32) + return uint64(max(x, y) - min(x, y)) + case types.T_int64: + x, y := a.(int64), b.(int64) + return uint64(max(x, y) - min(x, y)) + case types.T_uint8: + x, y := a.(uint8), b.(uint8) + return uint64(max(x, y) - min(x, y)) + case types.T_uint16: + x, y := a.(uint16), b.(uint16) + return uint64(max(x, y) - min(x, y)) + case types.T_uint32: + x, y := a.(uint32), b.(uint32) + return uint64(max(x, y) - min(x, y)) + case types.T_uint64: + x, y := a.(uint64), b.(uint64) + return max(x, y) - min(x, y) + case types.T_float32: + x, y := a.(float32), b.(float32) + return uint64(max(x, y) - min(x, y)) + case types.T_float64: + x, y := a.(float64), b.(float64) + return uint64(max(x, y) - min(x, y)) + case types.T_date: + x, y := a.(types.Date), b.(types.Date) + return uint64(max(x, y) - min(x, y)) + case types.T_time: + x, y := a.(types.Time), b.(types.Time) + return uint64(max(x, y) - min(x, y)) + case types.T_datetime: + x, y := a.(types.Datetime), b.(types.Datetime) + return uint64(max(x, y) - min(x, y)) + case types.T_timestamp: + x, y := a.(types.Timestamp), b.(types.Timestamp) + return uint64(max(x, y) - min(x, y)) + case types.T_enum: + x, y := a.(types.Enum), b.(types.Enum) + return uint64(max(x, y) - min(x, y)) + case types.T_decimal64: + x, y := a.(types.Decimal64), b.(types.Decimal64) + return uint64(max(x, y) - min(x, y)) + default: + } + return math.MaxUint64 +} + +func removeOversize(objs []*catalog.ObjectEntry) []*catalog.ObjectEntry { + if len(objs) < 2 { + return objs + } + slices.SortFunc(objs, func(a, b *catalog.ObjectEntry) int { + return cmp.Compare(a.OriginSize(), b.OriginSize()) + }) + + if len(objs) == 2 { + if uint(objs[1].OriginSize()) < 3*uint(objs[0].OriginSize()) { + return objs[:2] + } + return nil + } + + accSize := int(objs[0].OriginSize()) + int(objs[1].OriginSize()) + i := 2 + for i < len(objs) { + size := int(objs[i].OriginSize()) + if size > accSize { + break + } + accSize += size + i++ + } + for j := i; j < len(objs); j++ { + objs[j] = nil + } + if i == 2 { + if uint(objs[1].OriginSize()) < 3*uint(objs[0].OriginSize()) { + return objs[:2] + } + return nil + } + return objs[:i] +} + func estimateMergeSize(objs []*catalog.ObjectEntry) int { size := 0 for _, o := range objs { - size += int(o.Rows() * estimateMemUsagePerRow) + size += int(o.Rows()) * estimateMemUsagePerRow } return size } -func entryOutdated(entry *catalog.ObjectEntry, lifetime time.Duration) bool { - createdAt := entry.CreatedAt.Physical() - return time.Unix(0, createdAt).Add(lifetime).Before(time.Now()) -} - type resourceController struct { proc *process.Process diff --git a/pkg/vm/engine/tae/db/merge/utils_test.go b/pkg/vm/engine/tae/db/merge/utils_test.go index 00544a1456800..c7a6cfc7cfbad 100644 --- a/pkg/vm/engine/tae/db/merge/utils_test.go +++ b/pkg/vm/engine/tae/db/merge/utils_test.go @@ -16,11 +16,14 @@ package merge import ( "context" + "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/defines" "github.com/matrixorigin/matrixone/pkg/fileservice" "github.com/matrixorigin/matrixone/pkg/pb/api" + "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/catalog" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "math" "os" "path" "testing" @@ -67,3 +70,71 @@ func Test_CleanUpUselessFiles(t *testing.T) { CleanUpUselessFiles(ent, fs) } + +func TestScore(t *testing.T) { + o1 := newSortedTestObjectEntry(t, 0, 3, 1) + o2 := newSortedTestObjectEntry(t, 1, 4, 2) + o3 := newSortedTestObjectEntry(t, 1, 5, 4) + o4 := newSortedTestObjectEntry(t, 1, 100, 4) + o5 := newSortedTestObjectEntry(t, 5, 10, math.MaxInt32) + + // should merge + require.Less(t, 1.1, score([]*catalog.ObjectEntry{o1, o1})) + require.Less(t, 1.1, score([]*catalog.ObjectEntry{o1, o2})) + require.Less(t, 1.1, score([]*catalog.ObjectEntry{o1, o3})) + // should not merge + require.Greater(t, 1.1, score([]*catalog.ObjectEntry{o1, o4})) + require.Greater(t, 1.1, score([]*catalog.ObjectEntry{o1, o2, o4})) + require.Greater(t, 1.1, score([]*catalog.ObjectEntry{o1, o5})) + + o6 := newTestVarcharObjectEntry(t, "a", "z", 1) + o7 := newTestVarcharObjectEntry(t, "b", "y", 1) + + require.Less(t, 1.1, score([]*catalog.ObjectEntry{o6, o7})) +} + +func TestRemoveOversize(t *testing.T) { + o1 := newSortedTestObjectEntry(t, 0, 0, 1) + o2 := newSortedTestObjectEntry(t, 0, 0, 2) + o3 := newSortedTestObjectEntry(t, 0, 0, 4) + o5 := newSortedTestObjectEntry(t, 0, 0, math.MaxInt32) + + require.ElementsMatch(t, []*catalog.ObjectEntry{o1, o2}, removeOversize([]*catalog.ObjectEntry{o1, o2})) + require.ElementsMatch(t, []*catalog.ObjectEntry{o1, o2}, removeOversize([]*catalog.ObjectEntry{o5, o1, o2})) + require.ElementsMatch(t, nil, removeOversize([]*catalog.ObjectEntry{o1, o3})) +} + +func TestDiff(t *testing.T) { + require.Equal(t, uint64(0), diff(false, false, types.T_bool)) + require.Equal(t, uint64(1), diff(false, true, types.T_bool)) + require.Equal(t, uint64(10), diff(int8(1), int8(11), types.T_int8)) + require.Equal(t, uint64(10), diff(uint8(1), uint8(11), types.T_uint8)) + require.Equal(t, uint64(10), diff(int16(1), int16(11), types.T_int16)) + require.Equal(t, uint64(10), diff(uint16(1), uint16(11), types.T_uint16)) + require.Equal(t, uint64(10), diff(int32(1), int32(11), types.T_int32)) + require.Equal(t, uint64(10), diff(uint32(1), uint32(11), types.T_uint32)) + require.Equal(t, uint64(10), diff(int64(1), int64(11), types.T_int64)) + require.Equal(t, uint64(10), diff(uint64(1), uint64(11), types.T_uint64)) + require.Equal(t, uint64(10), diff(int8(1), int8(11), types.T_int8)) + require.Equal(t, uint64(10), diff(int8(1), int8(11), types.T_int8)) + require.Equal(t, uint64(10), diff(float32(1), float32(11), types.T_float32)) + require.Equal(t, uint64(10), diff(float64(1), float64(11), types.T_float64)) + require.Equal(t, uint64(10), diff(types.Date(1), types.Date(11), types.T_date)) + require.Equal(t, uint64(10), diff(types.Datetime(1), types.Datetime(11), types.T_datetime)) + require.Equal(t, uint64(10), diff(types.Time(1), types.Time(11), types.T_time)) + require.Equal(t, uint64(10), diff(types.Timestamp(1), types.Timestamp(11), types.T_timestamp)) + require.Equal(t, uint64(10), diff(types.Enum(1), types.Enum(11), types.T_enum)) + require.Equal(t, uint64(10), diff(types.Decimal64(1), types.Decimal64(11), types.T_decimal64)) + require.Equal(t, uint64(math.MaxUint64), diff(types.Decimal128{}, types.Decimal128{}, types.T_decimal128)) +} + +func BenchmarkRemoveOversize(b *testing.B) { + o1 := newSortedTestObjectEntry(b, 0, 50, math.MaxInt32) + o2 := newSortedTestObjectEntry(b, 51, 100, 1) + o3 := newSortedTestObjectEntry(b, 49, 52, 2) + + b.ResetTimer() + for i := 0; i < b.N; i++ { + removeOversize([]*catalog.ObjectEntry{o1, o2, o3}) + } +} diff --git a/pkg/vm/engine/tae/tables/txnentries/flushTableTail.go b/pkg/vm/engine/tae/tables/txnentries/flushTableTail.go index 2f44e0dc4c192..fae9ae053c615 100644 --- a/pkg/vm/engine/tae/tables/txnentries/flushTableTail.go +++ b/pkg/vm/engine/tae/tables/txnentries/flushTableTail.go @@ -224,7 +224,8 @@ func (entry *flushTableTailEntry) collectDelsAndTransfer( row := rowid[i].GetRowOffset() destpos, ok := mapping[row] if !ok { - panic(fmt.Sprintf("%s find no transfer mapping for row %d", obj.ID().String(), row)) + err = moerr.NewInternalErrorNoCtxf("%s find no transfer mapping for row %d", obj.ID().String(), row) + return } blkID := objectio.NewBlockidWithObjectID(entry.createdObjHandle.GetID(), destpos.BlkIdx) entry.delTbls[destpos.BlkIdx] = blkID diff --git a/pkg/vm/engine/tae/tables/txnentries/mergeobjects.go b/pkg/vm/engine/tae/tables/txnentries/mergeobjects.go index c61335d769019..e62627bc06eaa 100644 --- a/pkg/vm/engine/tae/tables/txnentries/mergeobjects.go +++ b/pkg/vm/engine/tae/tables/txnentries/mergeobjects.go @@ -289,9 +289,9 @@ func (entry *mergeObjectsEntry) transferObjectDeletes( _max = k } } - panic(fmt.Sprintf( - "%s-%d find no transfer mapping for row %d, mapping range (%d, %d)", - dropped.ID().String(), blkOffsetInObj, row, _min, _max)) + err = moerr.NewInternalErrorNoCtxf("%s-%d find no transfer mapping for row %d, mapping range (%d, %d)", + dropped.ID().String(), blkOffsetInObj, row, _min, _max) + return } if entry.delTbls[*entry.createdObjs[destpos.ObjIdx].ID()] == nil { entry.delTbls[*entry.createdObjs[destpos.ObjIdx].ID()] = make(map[uint16]struct{}) From f16d9ff0a74b7e8b7d553dc60f87f364f8c96323 Mon Sep 17 00:00:00 2001 From: YANGGMM Date: Tue, 10 Dec 2024 14:58:10 +0800 Subject: [PATCH 03/27] fix restore view with lower case table names equals to 0 (#20667) fix restore view with lower case table names equals to 0 Approved by: @daviszhen, @heni02 --- pkg/frontend/pitr.go | 30 +++- pkg/frontend/pitr_test.go | 135 ++++++++++++++++ pkg/frontend/snapshot.go | 31 +++- .../snapshot_restore_lower_case_2.result | 153 ++++++++++++++++++ .../snapshot_restore_lower_case_2.sql | 153 ++++++++++++++++++ 5 files changed, 486 insertions(+), 16 deletions(-) create mode 100644 test/distributed/cases/snapshot/snapshot_restore_lower_case_2.result create mode 100644 test/distributed/cases/snapshot/snapshot_restore_lower_case_2.sql diff --git a/pkg/frontend/pitr.go b/pkg/frontend/pitr.go index 400b0923ae4aa..623ac06dd55f5 100644 --- a/pkg/frontend/pitr.go +++ b/pkg/frontend/pitr.go @@ -1689,7 +1689,15 @@ func restoreViewsWithPitr( viewMap map[string]*tableInfo, accountName string, curAccount uint32) error { - snapshot := &pbplan.Snapshot{ + getLogger(ses.GetService()).Info(fmt.Sprintf("[%s] start to restore views", pitrName)) + var ( + err error + stmts []tree.Statement + sortedViews []string + snapshot *pbplan.Snapshot + oldSnapshot *pbplan.Snapshot + ) + snapshot = &pbplan.Snapshot{ TS: ×tamp.Timestamp{PhysicalTime: ts}, Tenant: &pbplan.SnapshotTenant{ TenantName: accountName, @@ -1698,23 +1706,29 @@ func restoreViewsWithPitr( } compCtx := ses.GetTxnCompileCtx() - oldSnapshot := compCtx.GetSnapshot() + oldSnapshot = compCtx.GetSnapshot() compCtx.SetSnapshot(snapshot) defer func() { compCtx.SetSnapshot(oldSnapshot) }() g := toposort{next: make(map[string][]string)} - for key, view := range viewMap { - stmts, err := parsers.Parse(ctx, dialect.MYSQL, view.createSql, 1) + for key, viewEntry := range viewMap { + getLogger(ses.GetService()).Info(fmt.Sprintf("[%s] start to restore view: %v", pitrName, viewEntry.tblName)) + stmts, err = parsers.Parse(ctx, dialect.MYSQL, viewEntry.createSql, 1) if err != nil { return err } - compCtx.SetDatabase(view.dbName) + compCtx.SetDatabase(viewEntry.dbName) // build create sql to find dependent views - if _, err = plan.BuildPlan(compCtx, stmts[0], false); err != nil { - return err + _, err = plan.BuildPlan(compCtx, stmts[0], false) + if err != nil { + stmts, _ = parsers.Parse(ctx, dialect.MYSQL, viewEntry.createSql, 0) + _, err = plan.BuildPlan(compCtx, stmts[0], false) + if err != nil { + return err + } } g.addVertex(key) @@ -1724,7 +1738,7 @@ func restoreViewsWithPitr( } // topsort - sortedViews, err := g.sort() + sortedViews, err = g.sort() if err != nil { return err } diff --git a/pkg/frontend/pitr_test.go b/pkg/frontend/pitr_test.go index a83f6e4eef1d7..ff27c5eec708f 100644 --- a/pkg/frontend/pitr_test.go +++ b/pkg/frontend/pitr_test.go @@ -2902,3 +2902,138 @@ func TestCheckDbIsSubDb(t *testing.T) { }) } } + +func Test_restoreViews(t *testing.T) { + convey.Convey("restoreViews", t, func() { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + ses := newTestSession(t, ctrl) + defer ses.Close() + + bh := &backgroundExecTest{} + bh.init() + + bhStub := gostub.StubFunc(&NewBackgroundExec, bh) + defer bhStub.Reset() + + pu := config.NewParameterUnit(&config.FrontendParameters{}, nil, nil, nil) + pu.SV.SetDefaultValues() + pu.SV.KillRountinesInterval = 0 + setPu("", pu) + ctx := context.WithValue(context.TODO(), config.ParameterUnitKey, pu) + rm, _ := NewRoutineManager(ctx, "") + ses.rm = rm + + tenant := &TenantInfo{ + Tenant: sysAccountName, + User: rootName, + DefaultRole: moAdminRoleName, + TenantID: sysAccountID, + UserID: rootID, + DefaultRoleID: moAdminRoleID, + } + ses.SetTenantInfo(tenant) + + ctx = context.WithValue(ctx, defines.TenantIDKey{}, uint32(sysAccountID)) + + //no result set + bh.sql2result["begin;"] = nil + bh.sql2result["commit;"] = nil + bh.sql2result["rollback;"] = nil + + viewMap := map[string]*tableInfo{} + err := restoreViews(ctx, ses, bh, "sp01", viewMap, 0) + assert.Error(t, err) + + sql := "select * from mo_catalog.mo_snapshots where sname = 'sp01'" + // string/ string/ int64/ string/ string/ string/ string/ uint64 + mrs := newMrsForPitrRecord([][]interface{}{{"1", "sp01", int64(0), "ACCOUNT", "sys", "", "", uint64(1)}}) + bh.sql2result[sql] = mrs + + sql = "select account_id, account_name, status, version, suspended_time from mo_catalog.mo_account where 1=1 and account_name = 'sys'" + mrs = newMrsForPitrRecord([][]interface{}{{uint64(0), "sys", "open", uint64(1), ""}}) + bh.sql2result[sql] = mrs + + err = restoreViews(ctx, ses, bh, "sp01", viewMap, 0) + assert.NoError(t, err) + + viewMap = map[string]*tableInfo{ + "view01": { + dbName: "db01", + tblName: "tbl01", + typ: "VIEW", + createSql: "create view view01", + }, + } + err = restoreViews(ctx, ses, bh, "sp01", viewMap, 0) + assert.Error(t, err) + }) +} + +func Test_restoreViewsWithPitr(t *testing.T) { + convey.Convey("restoreViewsWithPitr", t, func() { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + ses := newTestSession(t, ctrl) + defer ses.Close() + + bh := &backgroundExecTest{} + bh.init() + + bhStub := gostub.StubFunc(&NewBackgroundExec, bh) + defer bhStub.Reset() + + pu := config.NewParameterUnit(&config.FrontendParameters{}, nil, nil, nil) + pu.SV.SetDefaultValues() + pu.SV.KillRountinesInterval = 0 + setPu("", pu) + ctx := context.WithValue(context.TODO(), config.ParameterUnitKey, pu) + rm, _ := NewRoutineManager(ctx, "") + ses.rm = rm + + tenant := &TenantInfo{ + Tenant: sysAccountName, + User: rootName, + DefaultRole: moAdminRoleName, + TenantID: sysAccountID, + UserID: rootID, + DefaultRoleID: moAdminRoleID, + } + ses.SetTenantInfo(tenant) + + ctx = context.WithValue(ctx, defines.TenantIDKey{}, uint32(sysAccountID)) + + //no result set + bh.sql2result["begin;"] = nil + bh.sql2result["commit;"] = nil + bh.sql2result["rollback;"] = nil + + viewMap := map[string]*tableInfo{} + err := restoreViewsWithPitr(ctx, ses, bh, "sp01", 0, viewMap, "sys", 0) + assert.NoError(t, err) + + viewMap = map[string]*tableInfo{ + "view01": { + dbName: "db01", + tblName: "tbl01", + typ: "VIEW", + createSql: "create view view01", + }, + } + err = restoreViewsWithPitr(ctx, ses, bh, "sp01", 0, viewMap, "sys", 0) + assert.Error(t, err) + + viewMap = map[string]*tableInfo{ + "view01": { + dbName: "db01", + tblName: "tbl01", + typ: "VIEW", + createSql: "create database db02", + }, + } + err = restoreViewsWithPitr(ctx, ses, bh, "sp01", 0, viewMap, "sys", 0) + assert.NoError(t, err) + }) +} diff --git a/pkg/frontend/snapshot.go b/pkg/frontend/snapshot.go index 3c0c18808599c..aa7d5c6d5668b 100644 --- a/pkg/frontend/snapshot.go +++ b/pkg/frontend/snapshot.go @@ -992,29 +992,44 @@ func restoreViews( snapshotName string, viewMap map[string]*tableInfo, toAccountId uint32) error { - snapshot, err := getSnapshotPlanWithSharedBh(ctx, bh, snapshotName) + getLogger(ses.GetService()).Info("start to restore views") + var ( + err error + snapshot *plan.Snapshot + stmts []tree.Statement + sortedViews []string + oldSnapshot *plan.Snapshot + ) + snapshot, err = getSnapshotPlanWithSharedBh(ctx, bh, snapshotName) if err != nil { return err } compCtx := ses.GetTxnCompileCtx() - oldSnapshot := compCtx.GetSnapshot() + oldSnapshot = compCtx.GetSnapshot() compCtx.SetSnapshot(snapshot) defer func() { compCtx.SetSnapshot(oldSnapshot) }() g := toposort{next: make(map[string][]string)} - for key, view := range viewMap { - stmts, err := parsers.Parse(ctx, dialect.MYSQL, view.createSql, 1) + for key, viewEntry := range viewMap { + getLogger(ses.GetService()).Info(fmt.Sprintf("[%s] start to restore view: %v", snapshotName, viewEntry.tblName)) + stmts, err = parsers.Parse(ctx, dialect.MYSQL, viewEntry.createSql, 1) if err != nil { return err } - compCtx.SetDatabase(view.dbName) + compCtx.SetDatabase(viewEntry.dbName) // build create sql to find dependent views - if _, err = plan.BuildPlan(compCtx, stmts[0], false); err != nil { - return err + _, err = plan.BuildPlan(compCtx, stmts[0], false) + if err != nil { + getLogger(ses.GetService()).Info(fmt.Sprintf("try to build view %v failed, try to build it again", viewEntry.tblName)) + stmts, _ = parsers.Parse(ctx, dialect.MYSQL, viewEntry.createSql, 0) + _, err = plan.BuildPlan(compCtx, stmts[0], false) + if err != nil { + return err + } } g.addVertex(key) @@ -1024,7 +1039,7 @@ func restoreViews( } // toposort - sortedViews, err := g.sort() + sortedViews, err = g.sort() if err != nil { return err } diff --git a/test/distributed/cases/snapshot/snapshot_restore_lower_case_2.result b/test/distributed/cases/snapshot/snapshot_restore_lower_case_2.result new file mode 100644 index 0000000000000..cf1368989d030 --- /dev/null +++ b/test/distributed/cases/snapshot/snapshot_restore_lower_case_2.result @@ -0,0 +1,153 @@ +drop account if exists a1; +create account a1 ADMIN_NAME 'admin1' IDENTIFIED BY 'test123'; +select @@lower_case_table_names; +@@lower_case_table_names +1 +set global lower_case_table_names = 0; +select @@lower_case_table_names; +@@lower_case_table_names +0 +drop database if exists test02; +create database test02; +use test02; +drop table if exists Departments; +drop table if exists Employees; +create table Departments ( +DepartmentID INT PRIMARY KEY, +DepartmentName VARCHAR(255) NOT NULL +); +create table Employees ( +EmployeeID INT PRIMARY KEY, +FirstName VARCHAR(255) NOT NULL, +LastName VARCHAR(255) NOT NULL, +DepartmentID INT, +foreign key (DepartmentID) REFERENCES Departments(DepartmentID) +); +insert into Departments (DepartmentID, DepartmentName) values +(1, 'Human Resources'), +(2, 'Engineering'), +(3, 'Marketing'), +(4, 'Sales'), +(5, 'Finance'); +insert into Employees (EmployeeID, FirstName, LastName, DepartmentID) values +(101, 'John', 'Doe', 1), +(102, 'Jane', 'Smith', 2), +(103, 'Alice', 'Johnson', 3), +(104, 'Mark', 'Patterson', 4), +(105, 'David', 'Finley', 5); +drop view if exists EmployeeDepartmentView; +create view EmployeeDepartmentView as +select +e.FirstName, +e.LastName, +d.DepartmentName +from +Employees e +inner join +Departments d ON e.DepartmentID = d.DepartmentID; +select * from EmployeeDepartmentView; +firstname lastname departmentname +John Doe Human Resources +Jane Smith Engineering +Alice Johnson Marketing +Mark Patterson Sales +David Finley Finance +drop snapshot if exists sp02_restore_lower; +create snapshot sp02_restore_lower for account a1; +select @@lower_case_table_names; +@@lower_case_table_names +0 +drop database test02; +restore account a1 from snapshot sp02_restore_lower; +use test02; +show tables; +Tables_in_test02 +Departments +EmployeeDepartmentView +Employees +drop database if exists test02; +restore account a1 database test02 from snapshot sp02_restore_lower; +use test02; +show tables; +Tables_in_test02 +Departments +EmployeeDepartmentView +Employees +drop account if exists a1; +drop account if exists a1; +create account a1 ADMIN_NAME 'admin1' IDENTIFIED BY 'test123'; +select @@lower_case_table_names; +@@lower_case_table_names +1 +set global lower_case_table_names = 0; +select @@lower_case_table_names; +@@lower_case_table_names +0 +select @@lower_case_table_names; +@@lower_case_table_names +0 +drop database if exists test02; +create database test02; +use test02; +drop table if exists Departments; +drop table if exists Employees; +create table Departments ( +DepartmentID INT PRIMARY KEY, +DepartmentName VARCHAR(255) NOT NULL +); +create table Employees ( +EmployeeID INT PRIMARY KEY, +FirstName VARCHAR(255) NOT NULL, +LastName VARCHAR(255) NOT NULL, +DepartmentID INT, +foreign key (DepartmentID) REFERENCES Departments(DepartmentID) +); +insert into Departments (DepartmentID, DepartmentName) values +(1, 'Human Resources'), +(2, 'Engineering'), +(3, 'Marketing'), +(4, 'Sales'), +(5, 'Finance'); +insert into Employees (EmployeeID, FirstName, LastName, DepartmentID) values +(101, 'John', 'Doe', 1), +(102, 'Jane', 'Smith', 2), +(103, 'Alice', 'Johnson', 3), +(104, 'Mark', 'Patterson', 4), +(105, 'David', 'Finley', 5); +drop view if exists EmployeeDepartmentView; +create view EmployeeDepartmentView as +select +e.FirstName, +e.LastName, +d.DepartmentName +from +Employees e +inner join +Departments d ON e.DepartmentID = d.DepartmentID; +select * from EmployeeDepartmentView; +firstname lastname departmentname +John Doe Human Resources +Jane Smith Engineering +Alice Johnson Marketing +Mark Patterson Sales +David Finley Finance +show tables; +Tables_in_test02 +Departments +EmployeeDepartmentView +Employees +drop snapshot if exists sp02_restore_lower; +create snapshot sp02_restore_lower for account a1; +select @@lower_case_table_names; +@@lower_case_table_names +0 +drop database test02; +restore account a1 from snapshot sp02_restore_lower; +use test02; +show tables; +Tables_in_test02 +Departments +EmployeeDepartmentView +Employees +drop account if exists a1; +drop snapshot if exists sp02_restore_lower; diff --git a/test/distributed/cases/snapshot/snapshot_restore_lower_case_2.sql b/test/distributed/cases/snapshot/snapshot_restore_lower_case_2.sql new file mode 100644 index 0000000000000..4714147fabbf9 --- /dev/null +++ b/test/distributed/cases/snapshot/snapshot_restore_lower_case_2.sql @@ -0,0 +1,153 @@ +drop account if exists a1; +create account a1 ADMIN_NAME 'admin1' IDENTIFIED BY 'test123'; + +-- @session:id=1&user=a1:admin1&password=test123 +-- default value is 1 +select @@lower_case_table_names; +set global lower_case_table_names = 0; +-- @session + +-- @session:id=2&user=a1:admin1&password=test123 +-- it's 0 now +select @@lower_case_table_names; +drop database if exists test02; +create database test02; +use test02; +drop table if exists Departments; +drop table if exists Employees; +create table Departments ( + DepartmentID INT PRIMARY KEY, + DepartmentName VARCHAR(255) NOT NULL +); + +create table Employees ( +EmployeeID INT PRIMARY KEY, +FirstName VARCHAR(255) NOT NULL, +LastName VARCHAR(255) NOT NULL, +DepartmentID INT, +foreign key (DepartmentID) REFERENCES Departments(DepartmentID) +); + +insert into Departments (DepartmentID, DepartmentName) values +(1, 'Human Resources'), +(2, 'Engineering'), +(3, 'Marketing'), +(4, 'Sales'), +(5, 'Finance'); + +insert into Employees (EmployeeID, FirstName, LastName, DepartmentID) values +(101, 'John', 'Doe', 1), +(102, 'Jane', 'Smith', 2), +(103, 'Alice', 'Johnson', 3), +(104, 'Mark', 'Patterson', 4), +(105, 'David', 'Finley', 5); + +drop view if exists EmployeeDepartmentView; +create view EmployeeDepartmentView as +select + e.FirstName, + e.LastName, + d.DepartmentName +from + Employees e + inner join + Departments d ON e.DepartmentID = d.DepartmentID; +select * from EmployeeDepartmentView; + +drop snapshot if exists sp02_restore_lower; +create snapshot sp02_restore_lower for account a1; + +select @@lower_case_table_names; +drop database test02; + +restore account a1 from snapshot sp02_restore_lower; + +use test02; +show tables; + +drop database if exists test02; +restore account a1 database test02 from snapshot sp02_restore_lower; + +use test02; +show tables; +-- @session + +drop account if exists a1; + +drop account if exists a1; +create account a1 ADMIN_NAME 'admin1' IDENTIFIED BY 'test123'; + +-- @session:id=3&user=a1:admin1&password=test123 +-- default value is 1 +select @@lower_case_table_names; +set global lower_case_table_names = 0; +-- @session + +-- @session:id=4&user=a1:admin1&password=test123 +-- it's 0 now +select @@lower_case_table_names; +select @@lower_case_table_names; +drop database if exists test02; +create database test02; +use test02; +drop table if exists Departments; +drop table if exists Employees; +create table Departments ( + DepartmentID INT PRIMARY KEY, + DepartmentName VARCHAR(255) NOT NULL +); + +create table Employees ( +EmployeeID INT PRIMARY KEY, +FirstName VARCHAR(255) NOT NULL, +LastName VARCHAR(255) NOT NULL, +DepartmentID INT, +foreign key (DepartmentID) REFERENCES Departments(DepartmentID) +); + +insert into Departments (DepartmentID, DepartmentName) values +(1, 'Human Resources'), +(2, 'Engineering'), +(3, 'Marketing'), +(4, 'Sales'), +(5, 'Finance'); + +insert into Employees (EmployeeID, FirstName, LastName, DepartmentID) values +(101, 'John', 'Doe', 1), +(102, 'Jane', 'Smith', 2), +(103, 'Alice', 'Johnson', 3), +(104, 'Mark', 'Patterson', 4), +(105, 'David', 'Finley', 5); + +drop view if exists EmployeeDepartmentView; +create view EmployeeDepartmentView as +select + e.FirstName, + e.LastName, + d.DepartmentName +from + Employees e + inner join + Departments d ON e.DepartmentID = d.DepartmentID; +select * from EmployeeDepartmentView; + +show tables; +-- @session + +drop snapshot if exists sp02_restore_lower; +create snapshot sp02_restore_lower for account a1; + +-- @session:id=4&user=a1:admin1&password=test123 +select @@lower_case_table_names; +drop database test02; +-- @session + +restore account a1 from snapshot sp02_restore_lower; + +-- @session:id=4&user=a1:admin1&password=test123 +use test02; +show tables; +-- @session + +drop account if exists a1; +drop snapshot if exists sp02_restore_lower; From 6f2799d011f3f75798f0dfbe1d2893337af2a646 Mon Sep 17 00:00:00 2001 From: Kai Cao Date: Tue, 10 Dec 2024 16:14:49 +0800 Subject: [PATCH 04/27] [Cherry-pick] fix delete unsubscribed account in mo_subs (#20685) delete unsubscribed account in mo_subs Approved by: @daviszhen --- pkg/bootstrap/versions/v2_0_1/pubsub.go | 5 ++ pkg/bootstrap/versions/v2_0_1/pubsub_test.go | 54 +++++++++++++++++++- 2 files changed, 57 insertions(+), 2 deletions(-) diff --git a/pkg/bootstrap/versions/v2_0_1/pubsub.go b/pkg/bootstrap/versions/v2_0_1/pubsub.go index 899d95fc63f13..f8def79d7dfb7 100644 --- a/pkg/bootstrap/versions/v2_0_1/pubsub.go +++ b/pkg/bootstrap/versions/v2_0_1/pubsub.go @@ -107,6 +107,11 @@ func migrateMoPubs(txn executor.TxnExecutor) (err error) { if _, err = txn.Exec(insertSql, executor.StatementOption{}.WithAccountID(0)); err != nil { return } + + deleteSql := fmt.Sprintf("delete from mo_catalog.mo_subs where pub_account_name = '%s' and pub_name = '%s' and sub_name is null", info.PubAccountName, info.PubName) + if _, err = txn.Exec(deleteSql, executor.StatementOption{}.WithAccountID(0)); err != nil { + return + } } return } diff --git a/pkg/bootstrap/versions/v2_0_1/pubsub_test.go b/pkg/bootstrap/versions/v2_0_1/pubsub_test.go index c9784c034fc4d..b568d99001a8f 100644 --- a/pkg/bootstrap/versions/v2_0_1/pubsub_test.go +++ b/pkg/bootstrap/versions/v2_0_1/pubsub_test.go @@ -15,6 +15,7 @@ package v2_0_1 import ( + "strings" "testing" "github.com/matrixorigin/matrixone/pkg/bootstrap/versions" @@ -27,7 +28,9 @@ import ( "github.com/stretchr/testify/assert" ) -type MockTxnExecutor struct{} +type MockTxnExecutor struct { + flag bool +} func (MockTxnExecutor) Use(db string) { //TODO implement me @@ -39,7 +42,11 @@ func (MockTxnExecutor) LockTable(table string) error { panic("implement me") } -func (MockTxnExecutor) Exec(sql string, options executor.StatementOption) (executor.Result, error) { +func (e MockTxnExecutor) Exec(sql string, options executor.StatementOption) (executor.Result, error) { + if strings.HasPrefix(sql, "delete from mo_catalog.mo_subs") && e.flag { + return executor.Result{}, assert.AnError + } + bat := batch.New([]string{"a"}) bat.Vecs[0] = testutil.MakeInt32Vector([]int32{1}, nil) bat.SetRowCount(1) @@ -106,3 +113,46 @@ func Test_migrateMoPubs(t *testing.T) { err := migrateMoPubs(txn) assert.NoError(t, err) } + +func Test_migrateMoPubs_deleteFailed(t *testing.T) { + getAccountsStub := gostub.Stub( + &pubsub.GetAccounts, + func(_ executor.TxnExecutor) (map[string]*pubsub.AccountInfo, map[int32]*pubsub.AccountInfo, error) { + return map[string]*pubsub.AccountInfo{ + "acc1": {Id: 1, Name: "acc1"}, + }, nil, nil + }, + ) + defer getAccountsStub.Reset() + + getAllPubInfosStub := gostub.Stub( + &versions.GetAllPubInfos, + func(_ executor.TxnExecutor, _ map[string]*pubsub.AccountInfo) (map[string]*pubsub.PubInfo, error) { + return map[string]*pubsub.PubInfo{ + "sys#pubName": { + PubAccountName: "sys", + PubName: "pubName", + SubAccountsStr: pubsub.AccountAll, + }, + "acc1#pubName": { + PubAccountName: "acc1", + PubName: "pubName", + SubAccountsStr: pubsub.AccountAll, + }, + }, nil + }, + ) + defer getAllPubInfosStub.Reset() + + getSubbedAccNamesStub := gostub.Stub( + &getSubbedAccNames, + func(_ executor.TxnExecutor, _, _ string, _ map[int32]*pubsub.AccountInfo) ([]string, error) { + return []string{"acc2"}, nil + }, + ) + defer getSubbedAccNamesStub.Reset() + + txn := &MockTxnExecutor{flag: true} + err := migrateMoPubs(txn) + assert.Error(t, err) +} From 31d5996e6a38c032b0c0f4b51ad9012ff75a85b1 Mon Sep 17 00:00:00 2001 From: XuPeng-SH Date: Tue, 10 Dec 2024 17:36:13 +0800 Subject: [PATCH 05/27] small code refactor (#20686) more detailed log Approved by: @triump2020 --- pkg/vm/engine/disttae/txn_table.go | 17 ++++-------- pkg/vm/engine/engine_util/reader.go | 12 +++++++- pkg/vm/engine/tae/db/db.go | 4 +-- pkg/vm/engine/tae/db/open.go | 43 ++++++++++++----------------- 4 files changed, 35 insertions(+), 41 deletions(-) diff --git a/pkg/vm/engine/disttae/txn_table.go b/pkg/vm/engine/disttae/txn_table.go index a6ffb6312a0b5..498689afbe0ad 100644 --- a/pkg/vm/engine/disttae/txn_table.go +++ b/pkg/vm/engine/disttae/txn_table.go @@ -613,26 +613,19 @@ func (tbl *txnTable) doRanges(ctx context.Context, rangesParam engine.RangesPara tbl.enableLogFilterExpr.Store(true) } - if tbl.enableLogFilterExpr.Load() { + if ok, _ := objectio.RangesLogInjected(tbl.db.databaseName, tbl.tableDef.Name); ok || + err != nil || + tbl.enableLogFilterExpr.Load() || + cost > 5*time.Second { logutil.Info( "TXN-FILTER-RANGE-LOG", zap.String("name", tbl.tableDef.Name), zap.String("exprs", plan2.FormatExprs(rangesParam.BlockFilters)), - zap.Int("ranges-len", blocks.Len()), - zap.Uint64("tbl-id", tbl.tableId), - zap.String("txn", tbl.db.op.Txn().DebugString()), - ) - } - - if ok, _ := objectio.RangesLogInjected(tbl.db.databaseName, tbl.tableDef.Name); ok { - logutil.Info( - "INJECT-TRACE-RANGES", - zap.String("name", tbl.tableDef.Name), - zap.String("exprs", plan2.FormatExprs(rangesParam.BlockFilters)), zap.Uint64("tbl-id", tbl.tableId), zap.String("txn", tbl.db.op.Txn().DebugString()), zap.String("blocks", blocks.String()), zap.String("ps", fmt.Sprintf("%p", part)), + zap.Duration("cost", cost), zap.Error(err), ) } diff --git a/pkg/vm/engine/engine_util/reader.go b/pkg/vm/engine/engine_util/reader.go index af72dead65284..8ce9476d6792f 100644 --- a/pkg/vm/engine/engine_util/reader.go +++ b/pkg/vm/engine/engine_util/reader.go @@ -429,11 +429,18 @@ func (r *reader) Read( if isEnd { return } + blkStr := "nil" + if blkInfo != nil { + blkStr = blkInfo.String() + } if logLevel == 0 { logutil.Info( - "LOGREADER-INJECTED-1", + "DEBUG-SLOW-TXN-READER", zap.String("name", r.name), + zap.String("ts", r.ts.DebugString()), zap.Int("data-len", outBatch.RowCount()), + zap.Duration("duration", time.Since(start)), + zap.String("blk", blkStr), zap.Error(err), ) } else { @@ -444,8 +451,11 @@ func (r *reader) Read( logutil.Info( "LOGREADER-INJECTED-1", zap.String("name", r.name), + zap.String("ts", r.ts.DebugString()), + zap.Duration("duration", time.Since(start)), zap.Error(err), zap.String("data", common.MoBatchToString(outBatch, maxLogCnt)), + zap.String("blk", blkStr), ) } } diff --git a/pkg/vm/engine/tae/db/db.go b/pkg/vm/engine/tae/db/db.go index 2626486bd97dc..26207ea09b355 100644 --- a/pkg/vm/engine/tae/db/db.go +++ b/pkg/vm/engine/tae/db/db.go @@ -77,7 +77,7 @@ type DB struct { LogtailMgr *logtail.Manager Wal wal.Driver - GCJobs *tasks.CancelableJobs + CronJobs *tasks.CancelableJobs BGScanner wb.IHeartbeater BGCheckpointRunner checkpoint.Runner @@ -273,7 +273,7 @@ func (db *DB) Close() error { } db.Closed.Store(ErrClosed) db.Controller.Stop() - db.GCJobs.Reset() + db.CronJobs.Reset() db.BGScanner.Stop() db.BGCheckpointRunner.Stop() db.Runtime.Scheduler.Stop() diff --git a/pkg/vm/engine/tae/db/open.go b/pkg/vm/engine/tae/db/open.go index c3aead017b9e8..df0ace26c563b 100644 --- a/pkg/vm/engine/tae/db/open.go +++ b/pkg/vm/engine/tae/db/open.go @@ -281,9 +281,9 @@ func Open( db.DiskCleaner = gc2.NewDiskCleaner(cleaner) db.DiskCleaner.Start() - db.GCJobs = tasks.NewCancelableJobs() + db.CronJobs = tasks.NewCancelableJobs() - db.GCJobs.AddJob( + db.CronJobs.AddJob( "GC-Transfer-Table", opts.CheckpointCfg.TransferInterval, func(ctx context.Context) { @@ -293,7 +293,7 @@ func Open( }, 1, ) - db.GCJobs.AddJob( + db.CronJobs.AddJob( "GC-Disk", opts.GCCfg.ScanGCInterval, func(ctx context.Context) { @@ -301,7 +301,7 @@ func Open( }, 1, ) - db.GCJobs.AddJob( + db.CronJobs.AddJob( "GC-Checkpoint", opts.CheckpointCfg.GCCheckpointInterval, func(ctx context.Context) { @@ -321,7 +321,7 @@ func Open( }, 1, ) - db.GCJobs.AddJob( + db.CronJobs.AddJob( "GC-Catalog-Cache", opts.CatalogCfg.GCInterval, func(ctx context.Context) { @@ -336,7 +336,7 @@ func Open( }, 1, ) - db.GCJobs.AddJob( + db.CronJobs.AddJob( "GC-Logtail", opts.CheckpointCfg.GCCheckpointInterval, func(ctx context.Context) { @@ -349,7 +349,7 @@ func Open( }, 1, ) - db.GCJobs.AddJob( + db.CronJobs.AddJob( "GC-LockMerge", options.DefaultLockMergePruneInterval, func(ctx context.Context) { @@ -358,8 +358,17 @@ func Open( 1, ) + db.CronJobs.AddJob( + "REPORT-MPOOL-STATS", + time.Second*10, + func(ctx context.Context) { + mpoolAllocatorSubTask() + }, + 1, + ) + if opts.CheckpointCfg.MetadataCheckInterval != 0 { - db.GCJobs.AddJob( + db.CronJobs.AddJob( "META-CHECK", opts.CheckpointCfg.MetadataCheckInterval, func(ctx context.Context) { @@ -372,8 +381,6 @@ func Open( db.Controller = NewController(db) db.Controller.Start() - go TaeMetricsTask(ctx) - // For debug or test //fmt.Println(db.Catalog.SimplePPString(common.PPL3)) return @@ -391,22 +398,6 @@ func Open( // db.Catalog.RecurLoop(p) // } -func TaeMetricsTask(ctx context.Context) { - logutil.Info("tae metrics task started") - defer logutil.Info("tae metrics task exit") - - timer := time.NewTicker(time.Second * 10) - for { - select { - case <-ctx.Done(): - return - case <-timer.C: - mpoolAllocatorSubTask() - } - } - -} - func mpoolAllocatorSubTask() { v2.MemTAEDefaultAllocatorGauge.Set(float64(common.DefaultAllocator.CurrNB())) v2.MemTAEDefaultHighWaterMarkGauge.Set(float64(common.DefaultAllocator.Stats().HighWaterMark.Load())) From 84cbb4c5e13d60fd4382d53aebd203f70ecd1ef4 Mon Sep 17 00:00:00 2001 From: reusee Date: Tue, 10 Dec 2024 18:39:07 +0800 Subject: [PATCH 06/27] malloc: tune tests to run faster (#20689) make malloc tests run faster Approved by: @XuPeng-SH --- pkg/common/malloc/allocator_test.go | 2 +- pkg/common/malloc/metrics_allocator.go | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/pkg/common/malloc/allocator_test.go b/pkg/common/malloc/allocator_test.go index 9ddd5ca37aeb1..6c587701d28df 100644 --- a/pkg/common/malloc/allocator_test.go +++ b/pkg/common/malloc/allocator_test.go @@ -27,7 +27,7 @@ func testAllocator( t.Run("allocate", func(t *testing.T) { allocator := newAllocator() - for i := uint64(1); i < 128*MB; i = uint64(math.Ceil(float64(i) * 1.1)) { + for i := uint64(1); i < 8*MB; i = uint64(math.Ceil(float64(i) * 1.1)) { // allocate slice, dec, err := allocator.Allocate(i, NoHints) if err != nil { diff --git a/pkg/common/malloc/metrics_allocator.go b/pkg/common/malloc/metrics_allocator.go index fd75482679670..97f17cb1815db 100644 --- a/pkg/common/malloc/metrics_allocator.go +++ b/pkg/common/malloc/metrics_allocator.go @@ -107,9 +107,11 @@ func (m *MetricsAllocator[U]) Allocate(size uint64, hints Hints) ([]byte, Deallo ), nil } +const metricsAllocatorUpdateWindow = time.Millisecond * 100 + func (m *MetricsAllocator[U]) triggerUpdate() { if m.updating.CompareAndSwap(false, true) { - time.AfterFunc(time.Second, func() { + time.AfterFunc(metricsAllocatorUpdateWindow, func() { if m.allocateBytesCounter != nil { var n uint64 From 5f4cf9fcc3d9d81e5bd545fe72e72c4e5803292a Mon Sep 17 00:00:00 2001 From: XuPeng-SH Date: Tue, 10 Dec 2024 19:45:06 +0800 Subject: [PATCH 07/27] prefetch table meta for slow ranges (#20660) prefetch table meta for slow ranges Approved by: @daviszhen, @LeftHandCold, @triump2020, @reusee --- pkg/frontend/test/engine_mock.go | 14 +++++++++++ pkg/objectio/injects.go | 33 ++++++++++++++++++++++++-- pkg/vm/engine/disttae/change_handle.go | 1 + pkg/vm/engine/disttae/engine.go | 6 +++++ pkg/vm/engine/disttae/stats.go | 10 ++++++-- pkg/vm/engine/disttae/transfer.go | 1 + pkg/vm/engine/disttae/txn_database.go | 2 +- pkg/vm/engine/disttae/txn_table.go | 18 ++++++++++++++ pkg/vm/engine/engine_util/exec_util.go | 26 ++++++++++++++++++++ pkg/vm/engine/entire_engine.go | 4 ++++ pkg/vm/engine/entire_engine_test.go | 4 ++++ pkg/vm/engine/memoryengine/binded.go | 3 +++ pkg/vm/engine/memoryengine/engine.go | 4 ++++ pkg/vm/engine/test/reader_test.go | 19 ++++++++++++--- pkg/vm/engine/types.go | 3 +++ 15 files changed, 140 insertions(+), 8 deletions(-) diff --git a/pkg/frontend/test/engine_mock.go b/pkg/frontend/test/engine_mock.go index d532d1ff1f322..85d60f9d793c7 100644 --- a/pkg/frontend/test/engine_mock.go +++ b/pkg/frontend/test/engine_mock.go @@ -2070,6 +2070,20 @@ func (mr *MockEngineMockRecorder) Nodes(isInternal, tenant, username, cnLabel in return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Nodes", reflect.TypeOf((*MockEngine)(nil).Nodes), isInternal, tenant, username, cnLabel) } +// PrefetchTableMeta mocks base method. +func (m *MockEngine) PrefetchTableMeta(ctx context.Context, key statsinfo.StatsInfoKey) bool { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "PrefetchTableMeta", ctx, key) + ret0, _ := ret[0].(bool) + return ret0 +} + +// PrefetchTableMeta indicates an expected call of PrefetchTableMeta. +func (mr *MockEngineMockRecorder) PrefetchTableMeta(ctx, key interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PrefetchTableMeta", reflect.TypeOf((*MockEngine)(nil).PrefetchTableMeta), ctx, key) +} + // Stats mocks base method. func (m *MockEngine) Stats(ctx context.Context, key statsinfo.StatsInfoKey, sync bool) *statsinfo.StatsInfo { m.ctrl.T.Helper() diff --git a/pkg/objectio/injects.go b/pkg/objectio/injects.go index 009e4a5565c30..06afd8c087712 100644 --- a/pkg/objectio/injects.go +++ b/pkg/objectio/injects.go @@ -34,6 +34,7 @@ const ( FJ_TraceRanges = "fj/trace/ranges" FJ_TracePartitionState = "fj/trace/partitionstate" + FJ_PrefetchThreshold = "fj/prefetch/threshold" FJ_Debug19524 = "fj/debug/19524" FJ_Debug19787 = "fj/debug/19787" @@ -153,7 +154,7 @@ func LogReaderInjected(args ...string) (bool, int) { return checkLoggingArgs(int(iarg), sarg, args...) } -func InjectPartitionStateLogging( +func InjectLogPartitionState( databaseName string, tableName string, level int, @@ -274,7 +275,35 @@ func RangesLogInjected(dbName, tableName string) (bool, int) { return checkLoggingArgs(0, sarg, dbName, tableName) } -func InjectRanges( +func InjectPrefetchThreshold(threshold int) (rmFault func(), err error) { + if err = fault.AddFaultPoint( + context.Background(), + FJ_PrefetchThreshold, + ":::", + "echo", + int64(threshold), + "", + false, + ); err != nil { + return + } + rmFault = func() { + fault.RemoveFaultPoint(context.Background(), FJ_PrefetchThreshold) + } + return +} + +// bool: injected or not +// int: threshold. 1 means 1 millisecond +func PrefetchMetaThresholdInjected() (bool, int) { + iarg, _, injected := fault.TriggerFault(FJ_PrefetchThreshold) + if !injected { + return false, 0 + } + return true, int(iarg) +} + +func InjectLogRanges( ctx context.Context, tableName string, ) (rmFault func(), err error) { diff --git a/pkg/vm/engine/disttae/change_handle.go b/pkg/vm/engine/disttae/change_handle.go index 6308026446b8a..d2b2af064b576 100644 --- a/pkg/vm/engine/disttae/change_handle.go +++ b/pkg/vm/engine/disttae/change_handle.go @@ -196,6 +196,7 @@ func (h *CheckpointChangesHandle) initReader(ctx context.Context) (err error) { nil, nil, &blockList, + h.table.PrefetchAllMeta, h.fs, ); err != nil { return diff --git a/pkg/vm/engine/disttae/engine.go b/pkg/vm/engine/disttae/engine.go index 712bd5f261b7d..672a780916cb8 100644 --- a/pkg/vm/engine/disttae/engine.go +++ b/pkg/vm/engine/disttae/engine.go @@ -748,6 +748,12 @@ func (e *Engine) Stats(ctx context.Context, key pb.StatsInfoKey, sync bool) *pb. return e.globalStats.Get(ctx, key, sync) } +// return true if the prefetch is received +// return false if the prefetch is not rejected +func (e *Engine) PrefetchTableMeta(ctx context.Context, key pb.StatsInfoKey) bool { + return e.globalStats.PrefetchTableMeta(ctx, key) +} + func (e *Engine) GetMessageCenter() any { return e.messageCenter } diff --git a/pkg/vm/engine/disttae/stats.go b/pkg/vm/engine/disttae/stats.go index 47cfc172fb471..6f3d84de5356a 100644 --- a/pkg/vm/engine/disttae/stats.go +++ b/pkg/vm/engine/disttae/stats.go @@ -240,6 +240,10 @@ func (gs *GlobalStats) checkTriggerCond(key pb.StatsInfoKey, entryNum int64) boo return true } +func (gs *GlobalStats) PrefetchTableMeta(ctx context.Context, key pb.StatsInfoKey) bool { + return gs.triggerUpdate(key, false) +} + func (gs *GlobalStats) Get(ctx context.Context, key pb.StatsInfoKey, sync bool) *pb.StatsInfo { gs.mu.Lock() defer gs.mu.Unlock() @@ -365,17 +369,19 @@ func (gs *GlobalStats) updateWorker(ctx context.Context) { } } -func (gs *GlobalStats) triggerUpdate(key pb.StatsInfoKey, force bool) { +func (gs *GlobalStats) triggerUpdate(key pb.StatsInfoKey, force bool) bool { if force { gs.updateC <- key v2.StatsTriggerForcedCounter.Add(1) - return + return true } select { case gs.updateC <- key: v2.StatsTriggerUnforcedCounter.Add(1) + return true default: + return false } } diff --git a/pkg/vm/engine/disttae/transfer.go b/pkg/vm/engine/disttae/transfer.go index e9b1b1531f8e5..a9d601863dbe3 100644 --- a/pkg/vm/engine/disttae/transfer.go +++ b/pkg/vm/engine/disttae/transfer.go @@ -474,6 +474,7 @@ func doTransferRowids( objectList, nil, &blockList, + nil, fs, ); err != nil { return diff --git a/pkg/vm/engine/disttae/txn_database.go b/pkg/vm/engine/disttae/txn_database.go index 16496b178236f..be864ce871a3f 100644 --- a/pkg/vm/engine/disttae/txn_database.go +++ b/pkg/vm/engine/disttae/txn_database.go @@ -241,7 +241,7 @@ func (db *txnDatabase) deleteTable(ctx context.Context, name string, forAlter bo rmFault := func() {} if objectio.Debug19524Injected() { - if rmFault, err = objectio.InjectRanges( + if rmFault, err = objectio.InjectLogRanges( ctx, catalog.MO_TABLES, ); err != nil { diff --git a/pkg/vm/engine/disttae/txn_table.go b/pkg/vm/engine/disttae/txn_table.go index 498689afbe0ad..a40c1bbb1d416 100644 --- a/pkg/vm/engine/disttae/txn_table.go +++ b/pkg/vm/engine/disttae/txn_table.go @@ -77,6 +77,23 @@ func (tbl *txnTable) getTxn() *Transaction { return tbl.db.getTxn() } +// true if the prefetch is received +// false if the prefetch is rejected +func (tbl *txnTable) PrefetchAllMeta(ctx context.Context) bool { + // TODO: remove this check + if !tbl.db.op.IsSnapOp() { + return tbl.eng.PrefetchTableMeta( + ctx, + pb.StatsInfoKey{ + AccId: tbl.accountId, + DatabaseID: tbl.db.databaseId, + TableID: tbl.tableId, + }, + ) + } + return true +} + func (tbl *txnTable) Stats(ctx context.Context, sync bool) (*pb.StatsInfo, error) { _, err := tbl.getPartitionState(ctx) if err != nil { @@ -722,6 +739,7 @@ func (tbl *txnTable) rangesOnePart( nil, uncommittedObjects, outBlocks, + tbl.PrefetchAllMeta, tbl.getTxn().engine.fs, ); err != nil { return err diff --git a/pkg/vm/engine/engine_util/exec_util.go b/pkg/vm/engine/engine_util/exec_util.go index 34f3741bf89e5..be583078a0a81 100644 --- a/pkg/vm/engine/engine_util/exec_util.go +++ b/pkg/vm/engine/engine_util/exec_util.go @@ -16,6 +16,7 @@ package engine_util import ( "context" + "time" plan2 "github.com/matrixorigin/matrixone/pkg/sql/plan" "github.com/matrixorigin/matrixone/pkg/vm/engine" @@ -112,6 +113,7 @@ func FilterObjects( extraObjects []objectio.ObjectStats, outBlocks *objectio.BlockInfoSlice, highSelectivityHint bool, + metaPrefetcher func(context.Context) bool, fs fileservice.FileService, ) ( totalBlocks int, @@ -124,6 +126,19 @@ func FilterObjects( fastFilterHit int, err error, ) { + var ( + start time.Time + threshold time.Duration + ) + if metaPrefetcher != nil { + start = time.Now() + ok, ms := objectio.PrefetchMetaThresholdInjected() + if ok { + threshold = time.Duration(ms) * time.Millisecond + } else { + threshold = time.Second * 10 + } + } onObject := func(objStats *objectio.ObjectStats) (err error) { //if need to shuffle objects if plan2.ShouldSkipObjByShuffle(rangesParam.Rsp, objStats) { @@ -139,6 +154,13 @@ func FilterObjects( } } + if metaPrefetcher != nil && time.Since(start) > threshold { + // stop sending new request to prefetch if it was received + if received := metaPrefetcher(ctx); received { + metaPrefetcher = nil + } + } + var ( meta objectio.ObjectMeta bf objectio.BloomFilter @@ -230,6 +252,7 @@ func TryFastFilterBlocks( extraCommittedObjects []objectio.ObjectStats, uncommittedObjects []objectio.ObjectStats, outBlocks *objectio.BlockInfoSlice, + metaPrefetcher func(context.Context) bool, fs fileservice.FileService, ) (ok bool, err error) { fastFilterOp, loadOp, objectFilterOp, blockFilterOp, seekOp, ok, highSelectivityHint := CompileFilterExprs(rangesParam.BlockFilters, tableDef, fs) @@ -250,6 +273,7 @@ func TryFastFilterBlocks( extraCommittedObjects, uncommittedObjects, outBlocks, + metaPrefetcher, fs, highSelectivityHint, ) @@ -269,6 +293,7 @@ func FilterTxnObjects( extraCommittedObjects []objectio.ObjectStats, uncommittedObjects []objectio.ObjectStats, outBlocks *objectio.BlockInfoSlice, + metaPrefetcher func(context.Context) bool, fs fileservice.FileService, highSelectivityHint bool, ) (err error) { @@ -318,6 +343,7 @@ func FilterTxnObjects( extraCommittedObjects, outBlocks, highSelectivityHint, + metaPrefetcher, fs, ) diff --git a/pkg/vm/engine/entire_engine.go b/pkg/vm/engine/entire_engine.go index a62b68909e6c7..c39352041457d 100644 --- a/pkg/vm/engine/entire_engine.go +++ b/pkg/vm/engine/entire_engine.go @@ -106,6 +106,10 @@ func (e *EntireEngine) UnsubscribeTable(ctx context.Context, dbID, tbID uint64) return e.Engine.UnsubscribeTable(ctx, dbID, tbID) } +func (e *EntireEngine) PrefetchTableMeta(ctx context.Context, key pb.StatsInfoKey) bool { + return e.Engine.PrefetchTableMeta(ctx, key) +} + func (e *EntireEngine) Stats(ctx context.Context, key pb.StatsInfoKey, sync bool) *pb.StatsInfo { return e.Engine.Stats(ctx, key, sync) } diff --git a/pkg/vm/engine/entire_engine_test.go b/pkg/vm/engine/entire_engine_test.go index f298f82e47f44..e6216f5904436 100644 --- a/pkg/vm/engine/entire_engine_test.go +++ b/pkg/vm/engine/entire_engine_test.go @@ -330,6 +330,10 @@ func (e *testEngine) UnsubscribeTable(ctx context.Context, dbID, tbID uint64) er return nil } +func (e *testEngine) PrefetchTableMeta(ctx context.Context, key pb.StatsInfoKey) bool { + return true +} + func (e *testEngine) Stats(ctx context.Context, key pb.StatsInfoKey, sync bool) *pb.StatsInfo { return nil } diff --git a/pkg/vm/engine/memoryengine/binded.go b/pkg/vm/engine/memoryengine/binded.go index b9f48cd1803ca..f8a91f8fb32d8 100644 --- a/pkg/vm/engine/memoryengine/binded.go +++ b/pkg/vm/engine/memoryengine/binded.go @@ -109,6 +109,9 @@ func (b *BindedEngine) UnsubscribeTable(ctx context.Context, dbID, tbID uint64) return b.engine.UnsubscribeTable(ctx, dbID, tbID) } +func (b *BindedEngine) PrefetchTableMeta(ctx context.Context, key pb.StatsInfoKey) bool { + return b.engine.PrefetchTableMeta(ctx, key) +} func (b *BindedEngine) Stats(ctx context.Context, key pb.StatsInfoKey, sync bool) *pb.StatsInfo { return b.engine.Stats(ctx, key, sync) } diff --git a/pkg/vm/engine/memoryengine/engine.go b/pkg/vm/engine/memoryengine/engine.go index 35cfc8cc5f934..b1205ecdbf221 100644 --- a/pkg/vm/engine/memoryengine/engine.go +++ b/pkg/vm/engine/memoryengine/engine.go @@ -245,6 +245,10 @@ func (e *Engine) UnsubscribeTable(ctx context.Context, dbID, tbID uint64) error return nil } +func (e *Engine) PrefetchTableMeta(ctx context.Context, key pb.StatsInfoKey) bool { + return true +} + func (e *Engine) Stats(ctx context.Context, key pb.StatsInfoKey, sync bool) *pb.StatsInfo { return nil } diff --git a/pkg/vm/engine/test/reader_test.go b/pkg/vm/engine/test/reader_test.go index 1368835f4f449..90238e919a06b 100644 --- a/pkg/vm/engine/test/reader_test.go +++ b/pkg/vm/engine/test/reader_test.go @@ -279,7 +279,7 @@ func Test_ReaderCanReadCommittedInMemInsertAndDeletes(t *testing.T) { fault.Enable() defer fault.Disable() - rmFault, err := objectio.InjectPartitionStateLogging(objectio.FJ_EmptyDB, catalog.MO_TABLES, 0) + rmFault, err := objectio.InjectLogPartitionState(objectio.FJ_EmptyDB, catalog.MO_TABLES, 0) require.NoError(t, err) defer rmFault() @@ -430,7 +430,7 @@ func Test_ShardingHandler(t *testing.T) { fault.Enable() defer fault.Disable() - rmFault, err := objectio.InjectPartitionStateLogging(catalog.MO_CATALOG, catalog.MO_TABLES, 0) + rmFault, err := objectio.InjectLogPartitionState(catalog.MO_CATALOG, catalog.MO_TABLES, 0) require.NoError(t, err) defer rmFault() @@ -627,7 +627,7 @@ func Test_ShardingRemoteReader(t *testing.T) { fault.Enable() defer fault.Disable() - rmFault, err := objectio.InjectPartitionStateLogging(catalog.MO_CATALOG, objectio.FJ_EmptyTBL, 0) + rmFault, err := objectio.InjectLogPartitionState(catalog.MO_CATALOG, objectio.FJ_EmptyTBL, 0) require.NoError(t, err) defer rmFault() @@ -1105,6 +1105,19 @@ func Test_ShardingLocalReader(t *testing.T) { ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) + fault.Enable() + defer fault.Disable() + rmFault1, err := objectio.InjectLogPartitionState(catalog.MO_CATALOG, objectio.FJ_EmptyTBL, 0) + require.NoError(t, err) + defer rmFault1() + rmFault2, err := objectio.InjectLogRanges(ctx, catalog.MO_TABLES) + require.NoError(t, err) + defer rmFault2() + + rmFault5, err := objectio.InjectPrefetchThreshold(0) + require.NoError(t, err) + defer rmFault5() + // mock a schema with 4 columns and the 4th column as primary key // the first column is the 9th column in the predefined columns in // the mock function. Here we exepct the type of the primary key diff --git a/pkg/vm/engine/types.go b/pkg/vm/engine/types.go index 010384d209590..e0394872df529 100644 --- a/pkg/vm/engine/types.go +++ b/pkg/vm/engine/types.go @@ -1025,6 +1025,9 @@ type Engine interface { // just return nil if the current stats info has not been initialized. Stats(ctx context.Context, key pb.StatsInfoKey, sync bool) *pb.StatsInfo + // true if the prefetch is received, false if the prefetch is rejected + PrefetchTableMeta(ctx context.Context, key pb.StatsInfoKey) bool + GetMessageCenter() any GetService() string From f2e8880c01428c8d6e1e23dbd73a936a7c4d7091 Mon Sep 17 00:00:00 2001 From: Wenbin <85331908+Wenbin1002@users.noreply.github.com> Date: Tue, 10 Dec 2024 21:04:12 +0800 Subject: [PATCH 08/27] fix ut in fault injection (#20679) fix Test_CanTransferCnFaultInject and TestCanTransferQuery and Test_panic Approved by: @m-schen, @XuPeng-SH --- pkg/sql/plan/function/fault/fault_test.go | 6 +-- pkg/util/fault/fault_test.go | 50 ----------------------- 2 files changed, 3 insertions(+), 53 deletions(-) diff --git a/pkg/sql/plan/function/fault/fault_test.go b/pkg/sql/plan/function/fault/fault_test.go index 139a508e237be..dd6cf5a8aecd8 100644 --- a/pkg/sql/plan/function/fault/fault_test.go +++ b/pkg/sql/plan/function/fault/fault_test.go @@ -31,7 +31,7 @@ import ( func Test_CanHandleFaultInjection(t *testing.T) { id := uuid.New().String() - addr := "127.0.0.1:7777" + addr := "127.0.0.1:7775" initRuntime([]string{id}, []string{addr}) runtime.SetupServiceBasedRuntime(id, runtime.ServiceRuntime("")) @@ -140,8 +140,8 @@ func Test_CanTransferCnFaultInject(t *testing.T) { uuid.New().String(), } addrs := []string{ - "127.0.0.1:7777", - "127.0.0.1:5555", + "127.0.0.1:7775", + "127.0.0.1:5557", } a1.proc = new(process.Process) diff --git a/pkg/util/fault/fault_test.go b/pkg/util/fault/fault_test.go index 51fc7847d2d52..8b98f47a7f922 100644 --- a/pkg/util/fault/fault_test.go +++ b/pkg/util/fault/fault_test.go @@ -182,56 +182,6 @@ func TestWait(t *testing.T) { Disable() } -func TestFaultMapRace(t *testing.T) { - var ctx = context.TODO() - - Enable() - require.NoError(t, AddFaultPoint(ctx, "a", ":::", "return", 0, "", false)) - - go func() { - Disable() - }() - - go func() { - TriggerFault("a") - }() - - Disable() -} - -func TestFaultMapRace2(t *testing.T) { - go func() { - Enable() - }() - - go func() { - Enable() - }() - - Disable() -} - -func TestFaultMapRace3(t *testing.T) { - var ctx = context.TODO() - - Enable() - require.NoError(t, AddFaultPoint(ctx, "a", ":::", "return", 0, "", false)) - - go func() { - Disable() - }() - - go func() { - TriggerFault("a") - }() - - go func() { - Disable() - }() - - Disable() -} - func Test_panic(t *testing.T) { var ctx = context.TODO() From 1b9186ae6fc5fd8038aa514bea8899ffd24482af Mon Sep 17 00:00:00 2001 From: LiuBo Date: Tue, 10 Dec 2024 22:03:37 +0800 Subject: [PATCH 09/27] [improvement] stats: add some logs and UT (#20619) add some logs and UT for stats Approved by: @XuPeng-SH, @daviszhen --- pkg/cnservice/server_query.go | 3 +- pkg/vm/engine/disttae/engine.go | 13 +- pkg/vm/engine/disttae/logtail_consumer.go | 10 +- pkg/vm/engine/disttae/mo_table_stats.go | 350 +++++++++--------- pkg/vm/engine/disttae/stats.go | 76 ++-- pkg/vm/engine/disttae/stats_test.go | 218 +++++++++++ pkg/vm/engine/disttae/types.go | 2 + pkg/vm/engine/test/change_handle_test.go | 57 ++- pkg/vm/engine/test/disttae_engine_test.go | 18 +- pkg/vm/engine/test/mo_table_stats_test.go | 46 ++- pkg/vm/engine/test/reader_test.go | 35 +- pkg/vm/engine/test/testutil/disttae_engine.go | 12 +- pkg/vm/engine/test/testutil/logtailserver.go | 7 +- pkg/vm/engine/test/testutil/tae_engine.go | 14 +- pkg/vm/engine/test/testutil/util.go | 24 +- pkg/vm/engine/test/workspace_test.go | 60 +-- 16 files changed, 651 insertions(+), 294 deletions(-) diff --git a/pkg/cnservice/server_query.go b/pkg/cnservice/server_query.go index b15f93c6d7da8..b0528999f94fb 100644 --- a/pkg/cnservice/server_query.go +++ b/pkg/cnservice/server_query.go @@ -157,7 +157,8 @@ func (s *service) handleFaultInjection(ctx context.Context, req *query.Request, } func (s *service) handleMoTableStats(ctx context.Context, req *query.Request, resp *query.Response, _ *morpc.Buffer) error { - ret := disttae.HandleMoTableStatsCtl(req.CtlMoTableStatsRequest.Cmd) + e := s.storeEngine.(*disttae.Engine) + ret := e.HandleMoTableStatsCtl(req.CtlMoTableStatsRequest.Cmd) resp.CtlMoTableStatsResponse = query.CtlMoTableStatsResponse{ Resp: ret, } diff --git a/pkg/vm/engine/disttae/engine.go b/pkg/vm/engine/disttae/engine.go index 672a780916cb8..a5e21b231b781 100644 --- a/pkg/vm/engine/disttae/engine.go +++ b/pkg/vm/engine/disttae/engine.go @@ -137,11 +137,22 @@ func New( e.pClient.LogtailRPCClientFactory = DefaultNewRpcStreamToTnLogTailService e.pClient.ctx = ctx - initMoTableStatsConfig(ctx, e) + err = initMoTableStatsConfig(ctx, e) + if err != nil { + panic(err) + } return e } +func (e *Engine) Close() error { + if e.gcPool != nil { + e.gcPool.Release() + } + e.dynamicCtx.Close() + return nil +} + func (e *Engine) fillDefaults() { if e.config.insertEntryMaxCount <= 0 { e.config.insertEntryMaxCount = InsertEntryThreshold diff --git a/pkg/vm/engine/disttae/logtail_consumer.go b/pkg/vm/engine/disttae/logtail_consumer.go index 3be32195f3629..3626a522d3b67 100644 --- a/pkg/vm/engine/disttae/logtail_consumer.go +++ b/pkg/vm/engine/disttae/logtail_consumer.go @@ -621,7 +621,12 @@ func (c *PushClient) receiveLogtails(ctx context.Context, e *Engine) { } // Wait for resuming logtail receiver. - <-c.resumeC + select { + case <-ctx.Done(): + return + + case <-c.resumeC: + } logutil.Infof("%s logtail receiver resumed", logTag) default: @@ -1835,6 +1840,9 @@ func (c *PushClient) createRoutineToConsumeLogTails( errHappen := false for { select { + case <-ctx.Done(): + return + case cmd := <-receiver.signalChan: if errHappen { continue diff --git a/pkg/vm/engine/disttae/mo_table_stats.go b/pkg/vm/engine/disttae/mo_table_stats.go index fe0e6cba0b849..746e171f9a8d3 100644 --- a/pkg/vm/engine/disttae/mo_table_stats.go +++ b/pkg/vm/engine/disttae/mo_table_stats.go @@ -244,11 +244,10 @@ func initMoTableStatsConfig( ctx context.Context, eng *Engine, ) (err error) { - - dynamicCtx.once.Do(func() { + eng.dynamicCtx.once.Do(func() { defer func() { - dynamicCtx.defaultConf = dynamicCtx.conf + eng.dynamicCtx.defaultConf = eng.dynamicCtx.conf if err != nil { logutil.Error(logHeader, zap.String("source", "init mo table stats config"), @@ -256,49 +255,49 @@ func initMoTableStatsConfig( } }() - dynamicCtx.de = eng + eng.dynamicCtx.de = eng - if dynamicCtx.alphaTaskPool, err = ants.NewPool( + if eng.dynamicCtx.alphaTaskPool, err = ants.NewPool( runtime.NumCPU(), ants.WithNonblocking(false)); err != nil { return } - dynamicCtx.conf = eng.config.statsConf + eng.dynamicCtx.conf = eng.config.statsConf - function.MoTableRowsSizeUseOldImpl.Store(dynamicCtx.conf.DisableStatsTask) + function.MoTableRowsSizeUseOldImpl.Store(eng.dynamicCtx.conf.DisableStatsTask) - dynamicCtx.executorPool = sync.Pool{ + eng.dynamicCtx.executorPool = sync.Pool{ New: func() interface{} { return eng.config.ieFactory() }, } - dynamicCtx.sqlOpts = ie.NewOptsBuilder().Database(catalog.MO_CATALOG).Internal(true).Finish() + eng.dynamicCtx.sqlOpts = ie.NewOptsBuilder().Database(catalog.MO_CATALOG).Internal(true).Finish() - if dynamicCtx.conf.GetTableListLimit <= 0 { - dynamicCtx.conf.GetTableListLimit = defaultGetTableListLimit + if eng.dynamicCtx.conf.GetTableListLimit <= 0 { + eng.dynamicCtx.conf.GetTableListLimit = defaultGetTableListLimit } - if dynamicCtx.conf.UpdateDuration <= 0 { - dynamicCtx.conf.UpdateDuration = defaultAlphaCycleDur + if eng.dynamicCtx.conf.UpdateDuration <= 0 { + eng.dynamicCtx.conf.UpdateDuration = defaultAlphaCycleDur } - if dynamicCtx.conf.CorrectionDuration <= 0 { - dynamicCtx.conf.CorrectionDuration = defaultGamaCycleDur + if eng.dynamicCtx.conf.CorrectionDuration <= 0 { + eng.dynamicCtx.conf.CorrectionDuration = defaultGamaCycleDur } - dynamicCtx.objIdsPool = sync.Pool{ + eng.dynamicCtx.objIdsPool = sync.Pool{ New: func() interface{} { objIds := make([]types.Objectid, 0) return &objIds }, } - dynamicCtx.tblQueue = make(chan tablePair, options.DefaultBlockMaxRows*2) + eng.dynamicCtx.tblQueue = make(chan tablePair, options.DefaultBlockMaxRows*2) - dynamicCtx.cleanDeletesQueue = make(chan struct{}) - dynamicCtx.updateForgottenQueue = make(chan struct{}) + eng.dynamicCtx.cleanDeletesQueue = make(chan struct{}) + eng.dynamicCtx.updateForgottenQueue = make(chan struct{}) // registerMoTableSizeRows { @@ -306,7 +305,7 @@ func initMoTableStatsConfig( context.Context, []uint64, []uint64, []uint64, engine.Engine, bool, bool) ([]uint64, error) { - return MTSTableSize + return eng.dynamicCtx.MTSTableSize } function.GetMoTableSizeFunc.Store(&ff1) @@ -314,18 +313,18 @@ func initMoTableStatsConfig( context.Context, []uint64, []uint64, []uint64, engine.Engine, bool, bool) ([]uint64, error) { - return MTSTableRows + return eng.dynamicCtx.MTSTableRows } function.GetMoTableRowsFunc.Store(&ff2) } - dynamicCtx.tableStock.tbls = make([]tablePair, 0, 1) + eng.dynamicCtx.tableStock.tbls = make([]tablePair, 0, 1) { - dynamicCtx.beta.executor = betaTask - dynamicCtx.gama.executor = gamaTask + eng.dynamicCtx.beta.executor = eng.dynamicCtx.betaTask + eng.dynamicCtx.gama.executor = eng.dynamicCtx.gamaTask - if dynamicCtx.beta.taskPool, err = ants.NewPool( + if eng.dynamicCtx.beta.taskPool, err = ants.NewPool( runtime.NumCPU(), ants.WithNonblocking(false), ants.WithPanicHandler(func(e interface{}) { @@ -336,7 +335,7 @@ func initMoTableStatsConfig( return } - if dynamicCtx.gama.taskPool, err = ants.NewPool( + if eng.dynamicCtx.gama.taskPool, err = ants.NewPool( runtime.NumCPU(), ants.WithNonblocking(false), ants.WithPanicHandler(func(e interface{}) { @@ -362,26 +361,26 @@ func initMoTableStatsConfig( go func() { defer func() { - dynamicCtx.Lock() + eng.dynamicCtx.Lock() task.running = false - dynamicCtx.Unlock() + eng.dynamicCtx.Unlock() }() // there should not have a deadline - taskCtx := turn2SysCtx(context.Background()) + taskCtx := turn2SysCtx(ctx) task.executor(taskCtx, eng.service, eng) }() } - dynamicCtx.launchTask = func() { - dynamicCtx.Lock() - defer dynamicCtx.Unlock() + eng.dynamicCtx.launchTask = func() { + eng.dynamicCtx.Lock() + defer eng.dynamicCtx.Unlock() - launch("beta task", &dynamicCtx.beta) - launch("gama task", &dynamicCtx.gama) + launch("beta task", &eng.dynamicCtx.beta) + launch("gama task", &eng.dynamicCtx.gama) } - dynamicCtx.launchTask() + eng.dynamicCtx.launchTask() }) return err @@ -394,7 +393,7 @@ type taskState struct { launchTimes int } -var dynamicCtx struct { +type dynamicCtx struct { sync.RWMutex once sync.Once @@ -407,7 +406,6 @@ var dynamicCtx struct { updateForgottenQueue chan struct{} de *Engine - service string objIdsPool sync.Pool tableStock struct { @@ -427,9 +425,21 @@ var dynamicCtx struct { sqlOpts ie.SessionOverrideOptions } +func (d *dynamicCtx) Close() { + if d.alphaTaskPool != nil { + d.alphaTaskPool.Release() + } + if d.beta.taskPool != nil { + d.beta.taskPool.Release() + } + if d.gama.taskPool != nil { + d.gama.taskPool.Release() + } +} + ////////////////// MoTableStats Interface ////////////////// -func HandleMoTableStatsCtl(cmd string) string { +func (d *dynamicCtx) HandleMoTableStatsCtl(cmd string) string { cmds := strings.Split(cmd, ":") if len(cmds) != 2 { @@ -447,30 +457,30 @@ func HandleMoTableStatsCtl(cmd string) string { switch typ { case "use_old_impl": - return setUseOldImpl(val == "true") + return d.setUseOldImpl(val == "true") case "force_update": - return setForceUpdate(val == "true") + return d.setForceUpdate(val == "true") case "move_on": - return setMoveOnTask(val == "true") + return d.setMoveOnTask(val == "true") case "restore_default_setting": - return restoreDefaultSetting(val == "true") + return d.restoreDefaultSetting(val == "true") case "echo_current_setting": - return echoCurrentSetting(val == "true") + return d.echoCurrentSetting(val == "true") default: return "failed, cmd invalid" } } -func checkMoveOnTask() bool { - dynamicCtx.Lock() - defer dynamicCtx.Unlock() +func (d *dynamicCtx) checkMoveOnTask() bool { + d.Lock() + defer d.Unlock() - disable := dynamicCtx.conf.DisableStatsTask + disable := d.conf.DisableStatsTask logutil.Info(logHeader, zap.String("source", "check move on"), @@ -479,44 +489,44 @@ func checkMoveOnTask() bool { return disable } -func echoCurrentSetting(ok bool) string { +func (d *dynamicCtx) echoCurrentSetting(ok bool) string { if !ok { return "noop" } - dynamicCtx.Lock() - defer dynamicCtx.Unlock() + d.Lock() + defer d.Unlock() return fmt.Sprintf("move_on(%v), use_old_impl(%v), force_update(%v)", - !dynamicCtx.conf.DisableStatsTask, - dynamicCtx.conf.StatsUsingOldImpl, - dynamicCtx.conf.ForceUpdate) + !d.conf.DisableStatsTask, + d.conf.StatsUsingOldImpl, + d.conf.ForceUpdate) } -func restoreDefaultSetting(ok bool) string { +func (d *dynamicCtx) restoreDefaultSetting(ok bool) string { if !ok { return "noop" } - dynamicCtx.Lock() - defer dynamicCtx.Unlock() + d.Lock() + defer d.Unlock() - dynamicCtx.conf = dynamicCtx.defaultConf - function.MoTableRowsSizeUseOldImpl.Store(dynamicCtx.conf.StatsUsingOldImpl) - function.MoTableRowsSizeForceUpdate.Store(dynamicCtx.conf.ForceUpdate) + d.conf = d.defaultConf + function.MoTableRowsSizeUseOldImpl.Store(d.conf.StatsUsingOldImpl) + function.MoTableRowsSizeForceUpdate.Store(d.conf.ForceUpdate) return fmt.Sprintf("move_on(%v), use_old_impl(%v), force_update(%v)", - !dynamicCtx.conf.DisableStatsTask, - dynamicCtx.conf.StatsUsingOldImpl, - dynamicCtx.conf.ForceUpdate) + !d.conf.DisableStatsTask, + d.conf.StatsUsingOldImpl, + d.conf.ForceUpdate) } -func setMoveOnTask(newVal bool) string { - dynamicCtx.Lock() - defer dynamicCtx.Unlock() +func (d *dynamicCtx) setMoveOnTask(newVal bool) string { + d.Lock() + defer d.Unlock() - oldState := !dynamicCtx.conf.DisableStatsTask - dynamicCtx.conf.DisableStatsTask = !newVal + oldState := !d.conf.DisableStatsTask + d.conf.DisableStatsTask = !newVal ret := fmt.Sprintf("move on: %v to %v", oldState, newVal) logutil.Info(logHeader, @@ -526,13 +536,13 @@ func setMoveOnTask(newVal bool) string { return ret } -func setUseOldImpl(newVal bool) string { - dynamicCtx.Lock() - defer dynamicCtx.Unlock() +func (d *dynamicCtx) setUseOldImpl(newVal bool) string { + d.Lock() + defer d.Unlock() - oldState := dynamicCtx.conf.StatsUsingOldImpl + oldState := d.conf.StatsUsingOldImpl function.MoTableRowsSizeUseOldImpl.Store(newVal) - dynamicCtx.conf.StatsUsingOldImpl = newVal + d.conf.StatsUsingOldImpl = newVal ret := fmt.Sprintf("use old impl: %v to %v", oldState, newVal) logutil.Info(logHeader, @@ -542,13 +552,13 @@ func setUseOldImpl(newVal bool) string { return ret } -func setForceUpdate(newVal bool) string { - dynamicCtx.Lock() - defer dynamicCtx.Unlock() +func (d *dynamicCtx) setForceUpdate(newVal bool) string { + d.Lock() + defer d.Unlock() - oldState := dynamicCtx.conf.ForceUpdate + oldState := d.conf.ForceUpdate function.MoTableRowsSizeForceUpdate.Store(newVal) - dynamicCtx.conf.ForceUpdate = newVal + d.conf.ForceUpdate = newVal ret := fmt.Sprintf("force update: %v to %v", oldState, newVal) logutil.Info(logHeader, @@ -558,9 +568,9 @@ func setForceUpdate(newVal bool) string { return ret } -func executeSQL(ctx context.Context, sql string, hint string) ie.InternalExecResult { - exec := dynamicCtx.executorPool.Get() - defer dynamicCtx.executorPool.Put(exec) +func (d *dynamicCtx) executeSQL(ctx context.Context, sql string, hint string) ie.InternalExecResult { + exec := d.executorPool.Get() + defer d.executorPool.Put(exec) var ( newCtx = ctx @@ -572,7 +582,7 @@ func executeSQL(ctx context.Context, sql string, hint string) ie.InternalExecRes defer cancel() } - ret := exec.(ie.InternalExecutor).Query(newCtx, sql, dynamicCtx.sqlOpts) + ret := exec.(ie.InternalExecutor).Query(newCtx, sql, d.sqlOpts) if ret.Error() != nil { logutil.Info(logHeader, zap.String("source", hint), @@ -594,7 +604,7 @@ func intsJoin(items []uint64, delimiter string) string { return str } -func forceUpdateQuery( +func (d *dynamicCtx) forceUpdateQuery( ctx context.Context, wantedStatsIdxes []int, accs, dbs, tbls []uint64, @@ -626,7 +636,7 @@ func forceUpdateQuery( intsJoin(dbs, ","), intsJoin(tbls, ",")) - sqlRet := executeSQL(ctx, sql, "force update query") + sqlRet := d.executeSQL(ctx, sql, "force update query") if sqlRet.Error() != nil { return nil, sqlRet.Error() } @@ -667,11 +677,11 @@ func forceUpdateQuery( } } - if err = alphaTask(ctx, eng.service, eng, pairs, "forceUpdateQuery"); err != nil { + if err = d.alphaTask(ctx, eng.service, eng, pairs, "forceUpdateQuery"); err != nil { return nil, err } - if statsVals, err = normalQuery( + if statsVals, err = d.normalQuery( ctx, wantedStatsIdxes, accs, dbs, tbls); err != nil { return nil, err @@ -680,7 +690,7 @@ func forceUpdateQuery( return } -func normalQuery( +func (d *dynamicCtx) normalQuery( ctx context.Context, wantedStatsIdxes []int, accs, dbs, tbls []uint64, @@ -697,7 +707,7 @@ func normalQuery( intsJoin(dbs, ","), intsJoin(tbls, ",")) - sqlRet := executeSQL(ctx, sql, "normal query") + sqlRet := d.executeSQL(ctx, sql, "normal query") if sqlRet.Error() != nil { return nil, sqlRet.Error() } @@ -757,7 +767,7 @@ func normalQuery( return statsVals, nil } -func QueryTableStats( +func (d *dynamicCtx) QueryTableStats( ctx context.Context, wantedStatsIdxes []int, accs, dbs, tbls []uint64, @@ -766,17 +776,17 @@ func QueryTableStats( eng engine.Engine, ) (statsVals [][]any, err error) { - dynamicCtx.Lock() - useOld := dynamicCtx.conf.StatsUsingOldImpl - dynamicCtx.Unlock() + d.Lock() + useOld := d.conf.StatsUsingOldImpl + d.Unlock() if useOld { return } if eng == nil { - dynamicCtx.Lock() - eng = dynamicCtx.de - dynamicCtx.Unlock() + d.Lock() + eng = d.de + d.Unlock() } var now = time.Now() @@ -809,17 +819,17 @@ func QueryTableStats( de = eng.(*Engine) } - return forceUpdateQuery( + return d.forceUpdateQuery( newCtx, wantedStatsIdxes, accs, dbs, tbls, resetUpdateTime, de) } - return normalQuery(newCtx, wantedStatsIdxes, accs, dbs, tbls) + return d.normalQuery(newCtx, wantedStatsIdxes, accs, dbs, tbls) } -func MTSTableSize( +func (d *dynamicCtx) MTSTableSize( ctx context.Context, accs, dbs, tbls []uint64, eng engine.Engine, @@ -827,7 +837,7 @@ func MTSTableSize( resetUpdateTime bool, ) (sizes []uint64, err error) { - statsVals, err := QueryTableStats( + statsVals, err := d.QueryTableStats( ctx, []int{TableStatsTableSize}, accs, dbs, tbls, forceUpdate, resetUpdateTime, eng) @@ -846,7 +856,7 @@ func MTSTableSize( return } -func MTSTableRows( +func (d *dynamicCtx) MTSTableRows( ctx context.Context, accs, dbs, tbls []uint64, eng engine.Engine, @@ -854,7 +864,7 @@ func MTSTableRows( resetUpdateTime bool, ) (sizes []uint64, err error) { - statsVals, err := QueryTableStats( + statsVals, err := d.QueryTableStats( ctx, []int{TableStatsTableRows}, accs, dbs, tbls, forceUpdate, resetUpdateTime, eng) @@ -1000,20 +1010,20 @@ func GetMOTableStatsExecutor( sqlExecutor func() ie.InternalExecutor, ) func(ctx context.Context, task task.Task) error { return func(ctx context.Context, task task.Task) error { - return tableStatsExecutor(ctx, service, eng) + return eng.(*Engine).dynamicCtx.tableStatsExecutor(ctx, service, eng) } } func turn2SysCtx(ctx context.Context) context.Context { newCtx := ctx if val := ctx.Value(defines.TenantIDKey{}); val == nil || val.(uint32) != catalog.System_Account { - newCtx = context.WithValue(context.Background(), defines.TenantIDKey{}, catalog.System_Account) + newCtx = context.WithValue(ctx, defines.TenantIDKey{}, catalog.System_Account) } return newCtx } -func tableStatsExecutor( +func (d *dynamicCtx) tableStatsExecutor( ctx context.Context, service string, eng engine.Engine, @@ -1033,19 +1043,19 @@ func tableStatsExecutor( return newCtx.Err() case <-executeTicker.C: - if checkMoveOnTask() { + if d.checkMoveOnTask() { continue } - if err = prepare(newCtx, service, eng); err != nil { + if err = d.prepare(newCtx, service, eng); err != nil { return err } - dynamicCtx.Lock() - tbls := dynamicCtx.tableStock.tbls[:] - dynamicCtx.Unlock() + d.Lock() + tbls := d.tableStock.tbls[:] + d.Unlock() - if err = alphaTask( + if err = d.alphaTask( newCtx, service, eng, tbls, "main routine", @@ -1056,15 +1066,15 @@ func tableStatsExecutor( return err } - dynamicCtx.Lock() - executeTicker.Reset(dynamicCtx.conf.UpdateDuration) - dynamicCtx.tableStock.tbls = dynamicCtx.tableStock.tbls[:0] - dynamicCtx.Unlock() + d.Lock() + executeTicker.Reset(d.conf.UpdateDuration) + d.tableStock.tbls = d.tableStock.tbls[:0] + d.Unlock() } } } -func insertNewTables( +func (d *dynamicCtx) insertNewTables( ctx context.Context, service string, eng engine.Engine, @@ -1082,7 +1092,7 @@ func insertNewTables( //if dynamicCtx.lastCheckNewTables.IsEmpty() { sql = fmt.Sprintf(getMinTSSQL, catalog.MO_CATALOG, catalog.MO_TABLE_STATS) - sqlRet = executeSQL(ctx, sql, "insert new table-0: get min ts") + sqlRet = d.executeSQL(ctx, sql, "insert new table-0: get min ts") if err = sqlRet.Error(); err != nil { return err } @@ -1096,20 +1106,20 @@ func insertNewTables( return } - dynamicCtx.lastCheckNewTables = types.BuildTS(tm.UnixNano(), 0) + d.lastCheckNewTables = types.BuildTS(tm.UnixNano(), 0) } //} sql = fmt.Sprintf(getNewTablesSQL, catalog.MO_CATALOG, catalog.MO_TABLES, - dynamicCtx.lastCheckNewTables. + d.lastCheckNewTables. ToTimestamp(). ToStdTime(). Format("2006-01-02 15:04:05"), ) - sqlRet = executeSQL(ctx, sql, "insert new table-1: get new tables") + sqlRet = d.executeSQL(ctx, sql, "insert new table-1: get new tables") if err = sqlRet.Error(); err != nil { return err } @@ -1157,26 +1167,26 @@ func insertNewTables( catalog.MO_CATALOG, catalog.MO_TABLE_STATS, strings.Join(values, ",")) - sqlRet = executeSQL(ctx, sql, "insert new table-2: insert new tables") + sqlRet = d.executeSQL(ctx, sql, "insert new table-2: insert new tables") return sqlRet.Error() } -func prepare( +func (d *dynamicCtx) prepare( ctx context.Context, service string, eng engine.Engine, ) (err error) { - dynamicCtx.Lock() - defer dynamicCtx.Unlock() + d.Lock() + defer d.Unlock() - if err = insertNewTables(ctx, service, eng); err != nil { + if err = d.insertNewTables(ctx, service, eng); err != nil { return } offsetTS := types.TS{} - for len(dynamicCtx.tableStock.tbls) == 0 { - accs, dbs, tbls, ts, err := getCandidates(ctx, service, eng, dynamicCtx.conf.GetTableListLimit, offsetTS) + for len(d.tableStock.tbls) == 0 { + accs, dbs, tbls, ts, err := d.getCandidates(ctx, service, eng, d.conf.GetTableListLimit, offsetTS) if err != nil { return err } @@ -1187,8 +1197,8 @@ func prepare( err = getChangedTableList( ctx, service, eng, accs, dbs, tbls, ts, - &dynamicCtx.tableStock.tbls, - &dynamicCtx.tableStock.newest) + &d.tableStock.tbls, + &d.tableStock.newest) if err != nil { return err @@ -1197,13 +1207,13 @@ func prepare( // in case of all candidates have been deleted. offsetTS = types.TimestampToTS(*ts[len(ts)-1]) - if len(dynamicCtx.tableStock.tbls) == 0 && offsetTS.IsEmpty() { + if len(d.tableStock.tbls) == 0 && offsetTS.IsEmpty() { // there exists a large number of deleted table which are new inserts logutil.Info(logHeader, zap.String("source", "prepare"), zap.String("info", "found new inserts deletes, force clean")) - NotifyCleanDeletes() + d.NotifyCleanDeletes() break } } @@ -1211,7 +1221,7 @@ func prepare( return err } -func alphaTask( +func (d *dynamicCtx) alphaTask( ctx context.Context, service string, eng engine.Engine, @@ -1224,7 +1234,7 @@ func alphaTask( } // maybe the task exited, need to launch a new one - dynamicCtx.launchTask() + d.launchTask() var ( errWaitToReceive = len(tbls) @@ -1282,7 +1292,7 @@ func alphaTask( case <-ticker.C: if len(tbls) == 0 { // all submitted - dynamicCtx.tblQueue <- tablePair{} + d.tblQueue <- tablePair{} ticker.Reset(time.Second) if enterWait { @@ -1314,7 +1324,7 @@ func alphaTask( submitted++ wg.Add(1) - err = dynamicCtx.alphaTaskPool.Submit(func() { + err = d.alphaTaskPool.Submit(func() { defer wg.Done() var err2 error @@ -1333,7 +1343,7 @@ func alphaTask( tbls[i].pState = pState tbls[i].errChan = errQueue - dynamicCtx.tblQueue <- tbls[i] + d.tblQueue <- tbls[i] }) if err != nil { @@ -1349,7 +1359,7 @@ func alphaTask( wg.Wait() // let beta know that a batch done - dynamicCtx.tblQueue <- tablePair{} + d.tblQueue <- tablePair{} dur := time.Since(start) // the longer the update takes, the longer we would pause, @@ -1362,7 +1372,7 @@ func alphaTask( } } -func betaTask( +func (d *dynamicCtx) betaTask( ctx context.Context, service string, eng engine.Engine, @@ -1388,13 +1398,13 @@ func betaTask( err = ctx.Err() return - case tbl := <-dynamicCtx.tblQueue: + case tbl := <-d.tblQueue: if !tbl.valid { // an alpha batch transmit done - bulkUpdateTableOnlyTS(ctx, service, onlyTSBat) + d.bulkUpdateTableOnlyTS(ctx, service, onlyTSBat) bulkWait.Wait() - _ = bulkUpdateTableStatsList(ctx, service, &slBat) + _ = d.bulkUpdateTableStatsList(ctx, service, &slBat) slBat.Clear() onlyTSBat = onlyTSBat[:0] @@ -1411,10 +1421,10 @@ func betaTask( } bulkWait.Add(1) - if err = dynamicCtx.beta.taskPool.Submit(func() { + if err = d.beta.taskPool.Submit(func() { defer bulkWait.Done() - sl, err2 := statsCalculateOp(ctx, service, de.fs, tbl.snapshot, tbl.pState) + sl, err2 := d.statsCalculateOp(ctx, service, de.fs, tbl.snapshot, tbl.pState) if err2 != nil { tbl.Done(err2) } else { @@ -1429,15 +1439,15 @@ func betaTask( } } -func NotifyCleanDeletes() { - dynamicCtx.cleanDeletesQueue <- struct{}{} +func (d *dynamicCtx) NotifyCleanDeletes() { + d.cleanDeletesQueue <- struct{}{} } -func NotifyUpdateForgotten() { - dynamicCtx.updateForgottenQueue <- struct{}{} +func (d *dynamicCtx) NotifyUpdateForgotten() { + d.updateForgottenQueue <- struct{}{} } -func gamaTask( +func (d *dynamicCtx) gamaTask( ctx context.Context, service string, eng engine.Engine, @@ -1454,10 +1464,10 @@ func gamaTask( }) cnCnt = max(cnCnt, 1) - dynamicCtx.Lock() - gamaDur := dynamicCtx.conf.CorrectionDuration - gamaLimit := max(dynamicCtx.conf.GetTableListLimit/100, 100) - dynamicCtx.Unlock() + d.Lock() + gamaDur := d.conf.CorrectionDuration + gamaLimit := max(d.conf.GetTableListLimit/100, 100) + d.Unlock() decodeIdsFromSqlRet := func( sqlRet ie.InternalExecResult, @@ -1499,7 +1509,7 @@ func gamaTask( now := time.Now() sql := fmt.Sprintf(getNullStatsSQL, catalog.MO_CATALOG, catalog.MO_TABLE_STATS, gamaLimit) - sqlRet := executeSQL(ctx, sql, "gama task: get null stats list") + sqlRet := d.executeSQL(ctx, sql, "gama task: get null stats list") if sqlRet.Error() != nil { return } @@ -1522,7 +1532,7 @@ func gamaTask( tbls = append(tbls, tbl) } - if err = alphaTask( + if err = d.alphaTask( ctx, service, eng, tbls, "gama opA"); err != nil { return } @@ -1556,7 +1566,7 @@ func gamaTask( colName2[step], colName2[step], catalog.MO_CATALOG, catalog.MO_TABLE_STATS, colName2[step], options.DefaultBlockMaxRows) - sqlRet := executeSQL(ctx, sql, fmt.Sprintf("gama task-%d-0", step)) + sqlRet := d.executeSQL(ctx, sql, fmt.Sprintf("gama task-%d-0", step)) if sqlRet.Error() != nil { return } @@ -1569,7 +1579,7 @@ func gamaTask( sql = fmt.Sprintf(getCheckAliveSQL, colName1[step], catalog.MO_CATALOG, tblName[step], colName1[step], intsJoin(ids, ",")) - sqlRet = executeSQL(ctx, sql, fmt.Sprintf("gama task-%d-1", step)) + sqlRet = d.executeSQL(ctx, sql, fmt.Sprintf("gama task-%d-1", step)) if sqlRet.Error() != nil { return } @@ -1595,7 +1605,7 @@ func gamaTask( if len(ids) != 0 { sql = fmt.Sprintf(getDeleteFromStatsSQL, catalog.MO_CATALOG, catalog.MO_TABLE_STATS, colName2[step], intsJoin(ids, ",")) - sqlRet = executeSQL(ctx, sql, fmt.Sprintf("gama task-%d-2", step)) + sqlRet = d.executeSQL(ctx, sql, fmt.Sprintf("gama task-%d-2", step)) if sqlRet.Error() != nil { return } @@ -1636,26 +1646,26 @@ func gamaTask( return case <-tickerA.C: - dynamicCtx.gama.taskPool.Submit(opA) + d.gama.taskPool.Submit(opA) tickerA.Reset(randDuration()) - case <-dynamicCtx.updateForgottenQueue: - dynamicCtx.gama.taskPool.Submit(opA) + case <-d.updateForgottenQueue: + d.gama.taskPool.Submit(opA) tickerA.Reset(randDuration()) case <-tickerB.C: - dynamicCtx.gama.taskPool.Submit(opB) + d.gama.taskPool.Submit(opB) tickerB.Reset(randDuration()) - case <-dynamicCtx.cleanDeletesQueue: + case <-d.cleanDeletesQueue: // emergence, do clean now - dynamicCtx.gama.taskPool.Submit(opB) + d.gama.taskPool.Submit(opB) tickerB.Reset(randDuration()) } } } -func statsCalculateOp( +func (d *dynamicCtx) statsCalculateOp( ctx context.Context, service string, fs fileservice.FileService, @@ -1666,12 +1676,12 @@ func statsCalculateOp( bcs := betaCycleStash{ born: time.Now(), snapshot: snapshot, - dataObjIds: dynamicCtx.objIdsPool.Get().(*[]types.Objectid), + dataObjIds: d.objIdsPool.Get().(*[]types.Objectid), } defer func() { *bcs.dataObjIds = (*bcs.dataObjIds)[:0] - dynamicCtx.objIdsPool.Put(bcs.dataObjIds) + d.objIdsPool.Put(bcs.dataObjIds) }() if err = collectVisibleData(&bcs, pState); err != nil { @@ -1689,7 +1699,7 @@ func statsCalculateOp( return sl, nil } -func getCandidates( +func (d *dynamicCtx) getCandidates( ctx context.Context, service string, eng engine.Engine, @@ -1711,7 +1721,7 @@ func getCandidates( } sql = fmt.Sprintf(getNextReadyListSQL, catalog.MO_CATALOG, catalog.MO_TABLE_STATS, where, limit) - sqlRet = executeSQL(ctx, sql, "get next ready to update list") + sqlRet = d.executeSQL(ctx, sql, "get next ready to update list") if err = sqlRet.Error(); err != nil { return } @@ -2069,7 +2079,7 @@ func applyTombstones( return nil } -func bulkUpdateTableStatsList( +func (d *dynamicCtx) bulkUpdateTableStatsList( ctx context.Context, service string, bat *sync.Map, @@ -2123,11 +2133,11 @@ func bulkUpdateTableStatsList( catalog.MO_CATALOG, catalog.MO_TABLE_STATS, strings.Join(vals, ",")) - ret := executeSQL(ctx, sql, "bulk update table stats") + ret := d.executeSQL(ctx, sql, "bulk update table stats") return ret.Error() } -func bulkUpdateTableOnlyTS( +func (d *dynamicCtx) bulkUpdateTableOnlyTS( ctx context.Context, service string, tbls []tablePair, @@ -2160,7 +2170,7 @@ func bulkUpdateTableOnlyTS( catalog.MO_CATALOG, catalog.MO_TABLE_STATS, strings.Join(vals, ",")) - ret := executeSQL(ctx, sql, "bulk update only ts") + ret := d.executeSQL(ctx, sql, "bulk update only ts") for i := range tbls { tbls[i].Done(ret.Error()) diff --git a/pkg/vm/engine/disttae/stats.go b/pkg/vm/engine/disttae/stats.go index 6f3d84de5356a..bba9ec494928c 100644 --- a/pkg/vm/engine/disttae/stats.go +++ b/pkg/vm/engine/disttae/stats.go @@ -42,6 +42,10 @@ var ( // MinUpdateInterval is the minimal interval to update stats info as it // is necessary to update stats every time. MinUpdateInterval = time.Second * 15 + + initCheckInterval = time.Millisecond * 10 + maxCheckInterval = time.Second * 5 + checkTimeout = time.Minute ) // waitKeeper is used to mark the table has finished waited, @@ -137,6 +141,20 @@ func WithUpdateWorkerFactor(f int) GlobalStatsOption { } } +// WithStatsUpdater set the update function to update stats info. +func WithStatsUpdater(f func(pb.StatsInfoKey, *pb.StatsInfo) bool) GlobalStatsOption { + return func(s *GlobalStats) { + s.statsUpdater = f + } +} + +// WithApproxObjectNumUpdater set the update function to update approx object num. +func WithApproxObjectNumUpdater(f func() int64) GlobalStatsOption { + return func(s *GlobalStats) { + s.approxObjectNumUpdater = f + } +} + // updateRecord records the update status of a key. type updateRecord struct { // inProgress indicates if the stats of a table is being updated. @@ -193,6 +211,12 @@ type GlobalStats struct { KeyRouter client.KeyRouter[pb.StatsInfoKey] concurrentExecutor ConcurrentExecutor + + // statsUpdate is the function which updates the stats info. + // If it is nil, set it to doUpdate. + statsUpdater func(pb.StatsInfoKey, *pb.StatsInfo) bool + // for test only currently. + approxObjectNumUpdater func() int64 } func NewGlobalStats( @@ -214,6 +238,9 @@ func NewGlobalStats( for _, opt := range opts { opt(s) } + if s.statsUpdater == nil { + s.statsUpdater = s.doUpdate + } s.concurrentExecutor = newConcurrentExecutor(runtime.GOMAXPROCS(0) * s.updateWorkerFactor * 4) s.concurrentExecutor.Run(ctx) go s.consumeWorker(ctx) @@ -462,11 +489,11 @@ func (gs *GlobalStats) waitLogtailUpdated(tid uint64) { // 1. context done // 2. interval checking, whose init interval is 10ms and max interval is 5s // 3. logtail update notify, to check if it is the required table. - initCheckInterval := time.Millisecond * 10 - maxCheckInterval := time.Second * 5 checkInterval := initCheckInterval timer := time.NewTimer(checkInterval) defer timer.Stop() + timeout := time.NewTimer(checkTimeout) + defer timeout.Stop() var done bool for { @@ -480,6 +507,10 @@ func (gs *GlobalStats) waitLogtailUpdated(tid uint64) { case <-gs.ctx.Done(): return + case <-timeout.C: + logutil.Warnf("wait logtail updated timeout, table ID: %d", tid) + timeout.Reset(checkTimeout) + case <-timer.C: if checkUpdated() { return @@ -581,35 +612,38 @@ func (gs *GlobalStats) updateTableStats(key pb.StatsInfoKey) { var updated bool stats := plan2.NewStatsInfo() - defer func() { - gs.mu.Lock() - defer gs.mu.Unlock() - - if updated { - gs.mu.statsInfoMap[key] = stats - gs.broadcastStats(key) - } else if _, ok := gs.mu.statsInfoMap[key]; !ok { - gs.mu.statsInfoMap[key] = nil - } + if gs.statsUpdater != nil { + updated = gs.statsUpdater(key, stats) + } - // Notify all the waiters to read the new stats info. - gs.mu.cond.Broadcast() + gs.mu.Lock() + defer gs.mu.Unlock() + if updated { + gs.mu.statsInfoMap[key] = stats + gs.broadcastStats(key) + } else if _, ok := gs.mu.statsInfoMap[key]; !ok { + gs.mu.statsInfoMap[key] = nil + } - gs.doneUpdate(key, updated) - }() + // Notify all the waiters to read the new stats info. + gs.mu.cond.Broadcast() + + gs.doneUpdate(key, updated) +} +func (gs *GlobalStats) doUpdate(key pb.StatsInfoKey, stats *pb.StatsInfo) bool { table := gs.engine.GetLatestCatalogCache().GetTableById(key.AccId, key.DatabaseID, key.TableID) // table or its definition is nil, means that the table is created but not committed yet. if table == nil || table.TableDef == nil { logutil.Errorf("cannot get table by ID %v", key) - return + return false } partitionState := gs.engine.GetOrCreateLatestPart(key.DatabaseID, key.TableID).Snapshot() approxObjectNum := int64(partitionState.ApproxDataObjectsNum()) - if approxObjectNum == 0 { + if gs.approxObjectNumUpdater == nil && approxObjectNum == 0 { // There are no objects flushed yet. - return + return false } // the time used to init stats info is not need to be too precise. @@ -624,10 +658,10 @@ func (gs *GlobalStats) updateTableStats(key pb.StatsInfoKey) { ) if err := UpdateStats(gs.ctx, req, gs.concurrentExecutor); err != nil { logutil.Errorf("failed to init stats info for table %v, err: %v", key, err) - return + return false } v2.StatsUpdateBlockCounter.Add(float64(stats.BlockNumber)) - updated = true + return true } func getMinMaxValueByFloat64(typ types.Type, buf []byte) float64 { diff --git a/pkg/vm/engine/disttae/stats_test.go b/pkg/vm/engine/disttae/stats_test.go index d4a8224792172..48d89fad6b2cc 100644 --- a/pkg/vm/engine/disttae/stats_test.go +++ b/pkg/vm/engine/disttae/stats_test.go @@ -16,20 +16,234 @@ package disttae import ( "context" + "math/rand" "sync" "sync/atomic" "testing" "time" "github.com/lni/goutils/leaktest" + "github.com/matrixorigin/matrixone/pkg/catalog" + "github.com/matrixorigin/matrixone/pkg/clusterservice" + "github.com/matrixorigin/matrixone/pkg/common/mpool" + "github.com/matrixorigin/matrixone/pkg/common/runtime" + "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/lockservice" "github.com/matrixorigin/matrixone/pkg/pb/statsinfo" + plan2 "github.com/matrixorigin/matrixone/pkg/sql/plan" + "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae/cache" "github.com/stretchr/testify/assert" ) +func TestGetStats(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + gs := NewGlobalStats(ctx, nil, nil, + WithUpdateWorkerFactor(4), + WithStatsUpdater(func(key statsinfo.StatsInfoKey, info *statsinfo.StatsInfo) bool { + info.BlockNumber = 20 + return true + }), + ) + + tids := []uint64{2000, 2001, 2002} + go func() { + time.Sleep(time.Millisecond * 20) + for _, tid := range tids { + gs.notifyLogtailUpdate(tid) + } + }() + var wg sync.WaitGroup + wg.Add(100) + for i := 0; i < 100; i++ { + go func(j int) { + defer wg.Done() + rd := rand.New(rand.NewSource(time.Now().UnixNano())) + time.Sleep(time.Millisecond * time.Duration(10+rd.Intn(20))) + k := statsinfo.StatsInfoKey{ + DatabaseID: 1000, + TableID: tids[j%3], + } + info := gs.Get(ctx, k, true) + assert.NotNil(t, info) + assert.Equal(t, int64(20), info.BlockNumber) + }(i) + } + wg.Wait() +} + +func runTest( + t *testing.T, + test func(ctx context.Context, e *Engine), + opts ...GlobalStatsOption, +) { + defer leaktest.AfterTest(t)() + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sid := "s1" + rt := runtime.DefaultRuntime() + runtime.SetupServiceBasedRuntime(sid, rt) + cluster := clusterservice.NewMOCluster( + sid, + nil, + time.Hour, + clusterservice.WithDisableRefresh(), + ) + defer cluster.Close() + rt.SetGlobalVariables(runtime.ClusterService, cluster) + lk := lockservice.NewLockService(lockservice.Config{ + ServiceID: sid, + }) + defer lk.Close() + rt.SetGlobalVariables(runtime.LockService, lk) + mp, err := mpool.NewMPool(sid, 1024*1024, 0) + catalog.SetupDefines(sid) + assert.NoError(t, err) + e := New( + ctx, + sid, + mp, + nil, + nil, + nil, + nil, + 4, + ) + for _, opt := range opts { + opt(e.globalStats) + } + defer e.Close() + test(ctx, e) +} + +func insertTable( + t *testing.T, + e *Engine, + did, tid uint64, + dname, tname string, +) (uint64, uint64) { + tbl := catalog.Table{ + AccountId: 0, + UserId: 0, + RoleId: 0, + DatabaseId: did, + DatabaseName: dname, + TableId: tid, + TableName: tname, + } + packer := types.NewPacker() + bat, err := catalog.GenCreateTableTuple(tbl, e.mp, packer) + assert.NoError(t, err) + _, err = fillRandomRowidAndZeroTs(bat, e.mp) + assert.NoError(t, err) + e.catalog.InsertTable(bat) + tableItem := e.catalog.GetTableByName(0, did, tname) + assert.NotNil(t, tableItem) + defs, err := catalog.GenColumnsFromDefs( + 0, + tname, + dname, + tid, + did, + catalog.GetDefines(e.service).MoDatabaseTableDefs, + ) + assert.NoError(t, err) + cache.InitTableItemWithColumns(tableItem, defs) + return tableItem.DatabaseId, tableItem.Id +} + +func TestUpdateStats(t *testing.T) { + t.Run("no table", func(t *testing.T) { + runTest(t, func(ctx context.Context, e *Engine) { + k := statsinfo.StatsInfoKey{ + DatabaseID: 1000, + TableID: 1001, + } + stats := plan2.NewStatsInfo() + updated := e.globalStats.doUpdate(k, stats) + assert.False(t, updated) + }) + }) + + t.Run("no obj", func(t *testing.T) { + runTest(t, func(ctx context.Context, e *Engine) { + did := uint64(1000) + dname := "test-db" + tid := uint64(1001) + tname := "test-table" + did1, tid1 := insertTable(t, e, did, tid, dname, tname) + assert.Equal(t, did, did1) + assert.Equal(t, tid, tid1) + k := statsinfo.StatsInfoKey{ + DatabaseID: did, + TableID: tid, + } + stats := plan2.NewStatsInfo() + updated := e.globalStats.doUpdate(k, stats) + assert.False(t, updated) + }) + }) + + t.Run("objs", func(t *testing.T) { + runTest(t, func(ctx context.Context, e *Engine) { + did := uint64(1000) + dname := "test-db" + tid := uint64(1001) + tname := "test-table" + did1, tid1 := insertTable(t, e, did, tid, dname, tname) + assert.Equal(t, did, did1) + assert.Equal(t, tid, tid1) + k := statsinfo.StatsInfoKey{ + DatabaseID: did, + TableID: tid, + } + stats := plan2.NewStatsInfo() + updated := e.globalStats.doUpdate(k, stats) + assert.True(t, updated) + }, WithApproxObjectNumUpdater(func() int64 { + return 10 + })) + }) +} + +func TestWaitLogtailUpdate(t *testing.T) { + origInitCheckInterval := initCheckInterval + origMaxCheckInterval := maxCheckInterval + origCheckTimeout := checkTimeout + defer func() { + initCheckInterval = origInitCheckInterval + maxCheckInterval = origMaxCheckInterval + checkTimeout = origCheckTimeout + leaktest.AfterTest(t)() + }() + initCheckInterval = time.Millisecond * 2 + maxCheckInterval = time.Millisecond * 10 + checkTimeout = time.Millisecond * 200 + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + gs := NewGlobalStats(ctx, nil, nil) + assert.NotNil(t, gs) + tid := uint64(2) + gs.waitLogtailUpdated(tid) + + tid = 200 + go func() { + time.Sleep(time.Millisecond * 100) + gs.notifyLogtailUpdate(tid) + }() + gs.waitLogtailUpdated(tid) +} + func TestGlobalStats_ShouldUpdate(t *testing.T) { defer leaktest.AfterTest(t)() t.Run("basic", func(t *testing.T) { + origMinUpdateInterval := MinUpdateInterval + defer func() { + MinUpdateInterval = origMinUpdateInterval + }() MinUpdateInterval = time.Millisecond * 10 ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -47,6 +261,10 @@ func TestGlobalStats_ShouldUpdate(t *testing.T) { }) t.Run("parallel", func(t *testing.T) { + origMinUpdateInterval := MinUpdateInterval + defer func() { + MinUpdateInterval = origMinUpdateInterval + }() MinUpdateInterval = time.Second * 10 ctx, cancel := context.WithCancel(context.Background()) defer cancel() diff --git a/pkg/vm/engine/disttae/types.go b/pkg/vm/engine/disttae/types.go index c193a14b5d55a..c5bc5360f99b7 100644 --- a/pkg/vm/engine/disttae/types.go +++ b/pkg/vm/engine/disttae/types.go @@ -240,6 +240,8 @@ type Engine struct { moDatabaseCreatedTime *vector.Vector moTablesCreatedTime *vector.Vector moColumnsCreatedTime *vector.Vector + + dynamicCtx } func (e *Engine) SetService(svr string) { diff --git a/pkg/vm/engine/test/change_handle_test.go b/pkg/vm/engine/test/change_handle_test.go index 6742a8069060c..5da971e05f56e 100644 --- a/pkg/vm/engine/test/change_handle_test.go +++ b/pkg/vm/engine/test/change_handle_test.go @@ -53,9 +53,9 @@ func TestChangesHandle1(t *testing.T) { databaseName = "db1" ) - ctx := context.WithValue(context.Background(), defines.TenantIDKey{}, accountId) - ctx, cancel := context.WithTimeout(ctx, time.Minute*5) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() + ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) disttaeEngine, taeHandler, rpcAgent, _ := testutil.CreateEngines(ctx, testutil.TestOptions{}, t) defer func() { @@ -68,6 +68,8 @@ func TestChangesHandle1(t *testing.T) { schema.Name = tableName bat := catalog2.MockBatch(schema, 10) + ctx, cancel = context.WithTimeout(ctx, time.Minute*5) + defer cancel() _, _, err := disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) txn, rel := testutil2.GetRelation(t, accountId, taeHandler.GetDB(), databaseName, tableName) @@ -141,9 +143,9 @@ func TestChangesHandle2(t *testing.T) { databaseName = "db1" ) - ctx := context.WithValue(context.Background(), defines.TenantIDKey{}, accountId) - ctx, cancel := context.WithTimeout(ctx, time.Minute*5) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() + ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) disttaeEngine, taeHandler, rpcAgent, _ := testutil.CreateEngines(ctx, testutil.TestOptions{}, t) defer func() { @@ -157,6 +159,8 @@ func TestChangesHandle2(t *testing.T) { bat := catalog2.MockBatch(schema, 10) mp := common.DebugAllocator + ctx, cancel = context.WithTimeout(ctx, time.Minute*5) + defer cancel() _, _, err := disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) txn, rel := testutil2.GetRelation(t, accountId, taeHandler.GetDB(), databaseName, tableName) @@ -254,9 +258,9 @@ func TestChangesHandle3(t *testing.T) { databaseName = "db1" ) - ctx := context.WithValue(context.Background(), defines.TenantIDKey{}, accountId) - ctx, cancel := context.WithTimeout(ctx, time.Minute*5) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() + ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) opts := config.WithLongScanAndCKPOpts(nil) disttaeEngine, taeHandler, rpcAgent, _ := testutil.CreateEngines(ctx, testutil.TestOptions{TaeEngineOptions: opts}, t) @@ -271,6 +275,8 @@ func TestChangesHandle3(t *testing.T) { bat := catalog2.MockBatch(schema, 163840) mp := common.DebugAllocator + ctx, cancel = context.WithTimeout(ctx, time.Minute*5) + defer cancel() _, _, err := disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) txn, rel := testutil2.GetRelation(t, accountId, taeHandler.GetDB(), databaseName, tableName) @@ -361,9 +367,8 @@ func TestChangesHandleForCNWrite(t *testing.T) { disttaeEngine *testutil.TestDisttaeEngine ) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) schema := catalog2.MockSchemaAll(4, primaryKeyIdx) @@ -377,6 +382,8 @@ func TestChangesHandleForCNWrite(t *testing.T) { }() startTS := taeEngine.GetDB().TxnMgr.Now() + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() _, _, err = disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) @@ -466,9 +473,8 @@ func TestChangesHandle4(t *testing.T) { disttaeEngine *testutil.TestDisttaeEngine ) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) schema := catalog2.MockSchemaAll(4, primaryKeyIdx) @@ -482,6 +488,8 @@ func TestChangesHandle4(t *testing.T) { }() startTS := taeEngine.GetDB().TxnMgr.Now() + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() _, _, err = disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) @@ -598,9 +606,9 @@ func TestChangesHandle5(t *testing.T) { databaseName = "db1" ) - ctx := context.WithValue(context.Background(), defines.TenantIDKey{}, accountId) - ctx, cancel := context.WithTimeout(ctx, time.Minute*5) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() + ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) opts := config.WithLongScanAndCKPOpts(nil) disttaeEngine, taeHandler, rpcAgent, _ := testutil.CreateEngines(ctx, testutil.TestOptions{TaeEngineOptions: opts}, t) @@ -614,6 +622,8 @@ func TestChangesHandle5(t *testing.T) { schema.Name = tableName bat := catalog2.MockBatch(schema, 10) + ctx, cancel = context.WithTimeout(ctx, time.Minute*5) + defer cancel() _, _, err := disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) txn, rel := testutil2.GetRelation(t, accountId, taeHandler.GetDB(), databaseName, tableName) @@ -696,9 +706,9 @@ func TestChangesHandle6(t *testing.T) { databaseName = "db1" ) - ctx := context.WithValue(context.Background(), defines.TenantIDKey{}, accountId) - ctx, cancel := context.WithTimeout(ctx, time.Minute*5) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() + ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) opts := config.WithLongScanAndCKPOpts(nil) disttaeEngine, taeHandler, rpcAgent, _ := testutil.CreateEngines(ctx, testutil.TestOptions{TaeEngineOptions: opts}, t) @@ -712,6 +722,8 @@ func TestChangesHandle6(t *testing.T) { schema.Name = tableName bat := catalog2.MockBatch(schema, 10) + ctx, cancel = context.WithTimeout(ctx, time.Minute*5) + defer cancel() _, _, err := disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) txn, rel := testutil2.GetRelation(t, accountId, taeHandler.GetDB(), databaseName, tableName) @@ -796,9 +808,9 @@ func TestChangesHandleStaleFiles1(t *testing.T) { databaseName = "db1" ) - ctx := context.WithValue(context.Background(), defines.TenantIDKey{}, accountId) - ctx, cancel := context.WithTimeout(ctx, time.Minute*5) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() + ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) opts := config.WithLongScanAndCKPOpts(nil) disttaeEngine, taeHandler, rpcAgent, _ := testutil.CreateEngines(ctx, testutil.TestOptions{TaeEngineOptions: opts}, t) @@ -812,6 +824,8 @@ func TestChangesHandleStaleFiles1(t *testing.T) { schema.Name = tableName bat := catalog2.MockBatch(schema, 10) + ctx, cancel = context.WithTimeout(ctx, time.Minute*5) + defer cancel() _, _, err := disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) txn, rel := testutil2.GetRelation(t, accountId, taeHandler.GetDB(), databaseName, tableName) @@ -872,9 +886,8 @@ func TestChangesHandleStaleFiles2(t *testing.T) { disttaeEngine *testutil.TestDisttaeEngine ) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) schema := catalog2.MockSchemaAll(4, primaryKeyIdx) @@ -888,6 +901,8 @@ func TestChangesHandleStaleFiles2(t *testing.T) { }() startTS := taeEngine.GetDB().TxnMgr.Now() + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() _, _, err = disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) @@ -963,9 +978,9 @@ func TestChangesHandleStaleFiles5(t *testing.T) { databaseName = "db1" ) - ctx := context.WithValue(context.Background(), defines.TenantIDKey{}, accountId) - ctx, cancel := context.WithTimeout(ctx, time.Minute*5) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() + ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) disttaeEngine, taeHandler, rpcAgent, _ := testutil.CreateEngines(ctx, testutil.TestOptions{}, t) defer func() { @@ -979,6 +994,8 @@ func TestChangesHandleStaleFiles5(t *testing.T) { bat := catalog2.MockBatch(schema, 163840) mp := common.DebugAllocator + ctx, cancel = context.WithTimeout(ctx, time.Minute*5) + defer cancel() _, _, err := disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) txn, rel := testutil2.GetRelation(t, accountId, taeHandler.GetDB(), databaseName, tableName) diff --git a/pkg/vm/engine/test/disttae_engine_test.go b/pkg/vm/engine/test/disttae_engine_test.go index 88979dc9e5945..975beb89cd16f 100644 --- a/pkg/vm/engine/test/disttae_engine_test.go +++ b/pkg/vm/engine/test/disttae_engine_test.go @@ -62,9 +62,9 @@ func Test_InsertRows(t *testing.T) { databaseName = "db1" ) - ctx := context.WithValue(context.Background(), defines.TenantIDKey{}, accountId) - ctx, cancel := context.WithTimeout(ctx, time.Minute*5) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() + ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) disttaeEngine, taeHandler, rpcAgent, _ := testutil.CreateEngines(ctx, testutil.TestOptions{}, t) defer func() { @@ -73,6 +73,8 @@ func Test_InsertRows(t *testing.T) { rpcAgent.Close() }() + ctx, cancel = context.WithTimeout(ctx, time.Minute*5) + defer cancel() txn, err := disttaeEngine.NewTxnOperator(ctx, disttaeEngine.Now()) require.Nil(t, err) @@ -878,9 +880,9 @@ func TestShowDatabasesInRestoreTxn(t *testing.T) { func TestObjectStats1(t *testing.T) { catalog.SetupDefines("") - ctx := context.WithValue(context.Background(), defines.TenantIDKey{}, catalog.System_Account) - ctx, cancel := context.WithTimeout(ctx, time.Minute*5) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() + ctx = context.WithValue(ctx, defines.TenantIDKey{}, catalog.System_Account) disttaeEngine, taeHandler, rpcAgent, _ := testutil.CreateEngines(ctx, testutil.TestOptions{}, t) defer func() { @@ -893,6 +895,8 @@ func TestObjectStats1(t *testing.T) { testutil2.CreateRelationAndAppend(t, catalog.System_Account, taeHandler.GetDB(), "db", schema, bat, true) + ctx, cancel = context.WithTimeout(ctx, time.Minute*5) + defer cancel() txn, rel := testutil2.GetRelation(t, catalog.System_Account, taeHandler.GetDB(), "db", schema.Name) id := rel.GetMeta().(*catalog2.TableEntry).AsCommonID() appendableObjectID := testutil2.GetOneObject(rel).GetID() @@ -965,9 +969,9 @@ func TestObjectStats1(t *testing.T) { func TestObjectStats2(t *testing.T) { catalog.SetupDefines("") - ctx := context.WithValue(context.Background(), defines.TenantIDKey{}, catalog.System_Account) - ctx, cancel := context.WithTimeout(ctx, time.Minute*5) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() + ctx = context.WithValue(ctx, defines.TenantIDKey{}, catalog.System_Account) disttaeEngine, taeHandler, rpcAgent, _ := testutil.CreateEngines(ctx, testutil.TestOptions{}, t) defer func() { @@ -980,6 +984,8 @@ func TestObjectStats2(t *testing.T) { testutil2.CreateRelationAndAppend(t, catalog.System_Account, taeHandler.GetDB(), "db", schema, bat, true) + ctx, cancel = context.WithTimeout(ctx, time.Minute*5) + defer cancel() txn, rel := testutil2.GetRelation(t, catalog.System_Account, taeHandler.GetDB(), "db", schema.Name) id := rel.GetMeta().(*catalog2.TableEntry).AsCommonID() appendableObjectID := testutil2.GetOneObject(rel).GetID() diff --git a/pkg/vm/engine/test/mo_table_stats_test.go b/pkg/vm/engine/test/mo_table_stats_test.go index 98cdbd9538e88..62ac363af0efe 100644 --- a/pkg/vm/engine/test/mo_table_stats_test.go +++ b/pkg/vm/engine/test/mo_table_stats_test.go @@ -20,6 +20,7 @@ import ( "testing" "time" + "github.com/lni/goutils/leaktest" catalog2 "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/common/runtime" "github.com/matrixorigin/matrixone/pkg/container/types" @@ -37,6 +38,7 @@ import ( ) func TestMoTableStatsMoCtl(t *testing.T) { + defer leaktest.AfterTest(t)() var ( opts testutil.TestOptions tableName = "test1" @@ -59,13 +61,14 @@ func TestMoTableStatsMoCtl(t *testing.T) { exec := v.(executor.SQLExecutor) + e := p.D.Engine { - ret := disttae.HandleMoTableStatsCtl("restore_default_setting:true") + ret := e.HandleMoTableStatsCtl("restore_default_setting:true") require.Equal(t, "move_on(true), use_old_impl(false), force_update(false)", ret) } { - ret := disttae.HandleMoTableStatsCtl("move_on: false") + ret := e.HandleMoTableStatsCtl("move_on: false") require.Equal(t, "move on: true to false", ret) } @@ -92,7 +95,7 @@ func TestMoTableStatsMoCtl(t *testing.T) { } { - ret := disttae.HandleMoTableStatsCtl("force_update:true") + ret := e.HandleMoTableStatsCtl("force_update:true") require.Equal(t, "force update: false to true", ret) } @@ -109,7 +112,7 @@ func TestMoTableStatsMoCtl(t *testing.T) { } { - ret := disttae.HandleMoTableStatsCtl("use_old_impl:true") + ret := e.HandleMoTableStatsCtl("use_old_impl:true") require.Equal(t, "use old impl: false to true", ret) } @@ -125,13 +128,15 @@ func TestMoTableStatsMoCtl(t *testing.T) { } func TestMoTableStatsMoCtl2(t *testing.T) { + defer leaktest.AfterTest(t)() var opts testutil.TestOptions opts.TaeEngineOptions = config.WithLongScanAndCKPOpts(nil) p := testutil.InitEnginePack(opts, t) defer p.Close() - disttae.NotifyCleanDeletes() - disttae.NotifyUpdateForgotten() + e := p.D.Engine + e.NotifyCleanDeletes() + e.NotifyUpdateForgotten() schema := catalog.MockSchemaAll(3, 2) schema.Name = "test1" @@ -140,68 +145,69 @@ func TestMoTableStatsMoCtl2(t *testing.T) { _, rel := p.CreateDBAndTable(txnop, "db1", schema) require.NoError(t, txnop.Commit(p.Ctx)) - ret := disttae.HandleMoTableStatsCtl("restore_default_setting:true") + ret := e.HandleMoTableStatsCtl("restore_default_setting:true") require.Equal(t, "move_on(true), use_old_impl(false), force_update(false)", ret) dbId := rel.GetDBID(p.Ctx) tblId := rel.GetTableID(p.Ctx) - _, err := disttae.QueryTableStats(context.Background(), + _, err := e.QueryTableStats(context.Background(), []int{disttae.TableStatsTableRows}, []uint64{0}, []uint64{dbId}, []uint64{tblId}, false, false, nil) require.NotNil(t, err) - ret = disttae.HandleMoTableStatsCtl("use_old_impl:true") + ret = e.HandleMoTableStatsCtl("use_old_impl:true") require.Equal(t, "use old impl: false to true", ret) - _, err = disttae.QueryTableStats(context.Background(), + _, err = e.QueryTableStats(context.Background(), []int{disttae.TableStatsTableRows}, []uint64{0}, []uint64{dbId}, []uint64{tblId}, false, false, nil) require.NoError(t, err) - ret = disttae.HandleMoTableStatsCtl("use_old_impl:false") + ret = e.HandleMoTableStatsCtl("use_old_impl:false") require.Equal(t, "use old impl: true to false", ret) - ret = disttae.HandleMoTableStatsCtl("force_update:true") + ret = e.HandleMoTableStatsCtl("force_update:true") require.Equal(t, "force update: false to true", ret) - _, err = disttae.QueryTableStats(context.Background(), + _, err = e.QueryTableStats(context.Background(), []int{disttae.TableStatsTableRows}, []uint64{0}, []uint64{dbId}, []uint64{tblId}, true, false, nil) require.NotNil(t, err) - ret = disttae.HandleMoTableStatsCtl("move_on: false") + ret = e.HandleMoTableStatsCtl("move_on: false") require.Equal(t, "move on: true to false", ret) - _, err = disttae.QueryTableStats(context.Background(), + _, err = e.QueryTableStats(context.Background(), []int{disttae.TableStatsTableRows}, []uint64{0}, []uint64{dbId}, []uint64{tblId}, false, true, nil) require.NotNil(t, err) - ret = disttae.HandleMoTableStatsCtl("echo_current_setting:false") + ret = e.HandleMoTableStatsCtl("echo_current_setting:false") require.Equal(t, "noop", ret) - ret = disttae.HandleMoTableStatsCtl("echo_current_setting:true") + ret = e.HandleMoTableStatsCtl("echo_current_setting:true") require.Equal(t, "move_on(false), use_old_impl(false), force_update(true)", ret) { - ret = disttae.HandleMoTableStatsCtl("no_such_cmd:true") + ret = e.HandleMoTableStatsCtl("no_such_cmd:true") require.Equal(t, "failed, cmd invalid", ret) - ret = disttae.HandleMoTableStatsCtl("force_update:yes") + ret = e.HandleMoTableStatsCtl("force_update:yes") require.Equal(t, "failed, cmd invalid", ret) - ret = disttae.HandleMoTableStatsCtl("force_update:true:false") + ret = e.HandleMoTableStatsCtl("force_update:true:false") require.Equal(t, "invalid command", ret) } } func TestHandleGetChangedList(t *testing.T) { + defer leaktest.AfterTest(t)() var opts testutil.TestOptions opts.TaeEngineOptions = config.WithLongScanAndCKPOpts(nil) diff --git a/pkg/vm/engine/test/reader_test.go b/pkg/vm/engine/test/reader_test.go index 90238e919a06b..34f1f757f4417 100644 --- a/pkg/vm/engine/test/reader_test.go +++ b/pkg/vm/engine/test/reader_test.go @@ -72,9 +72,8 @@ func Test_ReaderCanReadRangesBlocksWithoutDeletes(t *testing.T) { disttaeEngine *testutil.TestDisttaeEngine ) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) schema := catalog2.MockSchemaAll(4, primaryKeyIdx) @@ -97,6 +96,8 @@ func Test_ReaderCanReadRangesBlocksWithoutDeletes(t *testing.T) { rpcAgent.Close() }() + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() _, _, err = disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) @@ -190,9 +191,8 @@ func TestReaderCanReadUncommittedInMemInsertAndDeletes(t *testing.T) { disttaeEngine *testutil.TestDisttaeEngine ) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) schema := catalog2.MockSchemaAll(4, primaryKeyIdx) @@ -205,6 +205,8 @@ func TestReaderCanReadUncommittedInMemInsertAndDeletes(t *testing.T) { rpcAgent.Close() }() + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() _, _, err = disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) @@ -272,9 +274,8 @@ func Test_ReaderCanReadCommittedInMemInsertAndDeletes(t *testing.T) { disttaeEngine *testutil.TestDisttaeEngine ) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) fault.Enable() @@ -305,6 +306,8 @@ func Test_ReaderCanReadCommittedInMemInsertAndDeletes(t *testing.T) { rpcAgent.Close() }() + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() _, _, err = disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) @@ -434,9 +437,8 @@ func Test_ShardingHandler(t *testing.T) { require.NoError(t, err) defer rmFault() - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) // mock a schema with 4 columns and the 4th column as primary key @@ -461,6 +463,8 @@ func Test_ShardingHandler(t *testing.T) { rpcAgent.Close() }() + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() _, _, err = disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) @@ -631,9 +635,8 @@ func Test_ShardingRemoteReader(t *testing.T) { require.NoError(t, err) defer rmFault() - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) // mock a schema with 4 columns and the 4th column as primary key @@ -657,6 +660,8 @@ func Test_ShardingRemoteReader(t *testing.T) { rpcAgent.Close() }() + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() _, _, err = disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) @@ -932,9 +937,8 @@ func Test_ShardingTableDelegate(t *testing.T) { disttaeEngine *testutil.TestDisttaeEngine ) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) // mock a schema with 4 columns and the 4th column as primary key @@ -958,6 +962,8 @@ func Test_ShardingTableDelegate(t *testing.T) { rpcAgent.Close() }() + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() _, _, err = disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) @@ -1100,9 +1106,8 @@ func Test_ShardingLocalReader(t *testing.T) { disttaeEngine *testutil.TestDisttaeEngine ) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) fault.Enable() @@ -1139,6 +1144,8 @@ func Test_ShardingLocalReader(t *testing.T) { rpcAgent.Close() }() + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() _, _, err = disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) diff --git a/pkg/vm/engine/test/testutil/disttae_engine.go b/pkg/vm/engine/test/testutil/disttae_engine.go index d2bbe3725ac3e..70dc74aeb7233 100644 --- a/pkg/vm/engine/test/testutil/disttae_engine.go +++ b/pkg/vm/engine/test/testutil/disttae_engine.go @@ -64,6 +64,7 @@ type TestDisttaeEngine struct { ctx context.Context cancel context.CancelFunc txnClient client.TxnClient + queryClient qclient.QueryClient txnOperator client.TxnOperator timestampWaiter client.TimestampWaiter mp *mpool.MPool @@ -136,7 +137,7 @@ func NewTestDisttaeEngine( engineOpts = append(engineOpts, disttae.WithSQLExecFunc(internalExecutorFactory)) catalog.SetupDefines("") - de.Engine = disttae.New(ctx, + de.Engine = disttae.New(de.ctx, "", de.mp, fs, @@ -160,7 +161,7 @@ func NewTestDisttaeEngine( } }() - op, err := de.txnClient.New(ctx, types.TS{}.ToTimestamp()) + op, err := de.txnClient.New(de.ctx, types.TS{}.ToTimestamp()) if err != nil { return nil, err } @@ -168,11 +169,12 @@ func NewTestDisttaeEngine( close(wait) de.txnOperator = op - if err = de.Engine.New(ctx, op); err != nil { + if err = de.Engine.New(de.ctx, op); err != nil { return nil, err } qc, _ := qclient.NewQueryClient("", morpc.Config{}) + de.queryClient = qc sqlExecutor := compile.NewSQLExecutor( "127.0.0.1:2000", de.Engine, @@ -200,7 +202,7 @@ func NewTestDisttaeEngine( setServerLevelParams(de) // InitLoTailPushModel presupposes that the internal sql executor has been initialized. - err = de.Engine.InitLogTailPushModel(ctx, de.timestampWaiter) + err = de.Engine.InitLogTailPushModel(de.ctx, de.timestampWaiter) //err = de.prevSubscribeSysTables(ctx, rpcAgent) return de, err } @@ -433,6 +435,8 @@ func (de *TestDisttaeEngine) Close(ctx context.Context) { close(de.logtailReceiver) de.cancel() de.wg.Wait() + de.Engine.Close() + de.queryClient.Close() if err := os.RemoveAll(de.rootDir); err != nil { logutil.Errorf("remove root dir failed (%s): %v", de.rootDir, err) diff --git a/pkg/vm/engine/test/testutil/logtailserver.go b/pkg/vm/engine/test/testutil/logtailserver.go index 2c8df36bab105..c01cb4c14d83b 100644 --- a/pkg/vm/engine/test/testutil/logtailserver.go +++ b/pkg/vm/engine/test/testutil/logtailserver.go @@ -36,7 +36,12 @@ func (cs *testClientSession) SessionCtx() context.Context { func (cs *testClientSession) Close() error { return nil } func (cs *testClientSession) Write(ctx context.Context, response morpc.Message) error { - cs.tailReceiveQueue <- response + select { + case <-ctx.Done(): + return ctx.Err() + + case cs.tailReceiveQueue <- response: + } return nil } diff --git a/pkg/vm/engine/test/testutil/tae_engine.go b/pkg/vm/engine/test/testutil/tae_engine.go index 4ed94e7be4ca1..b2e8ae75954c7 100644 --- a/pkg/vm/engine/test/testutil/tae_engine.go +++ b/pkg/vm/engine/test/testutil/tae_engine.go @@ -61,8 +61,18 @@ func (ts *TestTxnStorage) Shard() metadata.TNShard { func (ts *TestTxnStorage) Start() error { return nil } func (ts *TestTxnStorage) Close(destroy bool) error { - err := ts.GetDB().Close() - return err + var firstErr error + if err := ts.GetDB().Close(); err != nil { + firstErr = err + } + if err := ts.logtailServer.Close(); err != nil { + if firstErr == nil { + firstErr = err + } + } + ts.txnHandler.GCJob.Stop() + blockio.Stop("") + return firstErr } func (ts *TestTxnStorage) Read(ctx context.Context, request *txn.TxnRequest, response *txn.TxnResponse) error { return nil diff --git a/pkg/vm/engine/test/testutil/util.go b/pkg/vm/engine/test/testutil/util.go index 096d320d80f6f..fe05832472290 100644 --- a/pkg/vm/engine/test/testutil/util.go +++ b/pkg/vm/engine/test/testutil/util.go @@ -94,6 +94,11 @@ func CreateEngines( panic("cannot find account id in ctx") } + _, ok := ctx.Deadline() + if ok { + panic("context should not have deadline") + } + var err error rpcAgent = NewMockLogtailAgent() @@ -252,26 +257,27 @@ type EnginePack struct { } func InitEnginePack(opts TestOptions, t *testing.T) *EnginePack { - ctx := context.WithValue(context.Background(), defines.TenantIDKey{}, uint32(0)) - timeout := opts.Timeout - if timeout == 0 { - timeout = 5 * time.Minute - } - ctx, cancel := context.WithTimeoutCause(ctx, timeout, moerr.CauseInitEnginePack) + ctx, cancel := context.WithCancel(context.Background()) + ctx = context.WithValue(ctx, defines.TenantIDKey{}, uint32(0)) pack := &EnginePack{ - Ctx: ctx, t: t, cancelF: cancel, } - pack.D, pack.T, pack.R, pack.Mp = CreateEngines(pack.Ctx, opts, t) + pack.D, pack.T, pack.R, pack.Mp = CreateEngines(ctx, opts, t) + timeout := opts.Timeout + if timeout == 0 { + timeout = 5 * time.Minute + } + ctx, _ = context.WithTimeoutCause(ctx, timeout, moerr.CauseInitEnginePack) + pack.Ctx = ctx return pack } func (p *EnginePack) Close() { - p.cancelF() p.D.Close(p.Ctx) p.T.Close(true) p.R.Close() + p.cancelF() } func (p *EnginePack) StartCNTxn(opts ...client.TxnOption) client.TxnOperator { diff --git a/pkg/vm/engine/test/workspace_test.go b/pkg/vm/engine/test/workspace_test.go index 0b4a6b61071fb..7d661afdb953c 100644 --- a/pkg/vm/engine/test/workspace_test.go +++ b/pkg/vm/engine/test/workspace_test.go @@ -66,9 +66,8 @@ func Test_BasicInsertDelete(t *testing.T) { disttaeEngine *testutil.TestDisttaeEngine ) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) schema := catalog2.MockSchemaAll(4, primaryKeyIdx) @@ -85,6 +84,8 @@ func Test_BasicInsertDelete(t *testing.T) { rpcAgent.Close() }() + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() _, _, err = disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) @@ -153,9 +154,8 @@ func Test_BasicS3InsertDelete(t *testing.T) { disttaeEngine *testutil.TestDisttaeEngine ) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) schema := catalog2.MockSchemaAll(4, primaryKeyIdx) @@ -174,6 +174,8 @@ func Test_BasicS3InsertDelete(t *testing.T) { rpcAgent.Close() }() + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() _, _, err = disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) @@ -288,9 +290,8 @@ func Test_MultiTxnInsertDelete(t *testing.T) { disttaeEngine *testutil.TestDisttaeEngine ) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) schema := catalog2.MockSchemaAll(4, primaryKeyIdx) @@ -307,6 +308,8 @@ func Test_MultiTxnInsertDelete(t *testing.T) { rpcAgent.Close() }() + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() _, _, err = disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) @@ -443,9 +446,8 @@ func Test_MultiTxnS3InsertDelete(t *testing.T) { disttaeEngine *testutil.TestDisttaeEngine ) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) schema := catalog2.MockSchemaAll(4, primaryKeyIdx) @@ -464,6 +466,8 @@ func Test_MultiTxnS3InsertDelete(t *testing.T) { rpcAgent.Close() }() + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() _, _, err = disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) @@ -597,9 +601,8 @@ func Test_MultiTxnS3Tombstones(t *testing.T) { disttaeEngine *testutil.TestDisttaeEngine ) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) schema := catalog2.MockSchemaEnhanced(2, primaryKeyIdx, 2) @@ -618,6 +621,8 @@ func Test_MultiTxnS3Tombstones(t *testing.T) { rpcAgent.Close() }() + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() _, _, err = disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) @@ -770,9 +775,8 @@ func Test_BasicRollbackStatement(t *testing.T) { disttaeEngine *testutil.TestDisttaeEngine ) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) schema := catalog2.MockSchemaAll(4, primaryKeyIdx) @@ -789,6 +793,8 @@ func Test_BasicRollbackStatement(t *testing.T) { rpcAgent.Close() }() + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() _, _, err = disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) @@ -879,9 +885,8 @@ func Test_BasicRollbackStatementS3(t *testing.T) { disttaeEngine *testutil.TestDisttaeEngine ) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) schema := catalog2.MockSchemaAll(4, primaryKeyIdx) @@ -900,6 +905,8 @@ func Test_BasicRollbackStatementS3(t *testing.T) { rpcAgent.Close() }() + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() _, _, err = disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) @@ -1028,9 +1035,8 @@ func Test_MultiTxnRollbackStatement(t *testing.T) { disttaeEngine *testutil.TestDisttaeEngine ) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) schema := catalog2.MockSchemaAll(4, primaryKeyIdx) @@ -1047,6 +1053,8 @@ func Test_MultiTxnRollbackStatement(t *testing.T) { rpcAgent.Close() }() + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() _, _, err = disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) @@ -1198,9 +1206,8 @@ func Test_MultiTxnRollbackStatementS3(t *testing.T) { disttaeEngine *testutil.TestDisttaeEngine ) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) schema := catalog2.MockSchemaAll(4, primaryKeyIdx) @@ -1219,6 +1226,8 @@ func Test_MultiTxnRollbackStatementS3(t *testing.T) { rpcAgent.Close() }() + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() _, _, err = disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) @@ -1360,9 +1369,8 @@ func Test_DeleteUncommittedBlock(t *testing.T) { disttaeEngine *testutil.TestDisttaeEngine ) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) _ = colexec.NewServer(nil) @@ -1389,6 +1397,8 @@ func Test_DeleteUncommittedBlock(t *testing.T) { rpcAgent.Close() }() + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() _, _, err = disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) } @@ -1474,9 +1484,8 @@ func Test_BigDeleteWriteS3(t *testing.T) { disttaeEngine *testutil.TestDisttaeEngine ) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, accountId) // mock a schema with 4 columns and the 4th column as primary key @@ -1500,6 +1509,8 @@ func Test_BigDeleteWriteS3(t *testing.T) { rpcAgent.Close() }() + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() _, _, err = disttaeEngine.CreateDatabaseAndTable(ctx, databaseName, tableName, schema) require.NoError(t, err) } @@ -1549,9 +1560,8 @@ func Test_CNTransferTombstoneObjects(t *testing.T) { databaseName = "db1" ) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ctx = context.WithValue(ctx, defines.TenantIDKey{}, uint32(0)) opts.TaeEngineOptions = config.WithLongScanAndCKPOpts(nil) @@ -1562,6 +1572,8 @@ func Test_CNTransferTombstoneObjects(t *testing.T) { schema := catalog2.MockSchemaEnhanced(1, 0, 2) schema.Name = tableName + ctx, cancel = context.WithTimeout(ctx, time.Minute) + defer cancel() cnTxnOp := p.StartCNTxn() _, rel := p.CreateDBAndTable(cnTxnOp, databaseName, schema) require.NotNil(t, rel) From 745d4b2a2150fbca77da65c187a54a52ebf1a2a0 Mon Sep 17 00:00:00 2001 From: Kai Cao Date: Tue, 10 Dec 2024 23:05:37 +0800 Subject: [PATCH 10/27] [Cherry-pick]fix show table status (#20684) - use force_update policy in `show table status` - use `values` stmt instead of `union all` to improve performance Approved by: @daviszhen, @aressu1985 --- pkg/frontend/mysql_cmd_executor.go | 20 +++++++++---------- test/distributed/cases/dml/show/show4.sql | 1 + .../publication_subscription/pub_sub.sql | 2 +- .../publication_subscription/pub_sub4.sql | 1 + .../pub_sub_improvement.sql | 6 +++--- test/distributed/cases/tenant/pub_sub2.sql | 4 ++-- test/distributed/cases/tenant/pub_sub3.sql | 12 +++++------ 7 files changed, 24 insertions(+), 22 deletions(-) diff --git a/pkg/frontend/mysql_cmd_executor.go b/pkg/frontend/mysql_cmd_executor.go index 4882e51a8e805..43b39401e6ee9 100644 --- a/pkg/frontend/mysql_cmd_executor.go +++ b/pkg/frontend/mysql_cmd_executor.go @@ -34,9 +34,6 @@ import ( "github.com/confluentinc/confluent-kafka-go/v2/kafka" "github.com/google/uuid" - "go.uber.org/zap" - "go.uber.org/zap/zapcore" - "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/clusterservice" "github.com/matrixorigin/matrixone/pkg/common/moerr" @@ -74,6 +71,8 @@ import ( "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae" "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae/route" "github.com/matrixorigin/matrixone/pkg/vm/process" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" ) func createDropDatabaseErrorInfo() string { @@ -392,23 +391,24 @@ func handleShowTableStatus(ses *Session, execCtx *ExecCtx, stmt *tree.ShowTableS } // set session variable - if err = ses.SetSessionSysVar(ctx, "mo_table_stats.reset_update_time", "yes"); err != nil { + if err = ses.SetSessionSysVar(ctx, "mo_table_stats.force_update", "yes"); err != nil { return } defer func() { - _ = ses.SetSessionSysVar(ctx, "mo_table_stats.reset_update_time", "no") + _ = ses.SetSessionSysVar(ctx, "mo_table_stats.force_update", "no") }() sqlBuilder := strings.Builder{} - sqlBuilder.WriteString("select tbl, mo_table_rows(db, tbl), mo_table_size(db, tbl) from (") + sqlBuilder.WriteString("select tbl, mo_table_rows(db, tbl), mo_table_size(db, tbl) from (values ") for i, tblName := range tblNames { - if i > 0 { - sqlBuilder.WriteString(" union all ") + if i != 0 { + sqlBuilder.WriteString(", ") } - sqlBuilder.WriteString(fmt.Sprintf("select '%s' as db, '%s' as tbl", dbName, tblName)) + sqlBuilder.WriteString(fmt.Sprintf("row('%s', '%s')", dbName, tblName)) } - sqlBuilder.WriteString(") tmp") + sqlBuilder.WriteString(") as tmp(db, tbl)") + // get table stats var rets []ExecResult if rets, err = executeSQLInBackgroundSession(ctx, bh, sqlBuilder.String()); err != nil { return diff --git a/test/distributed/cases/dml/show/show4.sql b/test/distributed/cases/dml/show/show4.sql index 6d8907ba76306..db7ba158a85c9 100644 --- a/test/distributed/cases/dml/show/show4.sql +++ b/test/distributed/cases/dml/show/show4.sql @@ -38,6 +38,7 @@ show sequences from db; show table_number from db; show tables from db; +-- @ignore:3,5,10,11,12 show table status from db; show table_values from t1 from db; diff --git a/test/distributed/cases/publication_subscription/pub_sub.sql b/test/distributed/cases/publication_subscription/pub_sub.sql index 872cf5785c4b9..ded0aefaa260f 100644 --- a/test/distributed/cases/publication_subscription/pub_sub.sql +++ b/test/distributed/cases/publication_subscription/pub_sub.sql @@ -31,7 +31,7 @@ show tables; select * from table01; show columns from table01; desc table01; --- @ignore:10,11,12 +-- @ignore:3,5,10,11,12 show table status; -- @session diff --git a/test/distributed/cases/publication_subscription/pub_sub4.sql b/test/distributed/cases/publication_subscription/pub_sub4.sql index 49d2dac919a3b..4959b7d4a06d6 100644 --- a/test/distributed/cases/publication_subscription/pub_sub4.sql +++ b/test/distributed/cases/publication_subscription/pub_sub4.sql @@ -7,6 +7,7 @@ create publication pub_all database db1 account all; -- @session:id=1&user=acc1:admin1&password=111 create database syssub1 from sys publication pub_all; +-- @ignore:3,5,10,11,12 show table status from syssub1; -- @session diff --git a/test/distributed/cases/publication_subscription/pub_sub_improvement.sql b/test/distributed/cases/publication_subscription/pub_sub_improvement.sql index 7282ec3af0e70..eac3020ee5759 100644 --- a/test/distributed/cases/publication_subscription/pub_sub_improvement.sql +++ b/test/distributed/cases/publication_subscription/pub_sub_improvement.sql @@ -30,7 +30,7 @@ select * from table01; truncate table table01; delete from table01 where col1 = 1; update table01 set col1 = 100 where col2 = 2413242; --- @ignore:10,11,12 +-- @ignore:3,5,10,11,12 show table status; drop table table01; -- @ignore:5,7 @@ -115,7 +115,7 @@ show subscriptions all; use sub_database02; show tables; select * from table03; --- @ignore:10,11,12 +-- @ignore:3,5,10,11,12 show table status; -- @session drop publication publication02; @@ -139,7 +139,7 @@ use sub_database03; show tables; select * from table01; desc table01; --- @ignore:10,11,12 +-- @ignore:3,5,10,11,12 show table status; -- @session diff --git a/test/distributed/cases/tenant/pub_sub2.sql b/test/distributed/cases/tenant/pub_sub2.sql index 1020c5ec9fd23..eedf5c1374797 100644 --- a/test/distributed/cases/tenant/pub_sub2.sql +++ b/test/distributed/cases/tenant/pub_sub2.sql @@ -44,9 +44,9 @@ use syssub1; show tables; -- @ignore:5,7 show subscriptions all; --- @ignore:10 +-- @ignore:3,5,10,11,12 show table status; --- @ignore:10 +-- @ignore:3,5,10,11,12 show table status like 't1'; desc t1; show create table t1; diff --git a/test/distributed/cases/tenant/pub_sub3.sql b/test/distributed/cases/tenant/pub_sub3.sql index e25c5e6ea8f54..1979212158664 100644 --- a/test/distributed/cases/tenant/pub_sub3.sql +++ b/test/distributed/cases/tenant/pub_sub3.sql @@ -48,9 +48,9 @@ use syssub1; show tables; -- @ignore:5,7 show subscriptions all; --- @ignore:10 +-- @ignore:3,5,10,11,12 show table status; --- @ignore:10 +-- @ignore:3,5,10,11,12 show table status like 't1'; desc t1; show create table t1; @@ -100,9 +100,9 @@ use syssub1; show tables; -- @ignore:5,7 show subscriptions all; --- @ignore:10 +-- @ignore:3,5,10,11,12 show table status; --- @ignore:10 +-- @ignore:3,5,10,11,12 show table status like 't1'; desc t1; show create table t1; @@ -174,7 +174,7 @@ create database sys_sub_part_tbls from sys publication pub_part_tbls; show subscriptions; show full tables from sys_sub_part_tbls; --- @ignore:10 +-- @ignore:3,5,10,11,12 show table status from sys_sub_part_tbls; desc sys_sub_part_tbls.t1; show create table sys_sub_part_tbls.t1; @@ -185,7 +185,7 @@ select * from sys_sub_part_tbls.t2; use sys_sub_part_tbls; show full tables; --- @ignore:10 +-- @ignore:3,5,10,11,12 show table status; desc t1; show create table t1; From 2dca2e3b2e8312c8655bce43296cf372b5364615 Mon Sep 17 00:00:00 2001 From: nitao Date: Wed, 11 Dec 2024 00:05:30 +0800 Subject: [PATCH 11/27] [Tech Request]: optimize filters on composite columns (#20695) optimize filters on composite columns Approved by: @heni02, @ouyuanning, @aunjgr --- pkg/sql/plan/explain/explain_expr.go | 33 ++- pkg/sql/plan/expr_opt.go | 19 +- .../cases/optimizer/blockfilter.result | 195 ++++++++++++++++++ .../cases/optimizer/blockfilter.test | 63 ++++++ 4 files changed, 297 insertions(+), 13 deletions(-) create mode 100644 test/distributed/cases/optimizer/blockfilter.result create mode 100644 test/distributed/cases/optimizer/blockfilter.test diff --git a/pkg/sql/plan/explain/explain_expr.go b/pkg/sql/plan/explain/explain_expr.go index 0cf0a78825164..270759179bd9a 100644 --- a/pkg/sql/plan/explain/explain_expr.go +++ b/pkg/sql/plan/explain/explain_expr.go @@ -198,6 +198,21 @@ func describeExpr(ctx context.Context, expr *plan.Expr, options *ExplainOptions, return nil } +func needSpecialHandling(funcExpr *plan.Function) bool { + if funcExpr.Func.GetObjName() == "prefix_in" || funcExpr.Func.GetObjName() == "prefix_eq" || funcExpr.Func.GetObjName() == "prefix_between" { + return true + } + if len(funcExpr.Args) > 1 { + col := funcExpr.Args[0].GetCol() + if col != nil && funcExpr.Args[1].GetCol() == nil { + if strings.Contains(col.Name, catalog.PrefixCBColName) || strings.Contains(col.Name, catalog.PrefixPriColName) { + return true + } + } + } + return false +} + // generator function expression(Expr_F) explain information func funcExprExplain(ctx context.Context, funcExpr *plan.Function, Typ *plan.Type, options *ExplainOptions, buf *bytes.Buffer) error { // SysFunsAndOperatorsMap @@ -212,7 +227,7 @@ func funcExprExplain(ctx context.Context, funcExpr *plan.Function, Typ *plan.Typ switch layout { case function.STANDARD_FUNCTION: buf.WriteString(funcExpr.Func.GetObjName() + "(") - if funcExpr.Func.GetObjName() == "prefix_in" || funcExpr.Func.GetObjName() == "prefix_eq" || funcExpr.Func.GetObjName() == "prefix_between" { + if needSpecialHandling(funcExpr) { //contains invisible character, need special handling err = describeExpr(ctx, funcExpr.Args[0], options, buf) if err != nil { @@ -264,9 +279,11 @@ func funcExprExplain(ctx context.Context, funcExpr *plan.Function, Typ *plan.Typ return err } buf.WriteString(" " + funcExpr.Func.GetObjName() + " ") - err = describeExpr(ctx, funcExpr.Args[1], options, buf) - if err != nil { - return err + if !needSpecialHandling(funcExpr) { + err = describeExpr(ctx, funcExpr.Args[1], options, buf) + if err != nil { + return err + } } buf.WriteString(")") case function.MULTIARY_LOGICAL_OPERATOR: @@ -351,9 +368,11 @@ func funcExprExplain(ctx context.Context, funcExpr *plan.Function, Typ *plan.Typ return err } buf.WriteString(" " + funcExpr.Func.GetObjName() + " (") - err = describeExpr(ctx, funcExpr.Args[1], options, buf) - if err != nil { - return err + if !needSpecialHandling(funcExpr) { + err = describeExpr(ctx, funcExpr.Args[1], options, buf) + if err != nil { + return err + } } buf.WriteString(")") case function.EXISTS_ANY_PREDICATE: diff --git a/pkg/sql/plan/expr_opt.go b/pkg/sql/plan/expr_opt.go index 1869db9de0907..34557ec7d2115 100644 --- a/pkg/sql/plan/expr_opt.go +++ b/pkg/sql/plan/expr_opt.go @@ -18,6 +18,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/sql/plan/function" + "github.com/matrixorigin/matrixone/pkg/sql/util" ) func (builder *QueryBuilder) mergeFiltersOnCompositeKey(nodeID int32) { @@ -41,9 +42,15 @@ func (builder *QueryBuilder) mergeFiltersOnCompositeKey(nodeID int32) { } func (builder *QueryBuilder) doMergeFiltersOnCompositeKey(tableDef *plan.TableDef, tableTag int32, filters ...*plan.Expr) []*plan.Expr { - pkIdx := tableDef.Name2ColIndex[tableDef.Pkey.PkeyColName] + sortkeyIdx := tableDef.Name2ColIndex[tableDef.Pkey.PkeyColName] col2filter := make(map[int32]int) - numParts := len(tableDef.Pkey.Names) + Parts := tableDef.Pkey.Names + numParts := len(Parts) + if tableDef.ClusterBy != nil && util.JudgeIsCompositeClusterByColumn(tableDef.ClusterBy.Name) { + sortkeyIdx = tableDef.Name2ColIndex[tableDef.ClusterBy.Name] + Parts = util.SplitCompositeClusterByColumnName(tableDef.ClusterBy.Name) + numParts = len(Parts) + } for i, expr := range filters { fn := expr.GetF() @@ -179,7 +186,7 @@ func (builder *QueryBuilder) doMergeFiltersOnCompositeKey(tableDef *plan.TableDe if len(newOrArgs) == 1 { filters[i] = newOrArgs[0] colPos := firstEquiExpr.GetF().Args[0].GetCol().ColPos - if colPos != pkIdx { + if colPos != sortkeyIdx { col2filter[colPos] = i } } else { @@ -193,7 +200,7 @@ func (builder *QueryBuilder) doMergeFiltersOnCompositeKey(tableDef *plan.TableDe } filterIdx := make([]int, 0, numParts) - for _, part := range tableDef.Pkey.Names { + for _, part := range Parts { colIdx := tableDef.Name2ColIndex[part] idx, ok := col2filter[colIdx] if !ok { @@ -212,11 +219,11 @@ func (builder *QueryBuilder) doMergeFiltersOnCompositeKey(tableDef *plan.TableDe var compositePKFilter *plan.Expr pkExpr := &plan.Expr{ - Typ: tableDef.Cols[pkIdx].Typ, + Typ: tableDef.Cols[sortkeyIdx].Typ, Expr: &plan.Expr_Col{ Col: &plan.ColRef{ RelPos: tableTag, - ColPos: pkIdx, + ColPos: sortkeyIdx, }, }, } diff --git a/test/distributed/cases/optimizer/blockfilter.result b/test/distributed/cases/optimizer/blockfilter.result new file mode 100644 index 0000000000000..7a8941cdb3926 --- /dev/null +++ b/test/distributed/cases/optimizer/blockfilter.result @@ -0,0 +1,195 @@ +drop database if exists d1; +create database d1; +use d1; +drop table if exists t1; +drop table if exists t2; +create table t1(c2 int, c1 int, c3 int) cluster by (c1,c2); +create table t2(c1 int, c2 int, c3 int, primary key(c1,c2)); +insert into t1 select result%100,result%10000, result from generate_series(100000) g; +insert into t2 select result%100,*,* from generate_series(1000000) g; +select mo_ctl('dn', 'flush', 'd1.t1'); +mo_ctl(dn, flush, d1.t1) +{\n "method": "Flush",\n "result": [\n {\n "returnStr": "OK"\n }\n ]\n}\n +select mo_ctl('dn', 'flush', 'd1.t2'); +mo_ctl(dn, flush, d1.t2) +{\n "method": "Flush",\n "result": [\n {\n "returnStr": "OK"\n }\n ]\n}\n +select Sleep(1); +Sleep(1) +0 +explain select count(*) from t1 where c1 = 1; +TP QUERY PLAN +Project + -> Aggregate + Aggregate Functions: starcount(1) + -> Table Scan on d1.t1 + Filter Cond: prefix_eq(t1.__mo_cbkey_002c1002c2) + Block Filter Cond: prefix_eq(t1.__mo_cbkey_002c1002c2) +select count(*) from t1 where c1 = 1; +count(*) +10 +explain select count(*) from t1 where c1 > 10; +TP QUERY PLAN +Project + -> Aggregate + Aggregate Functions: starcount(1) + -> Table Scan on d1.t1 + Filter Cond: (t1.c1 > 10) + Block Filter Cond: (t1.c1 > 10) +select count(*) from t1 where c1 > 10; +count(*) +99890 +explain select count(*) from t1 where c1 in (1,2,3); +TP QUERY PLAN +Project + -> Aggregate + Aggregate Functions: starcount(1) + -> Table Scan on d1.t1 + Filter Cond: prefix_in(t1.__mo_cbkey_002c1002c2) + Block Filter Cond: prefix_in(t1.__mo_cbkey_002c1002c2) +select count(*) from t1 where c1 in (1,2,3); +count(*) +30 +explain select count(*) from t1 where c1 between 1 and 5; +TP QUERY PLAN +Project + -> Aggregate + Aggregate Functions: starcount(1) + -> Table Scan on d1.t1 + Filter Cond: t1.c1 BETWEEN 1 AND 5 + Block Filter Cond: t1.c1 BETWEEN 1 AND 5 +select count(*) from t1 where c1 between 1 and 5; +count(*) +50 +explain select count(*) from t1 where c1 = 2 and c2 = 10; +TP QUERY PLAN +Project + -> Aggregate + Aggregate Functions: starcount(1) + -> Table Scan on d1.t1 + Filter Cond: (t1.__mo_cbkey_002c1002c2 = ) + Block Filter Cond: (t1.__mo_cbkey_002c1002c2 = ) +select count(*) from t1 where c1 = 2 and c2 = 10; +count(*) +0 +explain select count(*) from t1 where c1 = 5 and c2 > 10; +TP QUERY PLAN +Project + -> Aggregate + Aggregate Functions: starcount(1) + -> Table Scan on d1.t1 + Filter Cond: prefix_eq(t1.__mo_cbkey_002c1002c2), (t1.c2 > 10) + Block Filter Cond: prefix_eq(t1.__mo_cbkey_002c1002c2), (t1.c2 > 10) +select count(*) from t1 where c1 = 5 and c2 > 10; +count(*) +0 +explain select count(*) from t1 where c1 = 3 and c2 in (1,2,3); +TP QUERY PLAN +Project + -> Aggregate + Aggregate Functions: starcount(1) + -> Table Scan on d1.t1 + Filter Cond: t1.__mo_cbkey_002c1002c2 in () + Block Filter Cond: t1.__mo_cbkey_002c1002c2 in () +select count(*) from t1 where c1 = 3 and c2 in (1,2,3); +count(*) +10 +explain select count(*) from t1 where c1=4 and c2 between 1 and 5; +TP QUERY PLAN +Project + -> Aggregate + Aggregate Functions: starcount(1) + -> Table Scan on d1.t1 + Filter Cond: prefix_eq(t1.__mo_cbkey_002c1002c2), t1.c2 BETWEEN 1 AND 5 + Block Filter Cond: prefix_eq(t1.__mo_cbkey_002c1002c2), t1.c2 BETWEEN 1 AND 5 +select count(*) from t1 where c1=4 and c2 between 1 and 5; +count(*) +10 +explain select count(*) from t2 where c1 = 1; +TP QUERY PLAN +Project + -> Aggregate + Aggregate Functions: starcount(1) + -> Table Scan on d1.t2 + Filter Cond: prefix_eq(t2.__mo_cpkey_col) + Block Filter Cond: prefix_eq(t2.__mo_cpkey_col) +select count(*) from t2 where c1 = 1; +count(*) +10000 +explain select count(*) from t2 where c1 > 10; +AP QUERY PLAN ON ONE CN(4 core) +Project + -> Aggregate + Aggregate Functions: starcount(1) + -> Table Scan on d1.t2 + Filter Cond: (t2.c1 > 10) + Block Filter Cond: (t2.c1 > 10) +select count(*) from t2 where c1 > 10; +count(*) +890000 +explain select count(*) from t2 where c1 in (1,2,3); +TP QUERY PLAN +Project + -> Aggregate + Aggregate Functions: starcount(1) + -> Table Scan on d1.t2 + Filter Cond: prefix_in(t2.__mo_cpkey_col) + Block Filter Cond: prefix_in(t2.__mo_cpkey_col) +select count(*) from t2 where c1 in (1,2,3); +count(*) +30000 +explain select count(*) from t2 where c1 between 1 and 5; +TP QUERY PLAN +Project + -> Aggregate + Aggregate Functions: starcount(1) + -> Table Scan on d1.t2 + Filter Cond: t2.c1 BETWEEN 1 AND 5 + Block Filter Cond: t2.c1 BETWEEN 1 AND 5 +select count(*) from t2 where c1 between 1 and 5; +count(*) +50000 +explain select count(*) from t2 where c1 = 2 and c2 = 10; +TP QUERY PLAN +Project + -> Aggregate + Aggregate Functions: starcount(1) + -> Table Scan on d1.t2 + Filter Cond: (t2.__mo_cpkey_col = ) + Block Filter Cond: (t2.__mo_cpkey_col = ) +select count(*) from t2 where c1 = 2 and c2 = 10; +count(*) +0 +explain select count(*) from t2 where c1 = 5 and c2 > 10; +TP QUERY PLAN +Project + -> Aggregate + Aggregate Functions: starcount(1) + -> Table Scan on d1.t2 + Filter Cond: prefix_eq(t2.__mo_cpkey_col), (t2.c2 > 10) + Block Filter Cond: prefix_eq(t2.__mo_cpkey_col), (t2.c2 > 10) +select count(*) from t2 where c1 = 5 and c2 > 10; +count(*) +9999 +explain select count(*) from t2 where c1 = 3 and c2 in (1,2,3); +TP QUERY PLAN +Project + -> Aggregate + Aggregate Functions: starcount(1) + -> Table Scan on d1.t2 + Filter Cond: t2.__mo_cpkey_col in () + Block Filter Cond: t2.__mo_cpkey_col in () +select count(*) from t2 where c1 = 3 and c2 in (1,2,3); +count(*) +1 +explain select count(*) from t2 where c1=4 and c2 between 1 and 5; +TP QUERY PLAN +Project + -> Aggregate + Aggregate Functions: starcount(1) + -> Table Scan on d1.t2 + Filter Cond: prefix_eq(t2.__mo_cpkey_col), t2.c2 BETWEEN 1 AND 5 + Block Filter Cond: prefix_eq(t2.__mo_cpkey_col), t2.c2 BETWEEN 1 AND 5 +select count(*) from t2 where c1=4 and c2 between 1 and 5; +count(*) +1 +drop database if exists d1; \ No newline at end of file diff --git a/test/distributed/cases/optimizer/blockfilter.test b/test/distributed/cases/optimizer/blockfilter.test new file mode 100644 index 0000000000000..f47979ad2fe3d --- /dev/null +++ b/test/distributed/cases/optimizer/blockfilter.test @@ -0,0 +1,63 @@ +drop database if exists d1; +create database d1; +use d1; +drop table if exists t1; +drop table if exists t2; +create table t1(c2 int, c1 int, c3 int) cluster by (c1,c2); +create table t2(c1 int, c2 int, c3 int, primary key(c1,c2)); +insert into t1 select result%100,result%10000, result from generate_series(100000) g; +insert into t2 select result%100,*,* from generate_series(1000000) g; +-- @separator:table +select mo_ctl('dn', 'flush', 'd1.t1'); +-- @separator:table +select mo_ctl('dn', 'flush', 'd1.t2'); +select Sleep(1); +-- @separator:table +explain select count(*) from t1 where c1 = 1; +select count(*) from t1 where c1 = 1; +-- @separator:table +explain select count(*) from t1 where c1 > 10; +select count(*) from t1 where c1 > 10; +-- @separator:table +explain select count(*) from t1 where c1 in (1,2,3); +select count(*) from t1 where c1 in (1,2,3); +-- @separator:table +explain select count(*) from t1 where c1 between 1 and 5; +select count(*) from t1 where c1 between 1 and 5; +-- @separator:table +explain select count(*) from t1 where c1 = 2 and c2 = 10; +select count(*) from t1 where c1 = 2 and c2 = 10; +-- @separator:table +explain select count(*) from t1 where c1 = 5 and c2 > 10; +select count(*) from t1 where c1 = 5 and c2 > 10; +-- @separator:table +explain select count(*) from t1 where c1 = 3 and c2 in (1,2,3); +select count(*) from t1 where c1 = 3 and c2 in (1,2,3); +-- @separator:table +explain select count(*) from t1 where c1=4 and c2 between 1 and 5; +select count(*) from t1 where c1=4 and c2 between 1 and 5; +-- @separator:table +explain select count(*) from t2 where c1 = 1; +select count(*) from t2 where c1 = 1; +-- @separator:table +explain select count(*) from t2 where c1 > 10; +select count(*) from t2 where c1 > 10; +-- @separator:table +explain select count(*) from t2 where c1 in (1,2,3); +select count(*) from t2 where c1 in (1,2,3); +-- @separator:table +explain select count(*) from t2 where c1 between 1 and 5; +select count(*) from t2 where c1 between 1 and 5; +-- @separator:table +explain select count(*) from t2 where c1 = 2 and c2 = 10; +select count(*) from t2 where c1 = 2 and c2 = 10; +-- @separator:table +explain select count(*) from t2 where c1 = 5 and c2 > 10; +select count(*) from t2 where c1 = 5 and c2 > 10; +-- @separator:table +explain select count(*) from t2 where c1 = 3 and c2 in (1,2,3); +select count(*) from t2 where c1 = 3 and c2 in (1,2,3); +-- @separator:table +explain select count(*) from t2 where c1=4 and c2 between 1 and 5; +select count(*) from t2 where c1=4 and c2 between 1 and 5; +drop database if exists d1; \ No newline at end of file From 44d20215c92ddee9b3ea4cd2195888260098a1da Mon Sep 17 00:00:00 2001 From: Wenbin <85331908+Wenbin1002@users.noreply.github.com> Date: Wed, 11 Dec 2024 01:06:42 +0800 Subject: [PATCH 12/27] add logs in fault enable and disable (#20691) add logs in fault enable and disable Approved by: @XuPeng-SH --- pkg/util/fault/fault.go | 24 ++++++++++++++++++++++-- 1 file changed, 22 insertions(+), 2 deletions(-) diff --git a/pkg/util/fault/fault.go b/pkg/util/fault/fault.go index 9110f58d602c4..5022f26a77c45 100644 --- a/pkg/util/fault/fault.go +++ b/pkg/util/fault/fault.go @@ -27,6 +27,8 @@ import ( jsoniter "github.com/json-iterator/go" "github.com/matrixorigin/matrixone/pkg/common/moerr" + "github.com/matrixorigin/matrixone/pkg/logutil" + "go.uber.org/zap" ) const ( @@ -209,12 +211,30 @@ func stopFaultMap() bool { // Enable fault injection func Enable() bool { - return startFaultMap() + changeStatus := startFaultMap() + status := "enabled" + if changeStatus { + status = "disabled" + } + logutil.Info( + "FAULT-INJECTION-ENABLED", + zap.String("previous-status", status), + ) + return changeStatus } // Disable fault injection func Disable() bool { - return stopFaultMap() + changeStatus := stopFaultMap() + status := "enabled" + if !changeStatus { + status = "disabled" + } + logutil.Info( + "FAULT-INJECTION-DISABLED", + zap.String("previous-status", status), + ) + return changeStatus } func Status() bool { From 48fa1b0b1ba744e6c980c272a6b74ca8cf78b077 Mon Sep 17 00:00:00 2001 From: Wenbin <85331908+Wenbin1002@users.noreply.github.com> Date: Wed, 11 Dec 2024 02:05:13 +0800 Subject: [PATCH 13/27] optimize workspace dump memory usage (#20678) Immediately free the batch that has been flushed when workspace dump Approved by: @triump2020 --- pkg/vm/engine/disttae/txn.go | 33 ++++++++++----------------------- pkg/vm/engine/disttae/types.go | 9 +-------- 2 files changed, 11 insertions(+), 31 deletions(-) diff --git a/pkg/vm/engine/disttae/txn.go b/pkg/vm/engine/disttae/txn.go index ce1a4ae160c9b..998bc5279bf84 100644 --- a/pkg/vm/engine/disttae/txn.go +++ b/pkg/vm/engine/disttae/txn.go @@ -468,9 +468,7 @@ func (txn *Transaction) dumpBatchLocked(ctx context.Context, offset int) error { if !dumpAll { for i := offset; i < len(txn.writes); i++ { - if txn.writes[i].tableId == catalog.MO_DATABASE_ID || - txn.writes[i].tableId == catalog.MO_TABLES_ID || - txn.writes[i].tableId == catalog.MO_COLUMNS_ID { + if txn.writes[i].isCatalog() { continue } if txn.writes[i].bat == nil || txn.writes[i].bat.RowCount() == 0 { @@ -518,9 +516,9 @@ func (txn *Transaction) dumpBatchLocked(ctx context.Context, offset int) error { } func (txn *Transaction) dumpInsertBatchLocked(ctx context.Context, offset int, size *uint64, pkCount *int) error { - mp := make(map[tableKey][]*batch.Batch) lastWritesIndex := offset writes := txn.writes + mp := make(map[tableKey][]*batch.Batch) for i := offset; i < len(txn.writes); i++ { if txn.writes[i].isCatalog() { writes[lastWritesIndex] = writes[i] @@ -545,12 +543,12 @@ func (txn *Transaction) dumpInsertBatchLocked(ctx context.Context, offset int, s *size += uint64(bat.Size()) *pkCount += bat.RowCount() // skip rowid - newBat := batch.NewWithSize(len(bat.Vecs) - 1) - newBat.SetAttributes(bat.Attrs[1:]) - newBat.Vecs = bat.Vecs[1:] - newBat.SetRowCount(bat.Vecs[0].Length()) - mp[tbKey] = append(mp[tbKey], newBat) - txn.toFreeBatches[tbKey] = append(txn.toFreeBatches[tbKey], bat) + newBatch := batch.NewWithSize(len(bat.Vecs) - 1) + newBatch.SetAttributes(bat.Attrs[1:]) + newBatch.Vecs = bat.Vecs[1:] + newBatch.SetRowCount(bat.Vecs[0].Length()) + mp[tbKey] = append(mp[tbKey], newBatch) + defer bat.Clean(txn.proc.GetMPool()) keepElement = false } @@ -626,9 +624,9 @@ func (txn *Transaction) dumpInsertBatchLocked(ctx context.Context, offset int, s func (txn *Transaction) dumpDeleteBatchLocked(ctx context.Context, offset int, size *uint64) error { deleteCnt := 0 - mp := make(map[tableKey][]*batch.Batch) lastWritesIndex := offset writes := txn.writes + mp := make(map[tableKey][]*batch.Batch) for i := offset; i < len(txn.writes); i++ { if txn.writes[i].isCatalog() { writes[lastWritesIndex] = writes[i] @@ -659,7 +657,7 @@ func (txn *Transaction) dumpDeleteBatchLocked(ctx context.Context, offset int, s newBat.SetRowCount(bat.Vecs[0].Length()) mp[tbKey] = append(mp[tbKey], newBat) - txn.toFreeBatches[tbKey] = append(txn.toFreeBatches[tbKey], bat) + defer bat.Clean(txn.proc.GetMPool()) keepElement = false } @@ -1402,7 +1400,6 @@ func (txn *Transaction) delTransaction() { } txn.writes[i].bat.Clean(txn.proc.Mp()) } - txn.CleanToFreeBatches() txn.tableCache = nil txn.tableOps = nil txn.databaseMap = nil @@ -1476,7 +1473,6 @@ func (txn *Transaction) CloneSnapshotWS() client.Workspace { }, cnBlkId_Pos: map[types.Blockid]Pos{}, batchSelectList: make(map[*batch.Batch][]int64), - toFreeBatches: make(map[tableKey][]*batch.Batch), cn_flushed_s3_tombstone_object_stats_list: new(sync.Map), } @@ -1497,15 +1493,6 @@ func (txn *Transaction) GetHaveDDL() bool { return txn.haveDDL.Load() } -func (txn *Transaction) CleanToFreeBatches() { - for key := range txn.toFreeBatches { - for _, bat := range txn.toFreeBatches[key] { - bat.Clean(txn.proc.Mp()) - } - delete(txn.toFreeBatches, key) - } -} - func newTableOps() *tableOpsChain { return &tableOpsChain{ names: make(map[tableKey][]tableOp), diff --git a/pkg/vm/engine/disttae/types.go b/pkg/vm/engine/disttae/types.go index c5bc5360f99b7..2f77c23d67a73 100644 --- a/pkg/vm/engine/disttae/types.go +++ b/pkg/vm/engine/disttae/types.go @@ -317,7 +317,6 @@ type Transaction struct { //} //select list for raw batch comes from txn.writes.batch. batchSelectList map[*batch.Batch][]int64 - toFreeBatches map[tableKey][]*batch.Batch rollbackCount int //current statement id @@ -428,7 +427,6 @@ func NewTxnWorkSpace(eng *Engine, proc *process.Process) *Transaction { }, cnBlkId_Pos: map[types.Blockid]Pos{}, batchSelectList: make(map[*batch.Batch][]int64), - toFreeBatches: make(map[tableKey][]*batch.Batch), syncCommittedTSCount: eng.cli.GetSyncLatestCommitTSTimes(), cn_flushed_s3_tombstone_object_stats_list: new(sync.Map), } @@ -479,7 +477,7 @@ func (txn *Transaction) PPString() string { return buf.String() } - return fmt.Sprintf("Transaction{writes: %v, batchSelectList: %v, tableOps:%v, tablesInVain: %v, tableCache: %v, toFreeBatches: %v, insertCount: %v, snapshotWriteOffset: %v, rollbackCount: %v, statementID: %v, offsets: %v, timestamps: %v}", + return fmt.Sprintf("Transaction{writes: %v, batchSelectList: %v, tableOps:%v, tablesInVain: %v, tableCache: %v, insertCount: %v, snapshotWriteOffset: %v, rollbackCount: %v, statementID: %v, offsets: %v, timestamps: %v}", writesString, stringifyMap(txn.batchSelectList, func(k, v any) string { return fmt.Sprintf("%p:%v", k, len(v.([]int64))) @@ -489,7 +487,6 @@ func (txn *Transaction) PPString() string { return fmt.Sprintf("%v:%v", k.(uint64), v.(int)) }), stringifySyncMap(txn.tableCache), - len(txn.toFreeBatches), txn.approximateInMemInsertCnt, txn.snapshotWriteOffset, txn.rollbackCount, @@ -530,8 +527,6 @@ func (txn *Transaction) IncrStatementID(ctx context.Context, commit bool) error txn.Lock() defer txn.Unlock() - //free batches - txn.CleanToFreeBatches() //merge writes for the last statement if err := txn.mergeTxnWorkspaceLocked(ctx); err != nil { return err @@ -779,8 +774,6 @@ func (txn *Transaction) RollbackLastStatement(ctx context.Context) error { afterEntries = len(txn.writes) - txn.CleanToFreeBatches() - for i := len(txn.restoreTxnTableFunc) - 1; i >= 0; i-- { txn.restoreTxnTableFunc[i]() } From 14a354c38f8378d03976bc6502a8e31892be6a8d Mon Sep 17 00:00:00 2001 From: ou yuanning <45346669+ouyuanning@users.noreply.github.com> Date: Wed, 11 Dec 2024 03:05:06 +0800 Subject: [PATCH 14/27] Lock mo_databases for DDL/DML (#20611) Lock mo_databases for DDL/DML Approved by: @aunjgr, @qingxinhome, @heni02, @badboynt1, @zhangxu19830126, @aressu1985 --- pkg/sql/compile/alter.go | 5 +- pkg/sql/compile/compile.go | 9 ++ pkg/sql/compile/ddl.go | 58 ++++++--- pkg/sql/compile/ddl_test.go | 48 +++++-- .../ddl_atomicity.result | 85 ++++++++++++ .../pessimistic_transaction/ddl_atomicity.sql | 123 ++++++++++++++++++ 6 files changed, 302 insertions(+), 26 deletions(-) create mode 100644 test/distributed/cases/pessimistic_transaction/ddl_atomicity.result create mode 100644 test/distributed/cases/pessimistic_transaction/ddl_atomicity.sql diff --git a/pkg/sql/compile/alter.go b/pkg/sql/compile/alter.go index 0811a2763130c..5ea1fc37a681d 100644 --- a/pkg/sql/compile/alter.go +++ b/pkg/sql/compile/alter.go @@ -36,9 +36,12 @@ func (s *Scope) AlterTableCopy(c *Compile) error { } tblName := qry.GetTableDef().GetName() + if err := lockMoDatabase(c, dbName, lock.LockMode_Shared); err != nil { + return err + } dbSource, err := c.e.Database(c.proc.Ctx, dbName, c.proc.GetTxnOperator()) if err != nil { - return err + return moerr.NewBadDB(c.proc.Ctx, dbName) } originRel, err := dbSource.Relation(c.proc.Ctx, tblName, nil) diff --git a/pkg/sql/compile/compile.go b/pkg/sql/compile/compile.go index abb081f79fe05..13bd89ff906e0 100644 --- a/pkg/sql/compile/compile.go +++ b/pkg/sql/compile/compile.go @@ -737,10 +737,12 @@ func (c *Compile) lockMetaTables() error { } sort.Strings(tables) + lockDbs := make(map[string]struct{}) for _, table := range tables { names := strings.SplitN(table, " ", 2) err := lockMoTable(c, names[0], names[1], lock.LockMode_Shared) + lockDbs[names[0]] = struct{}{} if err != nil { // if get error in locking mocatalog.mo_tables by it's dbName & tblName // that means the origin table's schema was changed. then return NeedRetryWithDefChanged err @@ -753,6 +755,13 @@ func (c *Compile) lockMetaTables() error { return err } } + for dbName := range lockDbs { + err := lockMoDatabase(c, dbName, lock.LockMode_Shared) + if err != nil { + return err + } + } + return nil } diff --git a/pkg/sql/compile/ddl.go b/pkg/sql/compile/ddl.go index 10637ad2e5a8f..6d01a83e803e0 100644 --- a/pkg/sql/compile/ddl.go +++ b/pkg/sql/compile/ddl.go @@ -63,7 +63,7 @@ func (s *Scope) CreateDatabase(c *Compile) error { return moerr.NewDBAlreadyExists(ctx, dbName) } - if err := lockMoDatabase(c, dbName); err != nil { + if err := lockMoDatabase(c, dbName, lock.LockMode_Exclusive); err != nil { return err } @@ -119,7 +119,7 @@ func (s *Scope) DropDatabase(c *Compile) error { return moerr.NewErrDropNonExistsDB(c.proc.Ctx, dbName) } - if err = lockMoDatabase(c, dbName); err != nil { + if err = lockMoDatabase(c, dbName, lock.LockMode_Exclusive); err != nil { return err } @@ -300,6 +300,9 @@ func (s *Scope) AlterView(c *Compile) error { dbName := c.db tblName := qry.GetTableDef().GetName() + if err := lockMoDatabase(c, dbName, lock.LockMode_Shared); err != nil { + return err + } dbSource, err := c.e.Database(c.proc.Ctx, dbName, c.proc.GetTxnOperator()) if err != nil { if qry.GetIfExists() { @@ -311,7 +314,7 @@ func (s *Scope) AlterView(c *Compile) error { if qry.GetIfExists() { return nil } - return err + return moerr.NewBadDB(c.proc.Ctx, dbName) } if err := lockMoTable(c, dbName, tblName, lock.LockMode_Exclusive); err != nil { @@ -383,9 +386,12 @@ func (s *Scope) AlterTableInplace(c *Compile) error { tblName := qry.GetTableDef().GetName() + if err := lockMoDatabase(c, dbName, lock.LockMode_Shared); err != nil { + return err + } dbSource, err := c.e.Database(c.proc.Ctx, dbName, c.proc.GetTxnOperator()) if err != nil { - return err + return moerr.NewBadDB(c.proc.Ctx, dbName) } databaseId := dbSource.GetDatabaseId(c.proc.Ctx) @@ -922,12 +928,16 @@ func (s *Scope) CreateTable(c *Compile) error { } tblName := qry.GetTableDef().GetName() + if err := lockMoDatabase(c, dbName, lock.LockMode_Shared); err != nil { + return err + } + dbSource, err := c.e.Database(c.proc.Ctx, dbName, c.proc.GetTxnOperator()) if err != nil { if dbName == "" { return moerr.NewNoDB(c.proc.Ctx) } - return err + return moerr.NewBadDB(c.proc.Ctx, dbName) } exists, err := dbSource.RelationExists(c.proc.Ctx, tblName, nil) @@ -1485,12 +1495,15 @@ func (s *Scope) CreateView(c *Compile) error { if qry.GetDatabase() != "" { dbName = qry.GetDatabase() } + if err := lockMoDatabase(c, dbName, lock.LockMode_Shared); err != nil { + return err + } dbSource, err := c.e.Database(c.proc.Ctx, dbName, c.proc.GetTxnOperator()) if err != nil { if dbName == "" { return moerr.NewNoDB(c.proc.Ctx) } - return err + return moerr.NewBadDB(c.proc.Ctx, dbName) } viewName := qry.GetTableDef().GetName() @@ -1685,6 +1698,9 @@ func (s *Scope) CreateIndex(c *Compile) error { if qry.GetDatabase() != "" { dbName = qry.GetDatabase() } + if err := lockMoDatabase(c, dbName, lock.LockMode_Shared); err != nil { + return moerr.NewBadDB(c.proc.Ctx, dbName) + } tblName := qry.GetTableDef().GetName() if err := lockMoTable(c, dbName, tblName, lock.LockMode_Exclusive); err != nil { return err @@ -1913,9 +1929,12 @@ func (s *Scope) DropIndex(c *Compile) error { defer s.ScopeAnalyzer.Stop() qry := s.Plan.GetDdl().GetDropIndex() + if err := lockMoDatabase(c, qry.Database, lock.LockMode_Shared); err != nil { + return err + } d, err := c.e.Database(c.proc.Ctx, qry.Database, c.proc.GetTxnOperator()) if err != nil { - return err + return moerr.NewBadDB(c.proc.Ctx, qry.Database) } r, err := d.Relation(c.proc.Ctx, qry.Table, nil) if err != nil { @@ -2160,9 +2179,12 @@ func (s *Scope) TruncateTable(c *Compile) error { keepAutoIncrement := false affectedRows := uint64(0) + if err := lockMoDatabase(c, dbName, lock.LockMode_Shared); err != nil { + return err + } dbSource, err = c.e.Database(c.proc.Ctx, dbName, c.proc.GetTxnOperator()) if err != nil { - return err + return moerr.NewBadDB(c.proc.Ctx, dbName) } if rel, err = dbSource.Relation(c.proc.Ctx, tblName, nil); err != nil { @@ -2395,13 +2417,17 @@ func (s *Scope) DropTable(c *Compile) error { var err error var isTemp bool + if err := lockMoDatabase(c, dbName, lock.LockMode_Shared); err != nil { + return err + } + tblId := qry.GetTableId() dbSource, err = c.e.Database(c.proc.Ctx, dbName, c.proc.GetTxnOperator()) if err != nil { if qry.GetIfExists() { return nil } - return err + return moerr.NewBadDB(c.proc.Ctx, dbName) } if rel, err = dbSource.Relation(c.proc.Ctx, tblName, nil); err != nil { @@ -2759,7 +2785,6 @@ func (s *Scope) CreateSequence(c *Compile) error { dbName = qry.GetDatabase() } tblName := qry.GetTableDef().GetName() - dbSource, err := c.e.Database(c.proc.Ctx, dbName, c.proc.GetTxnOperator()) if err != nil { if dbName == "" { @@ -2831,7 +2856,6 @@ func (s *Scope) AlterSequence(c *Compile) error { dbName = qry.GetDatabase() } tblName := qry.GetTableDef().GetName() - dbSource, err := c.e.Database(c.proc.Ctx, dbName, c.proc.GetTxnOperator()) if err != nil { if dbName == "" { @@ -3685,17 +3709,18 @@ func getLockVector(proc *process.Process, accountId uint32, names []string) (*ve return vec, nil } -func lockMoDatabase(c *Compile, dbName string) error { +func lockMoDatabase(c *Compile, dbName string, lockMode lock.LockMode) error { dbRel, err := getRelFromMoCatalog(c, catalog.MO_DATABASE) if err != nil { return err } - vec, err := getLockVector(c.proc, c.proc.GetSessionInfo().AccountId, []string{dbName}) + accountID := c.proc.GetSessionInfo().AccountId + vec, err := getLockVector(c.proc, accountID, []string{dbName}) if err != nil { return err } defer vec.Free(c.proc.Mp()) - if err := lockRows(c.e, c.proc, dbRel, vec, lock.LockMode_Exclusive, lock.Sharding_ByRow, c.proc.GetSessionInfo().AccountId); err != nil { + if err := lockRows(c.e, c.proc, dbRel, vec, lockMode, lock.Sharding_ByRow, accountID); err != nil { return err } return nil @@ -3710,13 +3735,14 @@ func lockMoTable( if err != nil { return err } - vec, err := getLockVector(c.proc, c.proc.GetSessionInfo().AccountId, []string{dbName, tblName}) + accountID := c.proc.GetSessionInfo().AccountId + vec, err := getLockVector(c.proc, accountID, []string{dbName, tblName}) if err != nil { return err } defer vec.Free(c.proc.Mp()) - if err := lockRows(c.e, c.proc, dbRel, vec, lockMode, lock.Sharding_ByRow, c.proc.GetSessionInfo().AccountId); err != nil { + if err := lockRows(c.e, c.proc, dbRel, vec, lockMode, lock.Sharding_ByRow, accountID); err != nil { return err } return nil diff --git a/pkg/sql/compile/ddl_test.go b/pkg/sql/compile/ddl_test.go index 5d4d1a363436a..9e6f0642a9c0d 100644 --- a/pkg/sql/compile/ddl_test.go +++ b/pkg/sql/compile/ddl_test.go @@ -23,6 +23,7 @@ import ( "github.com/smartystreets/goconvey/convey" "github.com/stretchr/testify/assert" + "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/common/buffer" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/defines" @@ -199,13 +200,21 @@ func TestScope_CreateTable(t *testing.T) { ctx := context.Background() proc.Ctx = context.Background() + txnCli, txnOp := newTestTxnClientAndOp(ctrl) + proc.Base.TxnClient = txnCli + proc.Base.TxnOperator = txnOp proc.ReplaceTopCtx(ctx) + relation := mock_frontend.NewMockRelation(ctrl) + relation.EXPECT().GetTableID(gomock.Any()).Return(uint64(1)).AnyTimes() + eng := mock_frontend.NewMockEngine(ctrl) mockDbMeta := mock_frontend.NewMockDatabase(ctrl) - eng.EXPECT().Database(gomock.Any(), gomock.Any(), gomock.Any()).Return(mockDbMeta, nil) + eng.EXPECT().Database(gomock.Any(), gomock.Any(), gomock.Any()).Return(mockDbMeta, nil).AnyTimes() - mockDbMeta.EXPECT().RelationExists(gomock.Any(), gomock.Any(), gomock.Any()).Return(false, moerr.NewInternalErrorNoCtx("test")) + mockDbMeta.EXPECT().RelationExists(gomock.Any(), "dept", gomock.Any()).Return(false, moerr.NewInternalErrorNoCtx("test")) + + mockDbMeta.EXPECT().Relation(gomock.Any(), catalog.MO_DATABASE, gomock.Any()).Return(relation, nil).AnyTimes() c := NewCompile("test", "test", sql, "", "", eng, proc, nil, false, nil, time.Now()) assert.Error(t, s.CreateTable(c)) @@ -216,6 +225,9 @@ func TestScope_CreateTable(t *testing.T) { defer ctrl.Finish() proc := testutil.NewProcess() + txnCli, txnOp := newTestTxnClientAndOp(ctrl) + proc.Base.TxnClient = txnCli + proc.Base.TxnOperator = txnOp proc.Base.SessionInfo.Buf = buffer.New() ctx := context.Background() @@ -224,9 +236,13 @@ func TestScope_CreateTable(t *testing.T) { relation := mock_frontend.NewMockRelation(ctrl) + meta_relation := mock_frontend.NewMockRelation(ctrl) + meta_relation.EXPECT().GetTableID(gomock.Any()).Return(uint64(1)).AnyTimes() + mockDbMeta := mock_frontend.NewMockDatabase(ctrl) - mockDbMeta.EXPECT().Relation(gomock.Any(), gomock.Any(), gomock.Any()).Return(relation, nil).AnyTimes() - mockDbMeta.EXPECT().RelationExists(gomock.Any(), gomock.Any(), gomock.Any()).Return(false, nil).AnyTimes() + mockDbMeta.EXPECT().Relation(gomock.Any(), "dept", gomock.Any()).Return(relation, nil).AnyTimes() + mockDbMeta.EXPECT().RelationExists(gomock.Any(), "dept", gomock.Any()).Return(false, nil).AnyTimes() + mockDbMeta.EXPECT().Relation(gomock.Any(), catalog.MO_DATABASE, gomock.Any()).Return(meta_relation, nil).AnyTimes() mockDbMeta2 := mock_frontend.NewMockDatabase(ctrl) mockDbMeta2.EXPECT().RelationExists(gomock.Any(), gomock.Any(), gomock.Any()).Return(false, moerr.NewInternalErrorNoCtx("test")) @@ -344,13 +360,20 @@ func TestScope_CreateView(t *testing.T) { ctx := context.Background() proc.Ctx = context.Background() + txnCli, txnOp := newTestTxnClientAndOp(ctrl) + proc.Base.TxnClient = txnCli + proc.Base.TxnOperator = txnOp proc.ReplaceTopCtx(ctx) + meta_relation := mock_frontend.NewMockRelation(ctrl) + meta_relation.EXPECT().GetTableID(gomock.Any()).Return(uint64(1)).AnyTimes() + eng := mock_frontend.NewMockEngine(ctrl) mockDbMeta := mock_frontend.NewMockDatabase(ctrl) - eng.EXPECT().Database(gomock.Any(), gomock.Any(), gomock.Any()).Return(mockDbMeta, nil) + eng.EXPECT().Database(gomock.Any(), gomock.Any(), gomock.Any()).Return(mockDbMeta, nil).AnyTimes() - mockDbMeta.EXPECT().RelationExists(gomock.Any(), gomock.Any(), gomock.Any()).Return(false, moerr.NewInternalErrorNoCtx("test")) + mockDbMeta.EXPECT().RelationExists(gomock.Any(), "v1", gomock.Any()).Return(false, moerr.NewInternalErrorNoCtx("test")) + mockDbMeta.EXPECT().Relation(gomock.Any(), catalog.MO_DATABASE, gomock.Any()).Return(meta_relation, nil).AnyTimes() sql := `create view v1 as select * from dept` c := NewCompile("test", "test", sql, "", "", eng, proc, nil, false, nil, time.Now()) @@ -366,16 +389,23 @@ func TestScope_CreateView(t *testing.T) { ctx := context.Background() proc.Ctx = context.Background() + txnCli, txnOp := newTestTxnClientAndOp(ctrl) + proc.Base.TxnClient = txnCli + proc.Base.TxnOperator = txnOp proc.ReplaceTopCtx(ctx) relation := mock_frontend.NewMockRelation(ctrl) + meta_relation := mock_frontend.NewMockRelation(ctrl) + meta_relation.EXPECT().GetTableID(gomock.Any()).Return(uint64(1)).AnyTimes() + mockDbMeta := mock_frontend.NewMockDatabase(ctrl) - mockDbMeta.EXPECT().Relation(gomock.Any(), gomock.Any(), gomock.Any()).Return(relation, nil).AnyTimes() - mockDbMeta.EXPECT().RelationExists(gomock.Any(), gomock.Any(), gomock.Any()).Return(false, nil).AnyTimes() + mockDbMeta.EXPECT().Relation(gomock.Any(), "v1", gomock.Any()).Return(relation, nil).AnyTimes() + mockDbMeta.EXPECT().RelationExists(gomock.Any(), "v1", gomock.Any()).Return(false, nil).AnyTimes() + mockDbMeta.EXPECT().Relation(gomock.Any(), catalog.MO_DATABASE, gomock.Any()).Return(meta_relation, nil).AnyTimes() mockDbMeta2 := mock_frontend.NewMockDatabase(ctrl) - mockDbMeta2.EXPECT().RelationExists(gomock.Any(), gomock.Any(), gomock.Any()).Return(false, moerr.NewInternalErrorNoCtx("test")) + mockDbMeta2.EXPECT().RelationExists(gomock.Any(), gomock.Any(), gomock.Any()).Return(false, moerr.NewInternalErrorNoCtx("test")).AnyTimes() eng := mock_frontend.NewMockEngine(ctrl) eng.EXPECT().Database(gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn(func(ctx context.Context, name string, arg any) (engine.Database, error) { diff --git a/test/distributed/cases/pessimistic_transaction/ddl_atomicity.result b/test/distributed/cases/pessimistic_transaction/ddl_atomicity.result new file mode 100644 index 0000000000000..b029b520ff83f --- /dev/null +++ b/test/distributed/cases/pessimistic_transaction/ddl_atomicity.result @@ -0,0 +1,85 @@ +create database db1; +use db1; +begin; +create table t2(a int); +drop database db1; +commit; +create database db1; +use db1; +begin; +drop database db1; +use db1; +create table t2(a int); +Unknown database db1 +commit; +create database db1; +use db1; +create table t2(a int); +begin; +drop table t2; +drop database db1; +commit; +create database db1; +use db1; +create table t2(a int); +begin; +drop database db1; +use db1; +drop table t2; +Unknown database db1 +commit; +create database db1; +use db1; +create table t2(a int); +begin; +alter table t2 add b int; +drop database db1; +commit; +create database db1; +use db1; +create table t2(a int); +begin; +drop database db1; +use db1; +alter table t2 add b int; +Unknown database db1 +commit; +create database db1; +use db1; +create table t2(a int); +begin; +create index t2_idx on t2(a); +drop database db1; +commit; +create database db1; +use db1; +create table t2(a int); +begin; +drop database db1; +use db1; +create index t2_idx on t2(a); +Unknown database db1 +commit; +create database db1; +use db1; +create table t2(a int, key t2_idx(a)); +begin; +drop index t2_idx on t2; +drop database db1; +commit; +create database db1; +use db1; +create table t2(a int, key t2_idx(a)); +begin; +drop database db1; +use db1; +drop index t2_idx on t2; +Unknown database db1 +commit; +create database db1; +use db1; +create table t2(a int); +begin; +insert into t2 values(1); +drop database db1; +commit; \ No newline at end of file diff --git a/test/distributed/cases/pessimistic_transaction/ddl_atomicity.sql b/test/distributed/cases/pessimistic_transaction/ddl_atomicity.sql new file mode 100644 index 0000000000000..aeb2e5827cd9d --- /dev/null +++ b/test/distributed/cases/pessimistic_transaction/ddl_atomicity.sql @@ -0,0 +1,123 @@ +create database db1; +use db1; +begin; +create table t2(a int); +-- @session:id=1{ +-- @wait:0:commit +drop database db1; +-- @session} +commit; + +create database db1; +use db1; +begin; +drop database db1; +-- @session:id=1{ +use db1; +-- @wait:0:commit +create table t2(a int); +-- @session} +commit; + +create database db1; +use db1; +create table t2(a int); +begin; +drop table t2; +-- @session:id=1{ +-- @wait:0:commit +drop database db1; +-- @session} +commit; + +create database db1; +use db1; +create table t2(a int); +begin; +drop database db1; +-- @session:id=1{ +use db1; +-- @wait:0:commit +drop table t2; +-- @session} +commit; + +create database db1; +use db1; +create table t2(a int); +begin; +alter table t2 add b int; +-- @session:id=1{ +-- @wait:0:commit +drop database db1; +-- @session} +commit; + +create database db1; +use db1; +create table t2(a int); +begin; +drop database db1; +-- @session:id=1{ +use db1; +-- @wait:0:commit +alter table t2 add b int; +-- @session} +commit; + +create database db1; +use db1; +create table t2(a int); +begin; +create index t2_idx on t2(a); +-- @session:id=1{ +-- @wait:0:commit +drop database db1; +-- @session} +commit; + +create database db1; +use db1; +create table t2(a int); +begin; +drop database db1; +-- @session:id=1{ +use db1; +-- @wait:0:commit +create index t2_idx on t2(a); +-- @session} +commit; + +create database db1; +use db1; +create table t2(a int, key t2_idx(a)); +begin; +drop index t2_idx on t2; +-- @session:id=1{ +-- @wait:0:commit +drop database db1; +-- @session} +commit; + +create database db1; +use db1; +create table t2(a int, key t2_idx(a)); +begin; +drop database db1; +-- @session:id=1{ +use db1; +-- @wait:0:commit +drop index t2_idx on t2; +-- @session} +commit; + +create database db1; +use db1; +create table t2(a int); +begin; +insert into t2 values(1); +-- @session:id=1{ +-- @wait:0:commit +drop database db1; +-- @session} +commit; \ No newline at end of file From f1acf1d942715ada9555d7166d24739aab522031 Mon Sep 17 00:00:00 2001 From: reusee Date: Wed, 11 Dec 2024 04:03:56 +0800 Subject: [PATCH 15/27] fileservice: in minio and qcloud sdk, do not load credentials from env vars if provided in arguments (#20564) do not load credentials from env vars if provided in arguments. Approved by: @fengttt --- pkg/fileservice/minio_sdk.go | 73 ++++++++++++------- pkg/fileservice/minio_sdk_test.go | 25 +++++++ pkg/fileservice/object_storage_arguments.go | 15 ---- .../object_storage_arguments_test.go | 11 --- pkg/fileservice/qcloud_sdk.go | 31 +++++++- pkg/fileservice/utils.go | 10 +++ pkg/fileservice/utils_test.go | 11 ++- 7 files changed, 120 insertions(+), 56 deletions(-) diff --git a/pkg/fileservice/minio_sdk.go b/pkg/fileservice/minio_sdk.go index bec1e4ead4d9a..ff00f486d4347 100644 --- a/pkg/fileservice/minio_sdk.go +++ b/pkg/fileservice/minio_sdk.go @@ -21,6 +21,7 @@ import ( "iter" "net/http" "net/url" + "os" gotrace "runtime/trace" "strings" "sync/atomic" @@ -56,49 +57,69 @@ func NewMinioSDK( options := new(minio.Options) - // credentials - var credentialProviders []credentials.Provider + // credential arguments + keyID := args.KeyID + keySecret := args.KeySecret + sessionToken := args.SessionToken if args.shouldLoadDefaultCredentials() { - credentialProviders = append(credentialProviders, - // aws env - new(credentials.EnvAWS), - // minio env - new(credentials.EnvMinio), + keyID = firstNonZero( + args.KeyID, + os.Getenv("AWS_ACCESS_KEY_ID"), + os.Getenv("AWS_ACCESS_KEY"), + os.Getenv("MINIO_ROOT_USER"), + os.Getenv("MINIO_ACCESS_KEY"), + ) + keySecret = firstNonZero( + args.KeySecret, + os.Getenv("AWS_SECRET_ACCESS_KEY"), + os.Getenv("AWS_SECRET_KEY"), + os.Getenv("MINIO_ROOT_PASSWORD"), + os.Getenv("MINIO_SECRET_KEY"), + ) + sessionToken = firstNonZero( + args.SessionToken, + os.Getenv("AWS_SESSION_TOKEN"), ) } - if args.KeyID != "" && args.KeySecret != "" { + + // credentials providers + var credentialProviders []credentials.Provider + + if keyID != "" && keySecret != "" { // static credentialProviders = append(credentialProviders, &credentials.Static{ Value: credentials.Value{ - AccessKeyID: args.KeyID, - SecretAccessKey: args.KeySecret, - SessionToken: args.SessionToken, + AccessKeyID: keyID, + SecretAccessKey: keySecret, + SessionToken: sessionToken, SignerType: credentials.SignatureV2, }, }) credentialProviders = append(credentialProviders, &credentials.Static{ Value: credentials.Value{ - AccessKeyID: args.KeyID, - SecretAccessKey: args.KeySecret, - SessionToken: args.SessionToken, + AccessKeyID: keyID, + SecretAccessKey: keySecret, + SessionToken: sessionToken, SignerType: credentials.SignatureV4, }, }) credentialProviders = append(credentialProviders, &credentials.Static{ Value: credentials.Value{ - AccessKeyID: args.KeyID, - SecretAccessKey: args.KeySecret, - SessionToken: args.SessionToken, + AccessKeyID: keyID, + SecretAccessKey: keySecret, + SessionToken: sessionToken, SignerType: credentials.SignatureDefault, }, }) + } + if args.RoleARN != "" { // assume role credentialProviders = append(credentialProviders, &credentials.STSAssumeRole{ Options: credentials.STSAssumeRoleOptions{ - AccessKey: args.KeyID, - SecretKey: args.KeySecret, + AccessKey: keyID, + SecretKey: keySecret, RoleARN: args.RoleARN, RoleSessionName: args.ExternalID, }, @@ -107,23 +128,23 @@ func NewMinioSDK( // special treatments for 天翼云 if strings.Contains(args.Endpoint, "ctyunapi.cn") { - if args.KeyID == "" { + if keyID == "" { // try to fetch one creds := credentials.NewChainCredentials(credentialProviders) value, err := creds.Get() if err != nil { return nil, err } - args.KeyID = value.AccessKeyID - args.KeySecret = value.SecretAccessKey - args.SessionToken = value.SessionToken + keyID = value.AccessKeyID + keySecret = value.SecretAccessKey + sessionToken = value.SessionToken } credentialProviders = []credentials.Provider{ &credentials.Static{ Value: credentials.Value{ - AccessKeyID: args.KeyID, - SecretAccessKey: args.KeySecret, - SessionToken: args.SessionToken, + AccessKeyID: keyID, + SecretAccessKey: keySecret, + SessionToken: sessionToken, SignerType: credentials.SignatureV2, }, }, diff --git a/pkg/fileservice/minio_sdk_test.go b/pkg/fileservice/minio_sdk_test.go index b5e37382e5fef..f4b6c17865bc5 100644 --- a/pkg/fileservice/minio_sdk_test.go +++ b/pkg/fileservice/minio_sdk_test.go @@ -139,3 +139,28 @@ func startMinio(dir string) (*exec.Cmd, error) { return cmd, nil } + +func TestMinioSDKRoleARN(t *testing.T) { + _, err := NewMinioSDK( + context.Background(), + ObjectStorageArguments{ + Endpoint: "http://localhost", + RoleARN: "abc", + NoBucketValidation: true, + }, + nil, + ) + assert.Nil(t, err) +} + +func TestMinioSDKTianYiYun(t *testing.T) { + _, err := NewMinioSDK( + context.Background(), + ObjectStorageArguments{ + Endpoint: "http://ctyunapi.cn", + NoBucketValidation: true, + }, + nil, + ) + assert.Nil(t, err) +} diff --git a/pkg/fileservice/object_storage_arguments.go b/pkg/fileservice/object_storage_arguments.go index 348270454b278..bc592fa4a0c54 100644 --- a/pkg/fileservice/object_storage_arguments.go +++ b/pkg/fileservice/object_storage_arguments.go @@ -18,7 +18,6 @@ import ( "encoding/json" "net/http" "net/url" - "os" "regexp" "strconv" "strings" @@ -181,20 +180,6 @@ func (o *ObjectStorageArguments) validate() error { o.RoleSessionName = "mo-service" } - // 腾讯云使用 AWS 环境变量配置 key id/secret - if strings.Contains(o.Endpoint, "myqcloud.com") { - if o.KeyID == "" { - if value := os.Getenv("AWS_ACCESS_KEY_ID"); value != "" { - o.KeyID = value - } - } - if o.KeySecret == "" { - if value := os.Getenv("AWS_SECRET_ACCESS_KEY"); value != "" { - o.KeySecret = value - } - } - } - return nil } diff --git a/pkg/fileservice/object_storage_arguments_test.go b/pkg/fileservice/object_storage_arguments_test.go index 5dc846d289c43..bed9c71b20e1a 100644 --- a/pkg/fileservice/object_storage_arguments_test.go +++ b/pkg/fileservice/object_storage_arguments_test.go @@ -167,14 +167,3 @@ func TestAWSRegion(t *testing.T) { args.validate() assert.Equal(t, "us-east-1", args.Region) } - -func TestQCloudKeyIDSecretFromAwsEnv(t *testing.T) { - args := ObjectStorageArguments{ - Endpoint: "http://cos.foobar.myqcloud.com", - } - t.Setenv("AWS_ACCESS_KEY_ID", "foo") - t.Setenv("AWS_SECRET_ACCESS_KEY", "bar") - args.validate() - assert.Equal(t, "foo", args.KeyID) - assert.Equal(t, "bar", args.KeySecret) -} diff --git a/pkg/fileservice/qcloud_sdk.go b/pkg/fileservice/qcloud_sdk.go index d7c888bc53907..9a5a8e03b5c39 100644 --- a/pkg/fileservice/qcloud_sdk.go +++ b/pkg/fileservice/qcloud_sdk.go @@ -23,6 +23,7 @@ import ( "iter" "net/http" "net/url" + "os" gotrace "runtime/trace" "strconv" "time" @@ -64,12 +65,36 @@ func NewQCloudSDK( return nil, err } + // credential arguments + keyID := args.KeyID + keySecret := args.KeySecret + sessionToken := args.SessionToken + if args.shouldLoadDefaultCredentials() { + keyID = firstNonZero( + args.KeyID, + os.Getenv("AWS_ACCESS_KEY_ID"), + os.Getenv("AWS_ACCESS_KEY"), + os.Getenv("TENCENTCLOUD_SECRETID"), + ) + keySecret = firstNonZero( + args.KeySecret, + os.Getenv("AWS_SECRET_ACCESS_KEY"), + os.Getenv("AWS_SECRET_KEY"), + os.Getenv("TENCENTCLOUD_SECRETKEY"), + ) + sessionToken = firstNonZero( + args.SessionToken, + os.Getenv("AWS_SESSION_TOKEN"), + os.Getenv("TENCENTCLOUD_SESSIONTOKEN"), + ) + } + // http client httpClient := newHTTPClient(args) httpClient.Transport = &cos.AuthorizationTransport{ - SecretID: args.KeyID, - SecretKey: args.KeySecret, - SessionToken: args.SessionToken, + SecretID: keyID, + SecretKey: keySecret, + SessionToken: sessionToken, Transport: httpClient.Transport, } diff --git a/pkg/fileservice/utils.go b/pkg/fileservice/utils.go index 88de05eb3cc2f..00cca45f0a482 100644 --- a/pkg/fileservice/utils.go +++ b/pkg/fileservice/utils.go @@ -52,3 +52,13 @@ func SortedList(seq iter.Seq2[*DirEntry, error]) (ret []DirEntry, err error) { }) return } + +func firstNonZero[T comparable](args ...T) T { + var zero T + for _, arg := range args { + if arg != zero { + return arg + } + } + return zero +} diff --git a/pkg/fileservice/utils_test.go b/pkg/fileservice/utils_test.go index eb23ad24ec1e5..1d3be67bc6abf 100644 --- a/pkg/fileservice/utils_test.go +++ b/pkg/fileservice/utils_test.go @@ -14,7 +14,11 @@ package fileservice -import "testing" +import ( + "testing" + + "github.com/stretchr/testify/assert" +) func TestZeroToNil(t *testing.T) { if testing.AllocsPerRun(10, func() { @@ -28,3 +32,8 @@ func TestZeroToNil(t *testing.T) { t.Fatal() } } + +func TestFirstNonZero(t *testing.T) { + assert.Equal(t, 42, firstNonZero(0, 42)) + assert.Equal(t, 0, firstNonZero[int]()) +} From 1c9177253e711b27209473d70fcef61a416b7d7b Mon Sep 17 00:00:00 2001 From: gouhongshen Date: Wed, 11 Dec 2024 11:14:10 +0800 Subject: [PATCH 16/27] making partition state index read only with no copy. (#20699) 1. making partition state index read-only with no copy. 2. reducing the heap allocation in partition state b-tree. Approved by: @XuPeng-SH --- .../disttae/logtailreplay/blocks_iter.go | 22 +++++++-------- .../disttae/logtailreplay/change_handle.go | 8 +++--- .../disttae/logtailreplay/partition_state.go | 27 +++++++++---------- .../engine/disttae/logtailreplay/rows_iter.go | 12 ++++----- 4 files changed, 34 insertions(+), 35 deletions(-) diff --git a/pkg/vm/engine/disttae/logtailreplay/blocks_iter.go b/pkg/vm/engine/disttae/logtailreplay/blocks_iter.go index 5e6e50bbf8860..29cf2bd689e12 100644 --- a/pkg/vm/engine/disttae/logtailreplay/blocks_iter.go +++ b/pkg/vm/engine/disttae/logtailreplay/blocks_iter.go @@ -87,7 +87,7 @@ func (p *PartitionState) newTombstoneObjectsIter( snapshot types.TS, onlyVisible bool) (ObjectsIter, error) { - iter := p.tombstoneObjectDTSIndex.Copy().Iter() + iter := p.tombstoneObjectDTSIndex.Iter() if onlyVisible { pivot := ObjectEntry{ ObjectInfo{ @@ -99,7 +99,7 @@ func (p *PartitionState) newTombstoneObjectsIter( if !iter.Prev() && p.tombstoneObjectDTSIndex.Len() > 0 { // reset iter only when seeked to the first item iter.Release() - iter = p.tombstoneObjectDTSIndex.Copy().Iter() + iter = p.tombstoneObjectDTSIndex.Iter() } } @@ -115,7 +115,7 @@ func (p *PartitionState) newDataObjectIter( snapshot types.TS, onlyVisible bool) (ObjectsIter, error) { - iter := p.dataObjectsNameIndex.Copy().Iter() + iter := p.dataObjectsNameIndex.Iter() ret := &objectsIter{ onlyVisible: onlyVisible, ts: snapshot, @@ -165,7 +165,7 @@ func (p *PartitionState) HasTombstoneChanged(from, to types.TS) (exist bool) { if p.tombstoneObjectDTSIndex.Len() == 0 { return false } - iter := p.tombstoneObjectDTSIndex.Copy().Iter() + iter := p.tombstoneObjectDTSIndex.Iter() defer iter.Release() // Created after from @@ -195,7 +195,7 @@ func (p *PartitionState) GetChangedObjsBetween( inserted = make(map[objectio.ObjectNameShort]struct{}) deleted = make(map[objectio.ObjectNameShort]struct{}) - iter := p.dataObjectTSIndex.Copy().Iter() + iter := p.dataObjectTSIndex.Iter() defer iter.Release() for ok := iter.Seek(ObjectIndexByTSEntry{ @@ -223,7 +223,7 @@ func (p *PartitionState) GetChangedObjsBetween( } func (p *PartitionState) BlockPersisted(blockID *types.Blockid) bool { - iter := p.dataObjectsNameIndex.Copy().Iter() + iter := p.dataObjectsNameIndex.Iter() defer iter.Release() pivot := ObjectEntry{} @@ -241,7 +241,7 @@ func (p *PartitionState) CollectObjectsBetween( start, end types.TS, ) (insertList, deletedList []objectio.ObjectStats) { - iter := p.dataObjectTSIndex.Copy().Iter() + iter := p.dataObjectTSIndex.Iter() defer iter.Release() if !iter.Seek(ObjectIndexByTSEntry{ @@ -250,7 +250,7 @@ func (p *PartitionState) CollectObjectsBetween( return } - nameIdx := p.dataObjectsNameIndex.Copy() + nameIdx := p.dataObjectsNameIndex for ok := true; ok; ok = iter.Next() { entry := iter.Item() @@ -303,9 +303,9 @@ func (p *PartitionState) CheckIfObjectDeletedBeforeTS( var tree *btree.BTreeG[ObjectEntry] if isTombstone { - tree = p.tombstoneObjectsNameIndex.Copy() + tree = p.tombstoneObjectsNameIndex } else { - tree = p.dataObjectsNameIndex.Copy() + tree = p.dataObjectsNameIndex } var stats objectio.ObjectStats @@ -324,7 +324,7 @@ func (p *PartitionState) CheckIfObjectDeletedBeforeTS( } func (p *PartitionState) GetObject(name objectio.ObjectNameShort) (ObjectInfo, bool) { - iter := p.dataObjectsNameIndex.Copy().Iter() + iter := p.dataObjectsNameIndex.Iter() defer iter.Release() pivot := ObjectEntry{} diff --git a/pkg/vm/engine/disttae/logtailreplay/change_handle.go b/pkg/vm/engine/disttae/logtailreplay/change_handle.go index 896f54c05979c..19bafbb69138b 100755 --- a/pkg/vm/engine/disttae/logtailreplay/change_handle.go +++ b/pkg/vm/engine/disttae/logtailreplay/change_handle.go @@ -513,17 +513,17 @@ func NewBaseHandler(state *PartitionState, changesHandle *ChangeHandler, start, } var iter btree.IterG[ObjectEntry] if tombstone { - iter = state.tombstoneObjectsNameIndex.Copy().Iter() + iter = state.tombstoneObjectsNameIndex.Iter() } else { - iter = state.dataObjectsNameIndex.Copy().Iter() + iter = state.dataObjectsNameIndex.Iter() } defer iter.Release() if tombstone { - dataIter := state.dataObjectsNameIndex.Copy().Iter() + dataIter := state.dataObjectsNameIndex.Iter() p.fillInSkipTS(dataIter, start, end) dataIter.Release() } - rowIter := state.rows.Copy().Iter() + rowIter := state.rows.Iter() defer rowIter.Release() p.inMemoryHandle = p.newBatchHandleWithRowIterator(ctx, rowIter, start, end, tombstone, mp) aobj, cnObj := p.getObjectEntries(iter, start, end) diff --git a/pkg/vm/engine/disttae/logtailreplay/partition_state.go b/pkg/vm/engine/disttae/logtailreplay/partition_state.go index 343d650037e87..8a0f69c92e4bb 100644 --- a/pkg/vm/engine/disttae/logtailreplay/partition_state.go +++ b/pkg/vm/engine/disttae/logtailreplay/partition_state.go @@ -587,17 +587,15 @@ func (p *PartitionState) Copy() *PartitionState { rows: p.rows.Copy(), dataObjectsNameIndex: p.dataObjectsNameIndex.Copy(), tombstoneObjectsNameIndex: p.tombstoneObjectsNameIndex.Copy(), - //blockDeltas: p.blockDeltas.Copy(), - rowPrimaryKeyIndex: p.rowPrimaryKeyIndex.Copy(), - inMemTombstoneRowIdIndex: p.inMemTombstoneRowIdIndex.Copy(), - noData: p.noData, - //dirtyBlocks: p.dirtyBlocks.Copy(), - dataObjectTSIndex: p.dataObjectTSIndex.Copy(), - tombstoneObjectDTSIndex: p.tombstoneObjectDTSIndex.Copy(), - shared: p.shared, - lastFlushTimestamp: p.lastFlushTimestamp, - start: p.start, - end: p.end, + rowPrimaryKeyIndex: p.rowPrimaryKeyIndex.Copy(), + inMemTombstoneRowIdIndex: p.inMemTombstoneRowIdIndex.Copy(), + noData: p.noData, + dataObjectTSIndex: p.dataObjectTSIndex.Copy(), + tombstoneObjectDTSIndex: p.tombstoneObjectDTSIndex.Copy(), + shared: p.shared, + lastFlushTimestamp: p.lastFlushTimestamp, + start: p.start, + end: p.end, } if len(p.checkpoints) > 0 { state.checkpoints = make([]string, len(p.checkpoints)) @@ -642,7 +640,8 @@ func NewPartitionState( tid uint64, ) *PartitionState { opts := btree.Options{ - Degree: 64, + Degree: 32, // may good for heap alloc + NoLocks: true, } ps := &PartitionState{ service: service, @@ -795,7 +794,7 @@ func (p *PartitionState) PKExistInMemBetween( to types.TS, keys [][]byte, ) (bool, bool) { - iter := p.rowPrimaryKeyIndex.Copy().Iter() + iter := p.rowPrimaryKeyIndex.Iter() pivot := RowEntry{ Time: types.BuildTS(math.MaxInt64, math.MaxUint32), } @@ -952,7 +951,7 @@ func (p *PartitionState) ScanRows( } func (p *PartitionState) CheckRowIdDeletedInMem(ts types.TS, rowId types.Rowid) bool { - iter := p.rows.Copy().Iter() + iter := p.rows.Iter() defer iter.Release() if !iter.Seek(RowEntry{ diff --git a/pkg/vm/engine/disttae/logtailreplay/rows_iter.go b/pkg/vm/engine/disttae/logtailreplay/rows_iter.go index 7b503257a5452..962e6268c01c1 100644 --- a/pkg/vm/engine/disttae/logtailreplay/rows_iter.go +++ b/pkg/vm/engine/disttae/logtailreplay/rows_iter.go @@ -589,7 +589,7 @@ func (p *primaryKeyDelIter) Next() bool { } func (p *PartitionState) NewRowsIter(ts types.TS, blockID *types.Blockid, iterDeleted bool) *rowsIter { - iter := p.rows.Copy().Iter() + iter := p.rows.Iter() ret := &rowsIter{ ts: ts, iter: iter, @@ -606,13 +606,13 @@ func (p *PartitionState) NewPrimaryKeyIter( ts types.TS, spec PrimaryKeyMatchSpec, ) *primaryKeyIter { - index := p.rowPrimaryKeyIndex.Copy() + index := p.rowPrimaryKeyIndex return &primaryKeyIter{ ts: ts, spec: spec, iter: index.Iter(), primaryIndex: index, - rows: p.rows.Copy(), + rows: p.rows, } } @@ -621,15 +621,15 @@ func (p *PartitionState) NewPrimaryKeyDelIter( spec PrimaryKeyMatchSpec, bid *types.Blockid, ) *primaryKeyDelIter { - index := p.rowPrimaryKeyIndex.Copy() + index := p.rowPrimaryKeyIndex delIter := &primaryKeyDelIter{ primaryKeyIter: primaryKeyIter{ ts: *ts, spec: spec, primaryIndex: index, iter: index.Iter(), - rows: p.rows.Copy(), - tombstoneRowIdIdx: p.inMemTombstoneRowIdIndex.Copy(), + rows: p.rows, + tombstoneRowIdIdx: p.inMemTombstoneRowIdIndex, }, bid: *bid, } From 45ce18a0a692003317c80d47da775f3851f877c4 Mon Sep 17 00:00:00 2001 From: reusee Date: Wed, 11 Dec 2024 12:12:06 +0800 Subject: [PATCH 17/27] fileservice: close idle connections periodically (#20520) close idle connections periodically to avoid connections leak Approved by: @fengttt --- pkg/fileservice/http_client.go | 20 ++++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/pkg/fileservice/http_client.go b/pkg/fileservice/http_client.go index 9107281c86e83..1c7fc5edcc71f 100644 --- a/pkg/fileservice/http_client.go +++ b/pkg/fileservice/http_client.go @@ -43,7 +43,7 @@ var dnsResolver = dns.NewCachingResolver( func init() { net.DefaultResolver = dnsResolver - http.DefaultTransport = httpTransport + http.DefaultTransport = httpRoundTripper } var httpDialer = &net.Dialer{ @@ -51,7 +51,7 @@ var httpDialer = &net.Dialer{ Resolver: dnsResolver, } -var httpTransport = wrapRoundTripper(&http.Transport{ +var httpTransport = &http.Transport{ DialContext: wrapDialContext(httpDialer.DialContext), MaxIdleConns: maxIdleConns, IdleConnTimeout: idleConnTimeout, @@ -64,7 +64,19 @@ var httpTransport = wrapRoundTripper(&http.Transport{ RootCAs: caPool, }, Proxy: http.ProxyFromEnvironment, -}) +} + +func init() { + // don't know why there is a large number of connections even though MaxConnsPerHost is set. + // close idle connections periodically. + go func() { + for range time.NewTicker(time.Second).C { + httpTransport.CloseIdleConnections() + } + }() +} + +var httpRoundTripper = wrapRoundTripper(httpTransport) var caPool = func() *x509.CertPool { pool, err := x509.SystemCertPool() @@ -97,7 +109,7 @@ func newHTTPClient(args ObjectStorageArguments) *http.Client { // client client := &http.Client{ - Transport: httpTransport, + Transport: httpRoundTripper, } return client From f51f4b9a3b73d93579338b8c9b2aadc7d8035665 Mon Sep 17 00:00:00 2001 From: Songxx-7 <144108388+Songxx-7@users.noreply.github.com> Date: Wed, 11 Dec 2024 13:33:28 +0800 Subject: [PATCH 18/27] Add security case:base main (#20434) test Approved by: @YANGGMM, @aressu1985 --- .../cases/security/password.result | 266 +++++++++++++++ test/distributed/cases/security/password.sql | 307 ++++++++++++++++++ 2 files changed, 573 insertions(+) create mode 100644 test/distributed/cases/security/password.result create mode 100644 test/distributed/cases/security/password.sql diff --git a/test/distributed/cases/security/password.result b/test/distributed/cases/security/password.result new file mode 100644 index 0000000000000..fc92083775a01 --- /dev/null +++ b/test/distributed/cases/security/password.result @@ -0,0 +1,266 @@ +sion:id=0&user=dump&password=111 +create user user1_group0 identified by '1234'; +SQL parser error: You have an error in your SQL syntax; check the manual that corresponds to your MatrixOne server version for the right syntax to use. syntax error at line 1 column 4 near "sion:id=0&user=dump&password=111 +create user user1_group0 identified by '1234';"; +drop user if exists user1_group0; +set global validate_password = 'ON'; +set global validate_password.check_user_name = 'ON'; +set global validate_password.changed_characters_percentage = 0; +set global validate_password.length = 5; +set global validate_password.mixed_case_count = 0; +set global validate_password.number_count = 0; +set global validate_password.special_char_count = 0; +set global validate_password.policy = '0'; +show variables like "%validate_password%"; +Variable_name Value +validate_password on +validate_password.changed_characters_percentage 0 +validate_password.check_user_name on +validate_password.length 5 +validate_password.mixed_case_count 0 +validate_password.number_count 0 +validate_password.policy 0 +validate_password.special_char_count 0 +create user user1_group1 identified by '1234'; -- fail +invalid input: Password '1234' is too short, require at least 5 characters +create user user2_group1 identified by 'abc'; -- fail +invalid input: Password 'abc' is too short, require at least 5 characters +create user user3_group1 identified by '12345'; -- success +create user user4_group1 identified by 'abcde'; -- success +select user_name from mo_catalog.mo_user where user_name in ('user3_group1', 'user4_group1'); +user_name +user3_group1 +user4_group1 +drop user if exists user1_group1, user2_group1, user3_group1, user4_group1; +set global validate_password = 'ON'; +set global validate_password.check_user_name = 'ON'; +set global validate_password.changed_characters_percentage = 60; +set global validate_password.length = 8; +set global validate_password.mixed_case_count = 2; +set global validate_password.number_count = 2; +set global validate_password.special_char_count = 0; +set global validate_password.policy = '1'; +show variables like "%validate_password%"; +Variable_name Value +validate_password on +validate_password.changed_characters_percentage 60 +validate_password.check_user_name on +validate_password.length 8 +validate_password.mixed_case_count 2 +validate_password.number_count 2 +validate_password.policy 1 +validate_password.special_char_count 0 +create user user1_group2 identified by '12345678'; -- Expected failure +invalid input: Password '12345678' does not contain enough changed characters +create user user2_group2 identified by 'abcdefgH'; -- Expected failure +invalid input: Password 'abcdefgH' does not meet the Uppercase requirements +create user useR32Go identified by 'oG23Resu'; -- Expected failure +create user user4_group2 identified by 'AbCLq56%'; -- Expected success +select user_name from mo_catalog.mo_user where user_name in ('user3_group2', 'user4_group2'); +user_name +user4_group2 +drop user if exists user1_group2, user2_group2, useR32Go, user4_group2; +set global validate_password = 'ON'; +set global validate_password.check_user_name = 'OFF'; +set global validate_password.changed_characters_percentage = 50; +set global validate_password.length = 30; +set global validate_password.mixed_case_count = 5; +set global validate_password.number_count = 8; +set global validate_password.special_char_count = 2; +set global validate_password.policy = '1'; +create user user1_group3 identified by 'abcdefgH'; -- Expected failure +invalid input: Password 'abcdefgH' is too short, require at least 30 characters +create user user2_group3 identified by '1234abcd'; -- Expected failure +invalid input: Password '1234abcd' is too short, require at least 30 characters +create user user3_group3 identified by 'Abc123KLedjrg563O28d'; -- Expected failure +invalid input: Password 'Abc123KLedjrg563O28d' is too short, require at least 30 characters +create user user4_group3 identified by 'Xyz78pLPAbc123JHedjrg563Ukkd_%'; -- Expected success +select user_name from mo_catalog.mo_user where user_name in ('user4_group3'); +user_name +user4_group3 +drop user if exists user1_group3, user2_group3, user3_group3, user4_group3; +set global validate_password = 'ON'; +set global validate_password.check_user_name = 'ON'; +set global validate_password.changed_characters_percentage = 40; +set global validate_password.length = 15; +set global validate_password.mixed_case_count = 3; +set global validate_password.number_count = 5; +set global validate_password.special_char_count = 3; +set global validate_password.policy = '1'; +create user user1_group4 identified by '123456789abcde'; -- Expected failure +invalid input: Password '123456789abcde' does not contain enough changed characters +create user user2_group4 identified by 'Abcdefg1234567'; -- Expected failure +invalid input: Password 'Abcdefg1234567' is too short, require at least 15 characters +create user user3_group4 identified by 'Abc123%_@RMXy78'; -- Expected success +select user_name from mo_catalog.mo_user where user_name in ('user3_group4', 'user4_group4'); +user_name +user3_group4 +drop user if exists user1_group4, user2_group4, user3_group4, user4_group4; +set global validate_password = 'HELLO'; +internal error: convert to the system variable bool type failed +set global validate_password.check_user_name = 'IN'; +internal error: convert to the system variable bool type failed +set global validate_password.changed_characters_percentage = 101; +internal error: convert to the system variable int type failed +set global validate_password.length = inv; +internal error: convert to the system variable int type failed +set global validate_password.mixed_case_count = abc; +internal error: convert to the system variable int type failed +set global validate_password.number_count = -3; +internal error: convert to the system variable int type failed +set global validate_password.special_char_count = -1; +internal error: convert to the system variable int type failed +set global validate_password.policy = 'HIGH'; +internal error: convert to the system variable int type failed +set global validate_password = 'OFF'; +set global validate_password.check_user_name = 'ON'; +set global validate_password.changed_characters_percentage = 0; +set global validate_password.length = 8; +set global validate_password.mixed_case_count = 1; +set global validate_password.number_count = 1; +set global validate_password.special_char_count = 1; +set global validate_password.policy = '0'; +SHOW VARIABLES LIKE "%connection_control_failed_connections_threshold%"; +Variable_name Value +connection_control_failed_connections_threshold 3 +SHOW VARIABLES LIKE "%connection_control_max_connection_delay%"; +Variable_name Value +connection_control_max_connection_delay 0 +SHOW VARIABLES LIKE "%password_history%"; +Variable_name Value +password_history 0 +SHOW VARIABLES LIKE "%password_reuse_interval%"; +Variable_name Value +password_reuse_interval 0 +SET GLOBAL connection_control_failed_connections_threshold = 3; +SET GLOBAL connection_control_max_connection_delay = 5000; +SET GLOBAL password_history = 0; +SET GLOBAL password_reuse_interval = 0; +SHOW VARIABLES LIKE "%connection_control_failed_connections_threshold%"; +Variable_name Value +connection_control_failed_connections_threshold 3 +SHOW VARIABLES LIKE "%connection_control_max_connection_delay%"; +Variable_name Value +connection_control_max_connection_delay 5000 +SHOW VARIABLES LIKE "%password_history%"; +Variable_name Value +password_history 0 +SHOW VARIABLES LIKE "%password_reuse_interval%"; +Variable_name Value +password_reuse_interval 0 +CREATE USER user1_group1 IDENTIFIED BY 'oldpassword'; +CREATE USER user2_group1 IDENTIFIED BY 'oldpassword2'; +ALTER USER user1_group1 IDENTIFIED BY 'newpassword1'; +SELECT SLEEP(1); +SLEEP(1) +0 +ALTER USER user2_group1 IDENTIFIED BY 'newpassword2'; +SELECT SLEEP(1); +SLEEP(1) +0 +SELECT user_name FROM mo_catalog.mo_user WHERE user_name IN ('user1_group1', 'user2_group1'); +user_name +user1_group1 +user2_group1 +DROP USER IF EXISTS user1_group1, user2_group1; +SET GLOBAL connection_control_failed_connections_threshold = 5; +SET GLOBAL connection_control_max_connection_delay = 1000; +SET GLOBAL password_history = 6; +SET GLOBAL password_reuse_interval = 10; +SHOW VARIABLES LIKE "%connection_control_failed_connections_threshold%"; +Variable_name Value +connection_control_failed_connections_threshold 5 +SHOW VARIABLES LIKE "%connection_control_max_connection_delay%"; +Variable_name Value +connection_control_max_connection_delay 1000 +SHOW VARIABLES LIKE "%password_history%"; +Variable_name Value +password_history 6 +SHOW VARIABLES LIKE "%password_reuse_interval%"; +Variable_name Value +password_reuse_interval 10 +CREATE USER user1_group2 IDENTIFIED BY 'oldpassword1'; +CREATE USER user2_group2 IDENTIFIED BY 'oldpassword2'; +ALTER USER user1_group2 IDENTIFIED BY 'newpassword1!'; +SELECT SLEEP(1); +SLEEP(1) +0 +ALTER USER user2_group2 IDENTIFIED BY 'newpassword2!'; +SELECT SLEEP(1); +SLEEP(1) +0 +ALTER USER user1_group2 IDENTIFIED BY 'oldpassword1'; -- Expected failure: Old password cannot be reused +invalid input: The password has been used before, please change another one. +SELECT SLEEP(1); +SLEEP(1) +0 +ALTER USER user2_group2 IDENTIFIED BY 'oldpassword2'; -- Expected failure: Old password cannot be reused +invalid input: The password has been used before, please change another one. +SELECT SLEEP(1); +SLEEP(1) +0 +SELECT user_name FROM mo_catalog.mo_user WHERE user_name IN ('user1_group2', 'user2_group2'); +user_name +user1_group2 +user2_group2 +DROP USER IF EXISTS user1_group2, user2_group2; +SET GLOBAL connection_control_failed_connections_threshold = 10; +SET GLOBAL connection_control_max_connection_delay = 2147483647; +SET GLOBAL password_history = 10; +SET GLOBAL password_reuse_interval = 30; +SHOW VARIABLES LIKE "%connection_control_failed_connections_threshold%"; +Variable_name Value +connection_control_failed_connections_threshold 10 +SHOW VARIABLES LIKE "%connection_control_max_connection_delay%"; +Variable_name Value +connection_control_max_connection_delay 2147483647 +SHOW VARIABLES LIKE "%password_history%"; +Variable_name Value +password_history 10 +SHOW VARIABLES LIKE "%password_reuse_interval%"; +Variable_name Value +password_reuse_interval 30 +CREATE USER user1_group3 IDENTIFIED BY 'password123'; +CREATE USER user2_group3 IDENTIFIED BY 'password456'; +DROP USER IF EXISTS user1_group3, user2_group3; +SET GLOBAL connection_control_failed_connections_threshold = 0; +SET GLOBAL connection_control_max_connection_delay = 1000; +SET GLOBAL password_history = 1; +SET GLOBAL password_reuse_interval = 7; +SHOW VARIABLES LIKE "%connection_control_failed_connections_threshold%"; +Variable_name Value +connection_control_failed_connections_threshold 0 +SHOW VARIABLES LIKE "%connection_control_max_connection_delay%"; +Variable_name Value +connection_control_max_connection_delay 1000 +SHOW VARIABLES LIKE "%password_history%"; +Variable_name Value +password_history 1 +SHOW VARIABLES LIKE "%password_reuse_interval%"; +Variable_name Value +password_reuse_interval 7 +CREATE USER user1_group4 IDENTIFIED BY 'oldpassword123'; +CREATE USER user2_group4 IDENTIFIED BY 'oldpassword456'; +ALTER USER user1_group4 IDENTIFIED BY 'newpassword123'; +ALTER USER user2_group4 IDENTIFIED BY 'newpassword456'; +ALTER USER user1_group4 IDENTIFIED BY 'oldpassword123'; -- Expected failure: Old password cannot be reused +invalid input: The password has been used before, please change another one +ALTER USER user2_group4 IDENTIFIED BY 'oldpassword456'; -- Expected failure: Old password cannot be reused +invalid input: The password has been used before, please change another one +SELECT user_name FROM mo_catalog.mo_user WHERE user_name IN ('user1_group4', 'user2_group4'); +user_name +user1_group4 +user2_group4 +DROP USER IF EXISTS user1_group4, user2_group4; +SET GLOBAL connection_control_failed_connections_threshold = -1; -- Invalid: cannot be negative +internal error: convert to the system variable int type failed +SET GLOBAL connection_control_max_connection_delay = -100; -- Invalid: cannot be negative +internal error: convert to the system variable int type failed +SET GLOBAL password_history = -2; -- Invalid: cannot be negative +internal error: convert to the system variable int type failed +SET GLOBAL password_reuse_interval = -5; -- Invalid: cannot be negative +internal error: convert to the system variable int type failed +SET GLOBAL connection_control_failed_connections_threshold = 3; +SET GLOBAL connection_control_max_connection_delay = 0; +SET GLOBAL password_history = 0; +SET GLOBAL password_reuse_interval = 0; diff --git a/test/distributed/cases/security/password.sql b/test/distributed/cases/security/password.sql new file mode 100644 index 0000000000000..e68f37a5a006d --- /dev/null +++ b/test/distributed/cases/security/password.sql @@ -0,0 +1,307 @@ +sion:id=0&user=dump&password=111 +-- Test cases for Group 0 +create user user1_group0 identified by '1234'; +-- Clean up Group 0 +drop user if exists user1_group0; + +set global validate_password = 'ON'; +set global validate_password.check_user_name = 'ON'; +set global validate_password.changed_characters_percentage = 0; +set global validate_password.length = 5; +set global validate_password.mixed_case_count = 0; +set global validate_password.number_count = 0; +set global validate_password.special_char_count = 0; +set global validate_password.policy = '0'; +-- @session + +-- ======================== +-- 1. Test Group 1 +-- ======================== +-- @session:id=1&user=dump&password=111 +show variables like "%validate_password%"; + +-- test +create user user1_group1 identified by '1234'; -- fail +create user user2_group1 identified by 'abc'; -- fail +create user user3_group1 identified by '12345'; -- success +create user user4_group1 identified by 'abcde'; -- success + +-- verify +select user_name from mo_catalog.mo_user where user_name in ('user3_group1', 'user4_group1'); +-- Clean up Group 1 +drop user if exists user1_group1, user2_group1, user3_group1, user4_group1; + +set global validate_password = 'ON'; +set global validate_password.check_user_name = 'ON'; +set global validate_password.changed_characters_percentage = 60; +set global validate_password.length = 8; +set global validate_password.mixed_case_count = 2; +set global validate_password.number_count = 2; +set global validate_password.special_char_count = 0; +set global validate_password.policy = '1'; +-- @session + +-- ======================== +-- 2. Test Group 2 +-- ======================== +-- @session:id=2&user=dump&password=111 +show variables like "%validate_password%"; + +-- Test cases +create user user1_group2 identified by '12345678'; -- Expected failure +create user user2_group2 identified by 'abcdefgH'; -- Expected failure +-- @bvt:issue#4511 +create user useR32Go identified by 'oG23Resu'; -- Expected failure +-- @bvt:issue +create user user4_group2 identified by 'AbCLq56%'; -- Expected success + +-- Verify results +select user_name from mo_catalog.mo_user where user_name in ('user3_group2', 'user4_group2'); + +-- Clean up Group 2 +drop user if exists user1_group2, user2_group2, useR32Go, user4_group2; + +-- Set parameters for Group 3 password complexity +set global validate_password = 'ON'; +set global validate_password.check_user_name = 'OFF'; +set global validate_password.changed_characters_percentage = 50; +set global validate_password.length = 30; +set global validate_password.mixed_case_count = 5; +set global validate_password.number_count = 8; +set global validate_password.special_char_count = 2; +set global validate_password.policy = '1'; +-- @session + +-- ======================== +-- 3. Test Group 3 +-- ======================== +-- @session:id=3&user=dump&password=111 +-- Test cases +create user user1_group3 identified by 'abcdefgH'; -- Expected failure +create user user2_group3 identified by '1234abcd'; -- Expected failure +create user user3_group3 identified by 'Abc123KLedjrg563O28d'; -- Expected failure +create user user4_group3 identified by 'Xyz78pLPAbc123JHedjrg563Ukkd_%'; -- Expected success + +-- Verify results +select user_name from mo_catalog.mo_user where user_name in ('user4_group3'); + +-- Clean up Group 3 +drop user if exists user1_group3, user2_group3, user3_group3, user4_group3; + +-- Set parameters for Group 4 password complexity +set global validate_password = 'ON'; +set global validate_password.check_user_name = 'ON'; +set global validate_password.changed_characters_percentage = 40; +set global validate_password.length = 15; +set global validate_password.mixed_case_count = 3; +set global validate_password.number_count = 5; +set global validate_password.special_char_count = 3; +set global validate_password.policy = '1'; +-- @session + +-- ======================== +-- 4. Test Group 4 +-- ======================== +-- @session:id=4&user=dump&password=111 +-- Test cases +create user user1_group4 identified by '123456789abcde'; -- Expected failure +create user user2_group4 identified by 'Abcdefg1234567'; -- Expected failure +create user user3_group4 identified by 'Abc123%_@RMXy78'; -- Expected success + +-- Verify results +select user_name from mo_catalog.mo_user where user_name in ('user3_group4', 'user4_group4'); + +-- Clean up Group 4 +drop user if exists user1_group4, user2_group4, user3_group4, user4_group4; + +-- ======================== +-- Invalid test +-- ======================== +set global validate_password = 'HELLO'; +set global validate_password.check_user_name = 'IN'; +set global validate_password.changed_characters_percentage = 101; +set global validate_password.length = inv; +set global validate_password.mixed_case_count = abc; +set global validate_password.number_count = -3; +set global validate_password.special_char_count = -1; +set global validate_password.policy = 'HIGH'; + +-- ======================== +-- Reset default configurations after all tests +-- ======================== +set global validate_password = 'OFF'; +set global validate_password.check_user_name = 'ON'; +set global validate_password.changed_characters_percentage = 0; +set global validate_password.length = 8; +set global validate_password.mixed_case_count = 1; +set global validate_password.number_count = 1; +set global validate_password.special_char_count = 1; +set global validate_password.policy = '0'; +-- @session + +-- ======================== +-- 1. Set global parameters for password lifecycle and failed connection threshold +-- ======================== +-- @session:id=5&user=dump&password=111 +-- Create dump user and grant privileges to modify other users + +-- Show current variables before setting new values +SHOW VARIABLES LIKE "%connection_control_failed_connections_threshold%"; +SHOW VARIABLES LIKE "%connection_control_max_connection_delay%"; +SHOW VARIABLES LIKE "%password_history%"; +SHOW VARIABLES LIKE "%password_reuse_interval%"; + +-- Set parameters for Group 1 password lifecycle +SET GLOBAL connection_control_failed_connections_threshold = 3; +SET GLOBAL connection_control_max_connection_delay = 5000; +SET GLOBAL password_history = 0; +SET GLOBAL password_reuse_interval = 0; +-- @session + +-- ======================== +-- 2. Verify parameters for Group 1 in a new session +-- ======================== + +-- @session:id=6&user=dump&password=111 +-- Show the current parameter settings after change +SHOW VARIABLES LIKE "%connection_control_failed_connections_threshold%"; +SHOW VARIABLES LIKE "%connection_control_max_connection_delay%"; +SHOW VARIABLES LIKE "%password_history%"; +SHOW VARIABLES LIKE "%password_reuse_interval%"; + +-- Modify test users' passwords using administrator +CREATE USER user1_group1 IDENTIFIED BY 'oldpassword'; +CREATE USER user2_group1 IDENTIFIED BY 'oldpassword2'; + +-- Change user passwords +ALTER USER user1_group1 IDENTIFIED BY 'newpassword1'; +SELECT SLEEP(1); +ALTER USER user2_group1 IDENTIFIED BY 'newpassword2'; +SELECT SLEEP(1); + +-- Verify users' password change status +SELECT user_name FROM mo_catalog.mo_user WHERE user_name IN ('user1_group1', 'user2_group1'); + +-- Clean up Group 1 +DROP USER IF EXISTS user1_group1, user2_group1; + +-- ======================== +-- 3. Test Group 2: Password history and reuse interval +-- ======================== + +-- Set parameters for Group 2 password lifecycle +SET GLOBAL connection_control_failed_connections_threshold = 5; +SET GLOBAL connection_control_max_connection_delay = 1000; +SET GLOBAL password_history = 6; +SET GLOBAL password_reuse_interval = 10; +-- @session + +-- @session:id=7&user=dump&password=111 +-- Show the current parameter settings before test +SHOW VARIABLES LIKE "%connection_control_failed_connections_threshold%"; +SHOW VARIABLES LIKE "%connection_control_max_connection_delay%"; +SHOW VARIABLES LIKE "%password_history%"; +SHOW VARIABLES LIKE "%password_reuse_interval%"; + +-- Modify test users' passwords using administrator +CREATE USER user1_group2 IDENTIFIED BY 'oldpassword1'; +CREATE USER user2_group2 IDENTIFIED BY 'oldpassword2'; + +-- Change user passwords +ALTER USER user1_group2 IDENTIFIED BY 'newpassword1!'; +SELECT SLEEP(1); +ALTER USER user2_group2 IDENTIFIED BY 'newpassword2!'; +SELECT SLEEP(1); + +-- Attempt to reuse old passwords (expected failure) +ALTER USER user1_group2 IDENTIFIED BY 'oldpassword1'; -- Expected failure: Old password cannot be reused +SELECT SLEEP(1); +ALTER USER user2_group2 IDENTIFIED BY 'oldpassword2'; -- Expected failure: Old password cannot be reused +SELECT SLEEP(1); + +-- Verify users' password change status +SELECT user_name FROM mo_catalog.mo_user WHERE user_name IN ('user1_group2', 'user2_group2'); + +-- Clean up Group 2 +DROP USER IF EXISTS user1_group2, user2_group2; + +-- ======================== +-- 4. Test Group 3: Failed connection attempts and lock time +-- ======================== + +-- Set parameters for Group 3 password lifecycle +SET GLOBAL connection_control_failed_connections_threshold = 10; +SET GLOBAL connection_control_max_connection_delay = 2147483647; +SET GLOBAL password_history = 10; +SET GLOBAL password_reuse_interval = 30; +-- @session + +-- @session:id=8&user=dump&password=111 +-- Show the current parameter settings before test +SHOW VARIABLES LIKE "%connection_control_failed_connections_threshold%"; +SHOW VARIABLES LIKE "%connection_control_max_connection_delay%"; +SHOW VARIABLES LIKE "%password_history%"; +SHOW VARIABLES LIKE "%password_reuse_interval%"; + +-- Modify test users' passwords using administrator +CREATE USER user1_group3 IDENTIFIED BY 'password123'; +CREATE USER user2_group3 IDENTIFIED BY 'password456'; + +-- Simulate failed connection attempts (requires client-side simulation) +-- Once failed attempts exceed threshold, verify if user gets locked + +-- Clean up Group 3 +DROP USER IF EXISTS user1_group3, user2_group3; + +-- ======================== +-- 5. Test Group 4: Test advanced settings (lock time and password history) +-- ======================== + +-- Set parameters for Group 4 password lifecycle +SET GLOBAL connection_control_failed_connections_threshold = 0; +SET GLOBAL connection_control_max_connection_delay = 1000; +SET GLOBAL password_history = 1; +SET GLOBAL password_reuse_interval = 7; +-- @session + +-- @session:id=9&user=dump&password=111 +-- Show the current parameter settings before test +SHOW VARIABLES LIKE "%connection_control_failed_connections_threshold%"; +SHOW VARIABLES LIKE "%connection_control_max_connection_delay%"; +SHOW VARIABLES LIKE "%password_history%"; +SHOW VARIABLES LIKE "%password_reuse_interval%"; + +-- Modify test users' passwords using administrator +CREATE USER user1_group4 IDENTIFIED BY 'oldpassword123'; +CREATE USER user2_group4 IDENTIFIED BY 'oldpassword456'; + +-- Change passwords +ALTER USER user1_group4 IDENTIFIED BY 'newpassword123'; +ALTER USER user2_group4 IDENTIFIED BY 'newpassword456'; + +-- Attempt to reuse old passwords (expected failure) +ALTER USER user1_group4 IDENTIFIED BY 'oldpassword123'; -- Expected failure: Old password cannot be reused +ALTER USER user2_group4 IDENTIFIED BY 'oldpassword456'; -- Expected failure: Old password cannot be reused + +-- Verify users' password change status +SELECT user_name FROM mo_catalog.mo_user WHERE user_name IN ('user1_group4', 'user2_group4'); + +-- Clean up Group 4 +DROP USER IF EXISTS user1_group4, user2_group4; +-- ======================== +-- 6. Test Group 5: Invalid values for lifecycle parameters +-- ======================== + +SET GLOBAL connection_control_failed_connections_threshold = -1; -- Invalid: cannot be negative +SET GLOBAL connection_control_max_connection_delay = -100; -- Invalid: cannot be negative +SET GLOBAL password_history = -2; -- Invalid: cannot be negative +SET GLOBAL password_reuse_interval = -5; -- Invalid: cannot be negative +-- ======================== +-- Reset default configurations after all tests +-- ======================== +SET GLOBAL connection_control_failed_connections_threshold = 3; +SET GLOBAL connection_control_max_connection_delay = 0; +SET GLOBAL password_history = 0; +SET GLOBAL password_reuse_interval = 0; +-- @session + From cc9ce43434f1b814bdc42db79abb7a302049ca3e Mon Sep 17 00:00:00 2001 From: iamlinjunhong <49111204+iamlinjunhong@users.noreply.github.com> Date: Wed, 11 Dec 2024 14:31:26 +0800 Subject: [PATCH 19/27] fix filter of window func (#20593) fix filter of window func Approved by: @ouyuanning, @heni02, @aunjgr, @aressu1985 --- pkg/sql/plan/opt_misc.go | 29 +++++++++++- pkg/sql/plan/opt_misc_test.go | 49 +++++++++++++++++++++ pkg/sql/plan/query_builder.go | 28 +++++++----- test/distributed/cases/window/window.result | 10 +++++ test/distributed/cases/window/window.sql | 10 +++++ 5 files changed, 112 insertions(+), 14 deletions(-) create mode 100644 pkg/sql/plan/opt_misc_test.go diff --git a/pkg/sql/plan/opt_misc.go b/pkg/sql/plan/opt_misc.go index 85d51b981ba4d..0d0979c698d77 100644 --- a/pkg/sql/plan/opt_misc.go +++ b/pkg/sql/plan/opt_misc.go @@ -395,20 +395,45 @@ func (builder *QueryBuilder) remapHavingClause(expr *plan.Expr, groupTag, aggreg } } -func (builder *QueryBuilder) remapWindowClause(expr *plan.Expr, windowTag int32, projectionSize int32) { +func (builder *QueryBuilder) remapWindowClause( + expr *plan.Expr, + windowTag int32, + projectionSize int32, + colMap map[[2]int32][2]int32, + remapInfo *RemapInfo, +) error { + // For window functions, + // a specific weight is required mapping switch exprImpl := expr.Expr.(type) { case *plan.Expr_Col: + // In the window function node, + // the filtering conditions also need to be remapped if exprImpl.Col.RelPos == windowTag { exprImpl.Col.Name = builder.nameByColRef[[2]int32{windowTag, exprImpl.Col.ColPos}] exprImpl.Col.RelPos = -1 exprImpl.Col.ColPos += projectionSize + } else { + // normal remap for other columns + // for example, + // where abs(sum(a) - avg(sum(a) over(partition by b)) + // sum(a) need remap + err := builder.remapSingleColRef(exprImpl.Col, colMap, remapInfo) + if err != nil { + return err + } } case *plan.Expr_F: + // loop function parameters for _, arg := range exprImpl.F.Args { - builder.remapWindowClause(arg, windowTag, projectionSize) + err := builder.remapWindowClause(arg, windowTag, projectionSize, colMap, remapInfo) + if err != nil { + return err + } } } + // return nil + return nil } // if join cond is a=b and a=c, we can remove a=c to improve join performance diff --git a/pkg/sql/plan/opt_misc_test.go b/pkg/sql/plan/opt_misc_test.go new file mode 100644 index 0000000000000..d17f83a04f9a5 --- /dev/null +++ b/pkg/sql/plan/opt_misc_test.go @@ -0,0 +1,49 @@ +// Copyright 2023 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 plan + +import ( + "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/pb/plan" + "github.com/stretchr/testify/require" + "testing" +) + +func TestRemapWindowClause(t *testing.T) { + expr := &plan.Expr{ + Typ: plan.Type{Id: int32(types.T_timestamp)}, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{ + RelPos: 3, + ColPos: 3, + Name: "test", + }, + }, + } + + f := &Expr{ + Expr: &plan.Expr_F{ + F: &plan.Function{ + Func: getFunctionObjRef(1, "n"), + Args: []*Expr{expr}, + }, + }, + Typ: plan.Type{}, + } + colMap := make(map[[2]int32][2]int32) + var b *QueryBuilder + err := b.remapWindowClause(f, 1, 1, colMap, nil) + require.Error(t, err) +} diff --git a/pkg/sql/plan/query_builder.go b/pkg/sql/plan/query_builder.go index c57404ce8b4f9..1e30108882175 100644 --- a/pkg/sql/plan/query_builder.go +++ b/pkg/sql/plan/query_builder.go @@ -102,12 +102,6 @@ func (builder *QueryBuilder) remapColRefForExpr(expr *Expr, colMap map[[2]int32] if err != nil { return err } - //for _, arg := range ne.W.PartitionBy { - // err = builder.remapColRefForExpr(arg, colMap) - // if err != nil { - // return err - // } - //} for _, order := range ne.W.OrderBy { err = builder.remapColRefForExpr(order.Expr, colMap, remapInfo) if err != nil { @@ -849,10 +843,6 @@ func (builder *QueryBuilder) remapAllColRefs(nodeID int32, step int32, colRefCnt timeTag := node.BindingTags[0] groupTag := node.BindingTags[1] - for i, expr := range node.FilterList { - builder.remapWindowClause(expr, timeTag, int32(i)) - } - // order by idx := 0 increaseRefCnt(node.OrderBy[0].Expr, -1, colRefCnt) @@ -971,11 +961,13 @@ func (builder *QueryBuilder) remapAllColRefs(nodeID int32, step int32, colRefCnt increaseRefCnt(expr, 1, colRefCnt) } + // remap children node childRemapping, err := builder.remapAllColRefs(node.Children[0], step, colRefCnt, colRefBool, sinkColRef) if err != nil { return nil, err } + // append children projection list childProjList := builder.qry.Nodes[node.Children[0]].ProjectList for i, globalRef := range childRemapping.localToGlobal { if colRefCnt[globalRef] == 0 { @@ -999,10 +991,22 @@ func (builder *QueryBuilder) remapAllColRefs(nodeID int32, step int32, colRefCnt windowTag := node.BindingTags[0] l := len(childProjList) + // In the window function node, + // the filtering conditions also need to be remapped for _, expr := range node.FilterList { - builder.remapWindowClause(expr, windowTag, int32(l)) + // get col pos from remap info + err = builder.remapWindowClause( + expr, + windowTag, + int32(l), + childRemapping.globalToLocal, + &remapInfo) + if err != nil { + return nil, err + } } + // remap all window function remapInfo.tip = "WinSpecList" for idx, expr := range node.WinSpecList { increaseRefCnt(expr, -1, colRefCnt) @@ -1012,7 +1016,7 @@ func (builder *QueryBuilder) remapAllColRefs(nodeID int32, step int32, colRefCnt return nil, err } - globalRef := [2]int32{windowTag, int32(node.GetWindowIdx())} + globalRef := [2]int32{windowTag, node.GetWindowIdx()} if colRefCnt[globalRef] == 0 { continue } diff --git a/test/distributed/cases/window/window.result b/test/distributed/cases/window/window.result index 38a9d4138d622..dfdd665a8f01d 100644 --- a/test/distributed/cases/window/window.result +++ b/test/distributed/cases/window/window.result @@ -4144,6 +4144,16 @@ col2 col1 count(col1) over (partition by col2 order by col1 desc) 老师 100 1 老师 21 2 老师 1 3 +create table item(i_item_sk integer not null,i_manufact_id integer , primary key (i_item_sk)); +INSERT INTO item (i_item_sk, i_manufact_id) VALUES (1, 100); +INSERT INTO item (i_item_sk, i_manufact_id) VALUES (2, 200); +INSERT INTO item (i_item_sk, i_manufact_id) VALUES (3, 300); +create table store_sales( ss_item_sk integer not null, ss_ticket_number integer not null, ss_sales_price decimal(7,2) , primary key (ss_item_sk, ss_ticket_number)); +INSERT INTO store_sales (ss_item_sk, ss_ticket_number, ss_sales_price) VALUES (1, 101, 19.99); +INSERT INTO store_sales (ss_item_sk, ss_ticket_number, ss_sales_price) VALUES (1, 102, 29.99); +INSERT INTO store_sales (ss_item_sk, ss_ticket_number, ss_sales_price) VALUES (2, 201, 39.99); +select * from (select i_manufact_id, sum(ss_sales_price) sum_sales, avg(sum(ss_sales_price)) over (partition by i_manufact_id) avg_quarterly_sales from item, store_sales group by i_manufact_id) tmp1 where abs(sum_sales - avg_quarterly_sales) / avg_quarterly_sales > 0.1 limit 1; +i_manufact_id sum_sales avg_quarterly_sales select group_concat(c.column_name order by ordinal_position) key_columns from information_schema.key_column_usage c where c.table_schema='test1' and c.table_name='region' and constraint_name='PRIMARY'; key_columns drop database test; diff --git a/test/distributed/cases/window/window.sql b/test/distributed/cases/window/window.sql index aa6bbcebdca2d..60194f41d79a2 100644 --- a/test/distributed/cases/window/window.sql +++ b/test/distributed/cases/window/window.sql @@ -1440,6 +1440,16 @@ select col2, col1, min(col1) over (partition by col2 order by col1 desc) from wi select col2, col1, max(col1) over (partition by col2 order by col1 desc) from window01; select col2, col1, count(col1) over (partition by col2 order by col1 desc) from window01; +create table item(i_item_sk integer not null,i_manufact_id integer , primary key (i_item_sk)); +INSERT INTO item (i_item_sk, i_manufact_id) VALUES (1, 100); +INSERT INTO item (i_item_sk, i_manufact_id) VALUES (2, 200); +INSERT INTO item (i_item_sk, i_manufact_id) VALUES (3, 300); +create table store_sales( ss_item_sk integer not null, ss_ticket_number integer not null, ss_sales_price decimal(7,2) , primary key (ss_item_sk, ss_ticket_number)); +INSERT INTO store_sales (ss_item_sk, ss_ticket_number, ss_sales_price) VALUES (1, 101, 19.99); +INSERT INTO store_sales (ss_item_sk, ss_ticket_number, ss_sales_price) VALUES (1, 102, 29.99); +INSERT INTO store_sales (ss_item_sk, ss_ticket_number, ss_sales_price) VALUES (2, 201, 39.99); +select * from (select i_manufact_id, sum(ss_sales_price) sum_sales, avg(sum(ss_sales_price)) over (partition by i_manufact_id) avg_quarterly_sales from item, store_sales group by i_manufact_id) tmp1 where abs(sum_sales - avg_quarterly_sales) / avg_quarterly_sales > 0.1 limit 1; + -- information_schema is now a table which is compatible with mysql, it is now an empty table select group_concat(c.column_name order by ordinal_position) key_columns from information_schema.key_column_usage c where c.table_schema='test1' and c.table_name='region' and constraint_name='PRIMARY'; drop database test; From 251c2707153ed53a5cb53420fd974e58c64decca Mon Sep 17 00:00:00 2001 From: nitao Date: Wed, 11 Dec 2024 15:30:19 +0800 Subject: [PATCH 20/27] fix a bug for case when expression (#20711) fix a bug for case when expression Approved by: @m-schen, @heni02, @aressu1985 --- pkg/sql/plan/function/func_compare.go | 6 +++--- .../cases/expression/case_when.result | 16 +++++++++++++++- test/distributed/cases/expression/case_when.sql | 7 ++++++- 3 files changed, 24 insertions(+), 5 deletions(-) diff --git a/pkg/sql/plan/function/func_compare.go b/pkg/sql/plan/function/func_compare.go index 69f22ff1d8ab7..818ba1dd6f501 100644 --- a/pkg/sql/plan/function/func_compare.go +++ b/pkg/sql/plan/function/func_compare.go @@ -135,14 +135,14 @@ func equalFn(parameters []*vector.Vector, result vector.FunctionResultWrapper, p return a == b }, selectList) case types.T_char, types.T_varchar, types.T_blob, types.T_json, types.T_text, types.T_binary, types.T_varbinary, types.T_datalink: - if parameters[0].GetArea() == nil && parameters[1].GetArea() == nil { + if parameters[0].GetArea() == nil && parameters[1].GetArea() == nil && (selectList == nil) { return compareVarlenaEqual(parameters, rs, proc, length, selectList) } return opBinaryStrStrToFixed[bool](parameters, rs, proc, length, func(v1, v2 string) bool { return v1 == v2 }, selectList) case types.T_array_float32: - if parameters[0].GetArea() == nil && parameters[1].GetArea() == nil { + if parameters[0].GetArea() == nil && parameters[1].GetArea() == nil && (selectList == nil) { return compareVarlenaEqual(parameters, rs, proc, length, selectList) } return opBinaryBytesBytesToFixed[bool](parameters, rs, proc, length, func(v1, v2 []byte) bool { @@ -152,7 +152,7 @@ func equalFn(parameters []*vector.Vector, result vector.FunctionResultWrapper, p return moarray.Compare[float32](_v1, _v2) == 0 }, selectList) case types.T_array_float64: - if parameters[0].GetArea() == nil && parameters[1].GetArea() == nil { + if parameters[0].GetArea() == nil && parameters[1].GetArea() == nil && (selectList == nil) { return compareVarlenaEqual(parameters, rs, proc, length, selectList) } return opBinaryBytesBytesToFixed[bool](parameters, rs, proc, length, func(v1, v2 []byte) bool { diff --git a/test/distributed/cases/expression/case_when.result b/test/distributed/cases/expression/case_when.result index 3b67db786a16f..7361a350af331 100755 --- a/test/distributed/cases/expression/case_when.result +++ b/test/distributed/cases/expression/case_when.result @@ -86,6 +86,14 @@ nothing 2 drop table if exists t1; create table t1 (`row` int not null, col int not null, val varchar(255) not null); insert into t1 values (1,1,'orange'),(1,2,'large'),(2,1,'yellow'),(2,2,'medium'),(3,1,'green'),(3,2,'small'); +select col,val, case when val="orange" then 1 when upper(val)="LARGE" then 2 else 3 end from t1; +col val case when val = orange then 1 when upper(val) = LARGE then 2 else 3 end +1 orange 1 +2 large 2 +1 yellow 3 +2 medium 3 +1 green 3 +2 small 3 select max(case col when 1 then val else null end) as color from t1 group by `row`; color orange @@ -219,4 +227,10 @@ id xxx 18 3 19 3 20 3 -DROP TABLE t1; \ No newline at end of file +DROP TABLE t1; +create table t1(a varchar(100)); +insert into t1 values ("a"); +select a, case when a="a" then 1 when upper(a)="b" then 2 end from t1; +a case when a = a then 1 when upper(a) = b then 2 end +a 1 +drop table if exists t1; \ No newline at end of file diff --git a/test/distributed/cases/expression/case_when.sql b/test/distributed/cases/expression/case_when.sql index 1ab3bc9d90f47..068a697067ffc 100755 --- a/test/distributed/cases/expression/case_when.sql +++ b/test/distributed/cases/expression/case_when.sql @@ -60,6 +60,7 @@ drop table if exists t1; -- @label:bvt create table t1 (`row` int not null, col int not null, val varchar(255) not null); insert into t1 values (1,1,'orange'),(1,2,'large'),(2,1,'yellow'),(2,2,'medium'),(3,1,'green'),(3,2,'small'); +select col,val, case when val="orange" then 1 when upper(val)="LARGE" then 2 else 3 end from t1; select max(case col when 1 then val else null end) as color from t1 group by `row`; drop table if exists t1; @@ -164,4 +165,8 @@ insert into t1 values (19, 7986, '1771-12-06'), (20, 7987, '1985-10-06'); select id, case when id < 5 then 0 when id < 10 then 1 when id < 15 then 2 when true then 3 else -1 end as xxx from t1; -DROP TABLE t1; \ No newline at end of file +DROP TABLE t1; +create table t1(a varchar(100)); +insert into t1 values ("a"); +select a, case when a="a" then 1 when upper(a)="b" then 2 end from t1; +drop table if exists t1; \ No newline at end of file From 16e9b56b95ea17769e10863e137be479467faff6 Mon Sep 17 00:00:00 2001 From: reusee Date: Wed, 11 Dec 2024 16:31:34 +0800 Subject: [PATCH 21/27] fileservice: more disk cache events logging (#20707) add more logs for disk cache Approved by: @fengttt --- pkg/fileservice/cache.go | 1 + pkg/fileservice/disk_cache.go | 31 +++++++++- pkg/fileservice/io_entry.go | 17 +++++- pkg/fileservice/local_fs.go | 2 +- pkg/fileservice/string.go | 111 +++++++++++++++++++++------------- 5 files changed, 112 insertions(+), 50 deletions(-) diff --git a/pkg/fileservice/cache.go b/pkg/fileservice/cache.go index 0937b788580e4..f48e7cc13d1e3 100644 --- a/pkg/fileservice/cache.go +++ b/pkg/fileservice/cache.go @@ -163,6 +163,7 @@ func readCache(ctx context.Context, cache IOVectorCache, vector *IOVector) error if err != nil { if errors.Is(err, context.DeadlineExceeded) { + LogEvent(ctx, str_read_cache_exceed_deadline) err = moerr.AttachCause(ctx, err) logutil.Warn("cache read exceed deadline", zap.Any("err", err), diff --git a/pkg/fileservice/disk_cache.go b/pkg/fileservice/disk_cache.go index d7fdedf57b8c1..cff7b09b760b7 100644 --- a/pkg/fileservice/disk_cache.go +++ b/pkg/fileservice/disk_cache.go @@ -246,6 +246,8 @@ func (d *DiskCache) Read( var numHit, numRead, numOpenIOEntry, numOpenFull, numError int64 defer func() { + LogEvent(ctx, str_update_metrics_begin) + metric.FSReadHitDiskCounter.Add(float64(numHit)) metric.FSReadReadDiskCounter.Add(float64(numRead)) perfcounter.Update(ctx, func(c *perfcounter.CounterSet) { @@ -257,6 +259,8 @@ func (d *DiskCache) Read( c.FileService.Cache.Disk.OpenIOEntryFile.Add(numOpenIOEntry) c.FileService.Cache.Disk.OpenFullFile.Add(numOpenFull) }, d.perfCounterSets...) + + LogEvent(ctx, str_update_metrics_end) }() path, err := ParsePath(vector.FilePath) @@ -266,12 +270,17 @@ func (d *DiskCache) Read( openedFiles := make(map[string]*os.File) defer func() { + LogEvent(ctx, str_close_disk_files_begin) for _, file := range openedFiles { _ = file.Close() } + LogEvent(ctx, str_close_disk_files_end) }() fillEntry := func(entry *IOEntry) error { + LogEvent(ctx, str_disk_cache_fill_entry_begin) + defer LogEvent(ctx, str_disk_cache_fill_entry_end) + if entry.done { return nil } @@ -288,14 +297,18 @@ func (d *DiskCache) Read( diskPath := d.pathForIOEntry(path.File, *entry) if f, ok := openedFiles[diskPath]; ok { // use opened file + LogEvent(ctx, str_disk_cache_file_seek_begin) _, err = file.Seek(entry.Offset, io.SeekStart) + LogEvent(ctx, str_disk_cache_file_seek_end) if err == nil { file = f } } else { // open file - d.waitUpdateComplete(diskPath) + d.waitUpdateComplete(ctx, diskPath) + LogEvent(ctx, str_disk_cache_file_open_begin) diskFile, err := os.Open(diskPath) + LogEvent(ctx, str_disk_cache_file_open_end) if err == nil { file = diskFile defer func() { @@ -310,21 +323,27 @@ func (d *DiskCache) Read( diskPath = d.pathForFile(path.File) if f, ok := openedFiles[diskPath]; ok { // use opened file + LogEvent(ctx, str_disk_cache_file_seek_begin) _, err = f.Seek(entry.Offset, io.SeekStart) + LogEvent(ctx, str_disk_cache_file_seek_end) if err == nil { file = f } } else { // open file - d.waitUpdateComplete(diskPath) + d.waitUpdateComplete(ctx, diskPath) + LogEvent(ctx, str_disk_cache_file_open_begin) diskFile, err := os.Open(diskPath) + LogEvent(ctx, str_disk_cache_file_open_end) if err == nil { defer func() { openedFiles[diskPath] = diskFile }() numOpenFull++ // seek + LogEvent(ctx, str_disk_cache_file_seek_begin) _, err = diskFile.Seek(entry.Offset, io.SeekStart) + LogEvent(ctx, str_disk_cache_file_seek_end) if err == nil { file = diskFile } @@ -337,14 +356,18 @@ func (d *DiskCache) Read( return nil } + LogEvent(ctx, str_disk_cache_update_states_begin) if _, ok := d.cache.Get(ctx, diskPath); !ok { // set cache + LogEvent(ctx, str_disk_cache_file_stat_begin) stat, err := file.Stat() + LogEvent(ctx, str_disk_cache_file_stat_end) if err != nil { return err } d.cache.Set(ctx, diskPath, struct{}{}, fileSize(stat)) } + LogEvent(ctx, str_disk_cache_update_states_end) if err := entry.ReadFromOSFile(ctx, file, d.cacheDataAllocator); err != nil { return err @@ -589,7 +612,9 @@ func (d *DiskCache) decodeFilePath(diskPath string) (string, error) { return fromOSPath(path), nil } -func (d *DiskCache) waitUpdateComplete(path string) { +func (d *DiskCache) waitUpdateComplete(ctx context.Context, path string) { + LogEvent(ctx, str_disk_cache_wait_update_complete_begin) + defer LogEvent(ctx, str_disk_cache_wait_update_complete_end) d.updatingPaths.L.Lock() for d.updatingPaths.m[path] { d.updatingPaths.Wait() diff --git a/pkg/fileservice/io_entry.go b/pkg/fileservice/io_entry.go index 314baab69d38d..961738193d795 100644 --- a/pkg/fileservice/io_entry.go +++ b/pkg/fileservice/io_entry.go @@ -53,10 +53,16 @@ func (i *IOEntry) setCachedData(ctx context.Context, allocator CacheDataAllocato } func (i *IOEntry) ReadFromOSFile(ctx context.Context, file *os.File, allocator CacheDataAllocator) (err error) { - finally := i.prepareData() + LogEvent(ctx, str_ReadFromOSFile_begin) + defer LogEvent(ctx, str_ReadFromOSFile_end) + + finally := i.prepareData(ctx) defer finally(&err) + r := io.LimitReader(file, i.Size) + LogEvent(ctx, str_io_readfull_begin) n, err := io.ReadFull(r, i.Data) + LogEvent(ctx, str_io_readfull_end) if err != nil { return err } @@ -65,7 +71,10 @@ func (i *IOEntry) ReadFromOSFile(ctx context.Context, file *os.File, allocator C } if i.WriterForRead != nil { - if _, err := i.WriterForRead.Write(i.Data); err != nil { + LogEvent(ctx, str_WriterForRead_Write_begin) + _, err := i.WriterForRead.Write(i.Data) + LogEvent(ctx, str_WriterForRead_Write_end) + if err != nil { return err } } @@ -92,7 +101,9 @@ func CacheOriginalData(ctx context.Context, r io.Reader, data []byte, allocator return } -func (i *IOEntry) prepareData() (finally func(err *error)) { +func (i *IOEntry) prepareData(ctx context.Context) (finally func(err *error)) { + LogEvent(ctx, str_prepareData_begin) + defer LogEvent(ctx, str_prepareData_end) if cap(i.Data) < int(i.Size) { slice, dec, err := ioAllocator().Allocate(uint64(i.Size), malloc.NoHints) if err != nil { diff --git a/pkg/fileservice/local_fs.go b/pkg/fileservice/local_fs.go index 09f93abc95d6c..3e7ce2138df07 100644 --- a/pkg/fileservice/local_fs.go +++ b/pkg/fileservice/local_fs.go @@ -609,7 +609,7 @@ func (l *LocalFS) read(ctx context.Context, vector *IOVector, bytesCounter *atom entry.Size = int64(len(data)) } else { - finally := entry.prepareData() + finally := entry.prepareData(ctx) defer finally(&err) var n int n, err = io.ReadFull(r, entry.Data) diff --git a/pkg/fileservice/string.go b/pkg/fileservice/string.go index c44282c926116..897894fac3e8b 100644 --- a/pkg/fileservice/string.go +++ b/pkg/fileservice/string.go @@ -15,49 +15,74 @@ package fileservice var ( - str_s3fs_read = internString("S3FS.Read") - str_read_return = internString("Read return") - str_ioMerger_Merge_begin = internString("ioMerger.Merge begin") - str_ioMerger_Merge_end = internString("ioMerger.Merge end") - str_ioMerger_Merge_initiate = internString("ioMerger.Merge initiate") - str_ioMerger_Merge_wait = internString("ioMerger.Merge wait") - str_read_vector_Caches_begin = internString("read vector.Caches begin") - str_read_vector_Caches_end = internString("read vector.Caches end") - str_update_vector_Caches_begin = internString("update vector.Caches begin") - str_update_vector_Caches_end = internString("update vector.Caches end") - str_read_memory_cache_Caches_begin = internString("read memory cache begin") - str_read_memory_cache_Caches_end = internString("read memory cache end") - str_update_memory_cache_Caches_begin = internString("update memory cache begin") - str_update_memory_cache_Caches_end = internString("update memory cache end") - str_read_disk_cache_Caches_begin = internString("read disk cache begin") - str_read_disk_cache_Caches_end = internString("read disk cache end") - str_update_disk_cache_Caches_begin = internString("update disk cache begin") - str_update_disk_cache_Caches_end = internString("update disk cache end") - str_read_remote_cache_Caches_begin = internString("read remote cache begin") - str_read_remote_cache_Caches_end = internString("read remote cache end") - str_get_reader_begin = internString("getReader begin") - str_get_reader_end = internString("getReader end") - str_reader_close = internString("reader close") - str_get_content_begin = internString("getContent begin") - str_get_content_end = internString("getContent end") - str_io_readall_begin = internString("io.ReadAll begin") - str_io_readall_end = internString("io.ReadAll end") - str_get_data_begin = internString("getData begin") - str_get_data_end = internString("getData end") - str_write_writerforread_begin = internString("write WriterForRead begin") - str_write_writerforread_end = internString("write WriterForRead end") - str_io_copybuffer_begin = internString("io.CopyBuffer begin") - str_io_copybuffer_end = internString("io.CopyBuffer end") - str_disk_cache_setfile_begin = internString("disk cache SetFile begin") - str_disk_cache_setfile_end = internString("disk cache SetFile end") - str_retryable_reader_new_reader_begin = internString("retryable reader new reader begin") - str_retryable_reader_new_reader_end = internString("retryable reader new reader end") - str_awssdkv2_get_object_begin = internString("AwsSDKv2 GetObject begin") - str_awssdkv2_get_object_end = internString("AwsSDKv2 GetObject end") - str_set_cache_data_begin = internString("setCacheData begin") - str_set_cache_data_end = internString("setCacheData end") - str_to_cache_data_begin = internString("ToCacheData begin") - str_to_cache_data_end = internString("ToCacheData end") + str_s3fs_read = internString("S3FS.Read") + str_read_return = internString("Read return") + str_ioMerger_Merge_begin = internString("ioMerger.Merge begin") + str_ioMerger_Merge_end = internString("ioMerger.Merge end") + str_ioMerger_Merge_initiate = internString("ioMerger.Merge initiate") + str_ioMerger_Merge_wait = internString("ioMerger.Merge wait") + str_read_vector_Caches_begin = internString("read vector.Caches begin") + str_read_vector_Caches_end = internString("read vector.Caches end") + str_update_vector_Caches_begin = internString("update vector.Caches begin") + str_update_vector_Caches_end = internString("update vector.Caches end") + str_read_memory_cache_Caches_begin = internString("read memory cache begin") + str_read_memory_cache_Caches_end = internString("read memory cache end") + str_update_memory_cache_Caches_begin = internString("update memory cache begin") + str_update_memory_cache_Caches_end = internString("update memory cache end") + str_read_disk_cache_Caches_begin = internString("read disk cache begin") + str_read_disk_cache_Caches_end = internString("read disk cache end") + str_update_disk_cache_Caches_begin = internString("update disk cache begin") + str_update_disk_cache_Caches_end = internString("update disk cache end") + str_read_remote_cache_Caches_begin = internString("read remote cache begin") + str_read_remote_cache_Caches_end = internString("read remote cache end") + str_get_reader_begin = internString("getReader begin") + str_get_reader_end = internString("getReader end") + str_reader_close = internString("reader close") + str_get_content_begin = internString("getContent begin") + str_get_content_end = internString("getContent end") + str_io_readall_begin = internString("io.ReadAll begin") + str_io_readall_end = internString("io.ReadAll end") + str_io_readfull_begin = internString("io.ReadFull begin") + str_io_readfull_end = internString("io.ReadFull end") + str_get_data_begin = internString("getData begin") + str_get_data_end = internString("getData end") + str_write_writerforread_begin = internString("write WriterForRead begin") + str_write_writerforread_end = internString("write WriterForRead end") + str_io_copybuffer_begin = internString("io.CopyBuffer begin") + str_io_copybuffer_end = internString("io.CopyBuffer end") + str_disk_cache_setfile_begin = internString("disk cache SetFile begin") + str_disk_cache_setfile_end = internString("disk cache SetFile end") + str_retryable_reader_new_reader_begin = internString("retryable reader new reader begin") + str_retryable_reader_new_reader_end = internString("retryable reader new reader end") + str_awssdkv2_get_object_begin = internString("AwsSDKv2 GetObject begin") + str_awssdkv2_get_object_end = internString("AwsSDKv2 GetObject end") + str_set_cache_data_begin = internString("setCacheData begin") + str_set_cache_data_end = internString("setCacheData end") + str_to_cache_data_begin = internString("ToCacheData begin") + str_to_cache_data_end = internString("ToCacheData end") + str_read_cache_exceed_deadline = internString("readCache exceed deadline") + str_update_metrics_begin = internString("update metrics begin") + str_update_metrics_end = internString("update metrics end") + str_close_disk_files_begin = internString("close disk files begin") + str_close_disk_files_end = internString("close disk files end") + str_disk_cache_fill_entry_begin = internString("disk cache fill entry begin") + str_disk_cache_fill_entry_end = internString("disk cache fill entry end") + str_disk_cache_file_seek_begin = internString("disk cache file seek begin") + str_disk_cache_file_seek_end = internString("disk cache file seek end") + str_disk_cache_wait_update_complete_begin = internString("disk cache wait update complete begin") + str_disk_cache_wait_update_complete_end = internString("disk cache wait update complete end") + str_disk_cache_file_open_begin = internString("disk cache file open begin") + str_disk_cache_file_open_end = internString("disk cache file open end") + str_disk_cache_file_stat_begin = internString("disk cache file stat begin") + str_disk_cache_file_stat_end = internString("disk cache file stat end") + str_disk_cache_update_states_begin = internString("disk cache update states begin") + str_disk_cache_update_states_end = internString("disk cache update states end") + str_ReadFromOSFile_begin = internString("ReadFromOSFile begin") + str_ReadFromOSFile_end = internString("ReadFromOSFile end") + str_prepareData_begin = internString("prepareData begin") + str_prepareData_end = internString("prepareData end") + str_WriterForRead_Write_begin = internString("WriterForRead.Write begin") + str_WriterForRead_Write_end = internString("WriterForRead.Write end") ) type stringRef struct { From 5d82985900d69743697807d3e57d3372c3f11fbf Mon Sep 17 00:00:00 2001 From: nitao Date: Wed, 11 Dec 2024 17:55:59 +0800 Subject: [PATCH 22/27] optimize stats for runtime filters when table not flushed (#20717) optimize stats for runtime filters when table not flushed Approved by: @ouyuanning --- pkg/sql/plan/runtime_filter.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/pkg/sql/plan/runtime_filter.go b/pkg/sql/plan/runtime_filter.go index fdfddfc0e5266..7c46884d336ae 100644 --- a/pkg/sql/plan/runtime_filter.go +++ b/pkg/sql/plan/runtime_filter.go @@ -163,7 +163,11 @@ func (builder *QueryBuilder) generateRuntimeFilters(nodeID int32) { sortOrder := GetSortOrder(tableDef, probeCol.ColPos) if node.JoinType != plan.Node_INDEX { probeNdv := getExprNdv(probeExprs[0], builder) - if probeNdv == -1 || node.Stats.HashmapStats.HashmapSize/probeNdv >= 0.1 { + if probeNdv <= 1 { + //maybe not flushed yet, set at least 100 to continue calculation + probeNdv = 100 + } + if node.Stats.HashmapStats.HashmapSize/probeNdv >= 0.1 { return } if sortOrder != 0 { From fb2e673a66d55db30758b8497e227252b0ad26b1 Mon Sep 17 00:00:00 2001 From: LiuBo Date: Thu, 12 Dec 2024 16:43:19 +0800 Subject: [PATCH 23/27] [improvement] logservice: add some logs (#20705) add some logs to investigate issues Approved by: @zhangxu19830126 --- pkg/logservice/service_bootstrap.go | 4 +++ pkg/logservice/service_bootstrap_test.go | 33 ++++++++++++++++++++++++ 2 files changed, 37 insertions(+) diff --git a/pkg/logservice/service_bootstrap.go b/pkg/logservice/service_bootstrap.go index 0d572a5eb16fb..08bc1766897b5 100644 --- a/pkg/logservice/service_bootstrap.go +++ b/pkg/logservice/service_bootstrap.go @@ -100,13 +100,17 @@ func (s *Service) BootstrapHAKeeper(ctx context.Context, cfg Config) error { return err } } + } else { + s.runtime.SubLogger(runtime.SystemInit).Info("backup is nil") } for i := 0; i < checkBootstrapCycles; i++ { select { case <-ctx.Done(): + s.runtime.SubLogger(runtime.SystemInit).Error("context error", zap.Error(ctx.Err())) return nil default: } + s.runtime.SubLogger(runtime.SystemInit).Info("before initial cluster info") if err := s.store.setInitialClusterInfo( numOfLogShards, numOfTNShards, diff --git a/pkg/logservice/service_bootstrap_test.go b/pkg/logservice/service_bootstrap_test.go index 2608d975bddb3..56f551168dd60 100644 --- a/pkg/logservice/service_bootstrap_test.go +++ b/pkg/logservice/service_bootstrap_test.go @@ -17,7 +17,9 @@ package logservice import ( "context" "path" + "strings" "testing" + "time" "github.com/matrixorigin/matrixone/pkg/defines" "github.com/matrixorigin/matrixone/pkg/fileservice" @@ -77,3 +79,34 @@ func TestGetBackupData(t *testing.T) { assert.Equal(t, nextID, restore.NextID) assert.Equal(t, nextIDByKey, restore.NextIDByKey) } + +func TestServiceBootstrap(t *testing.T) { + t.Run("ok", func(t *testing.T) { + fn := func(t *testing.T, s *Service) { + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + assert.Greater(t, len(s.cfg.BootstrapConfig.InitHAKeeperMembers), 0) + member := s.cfg.BootstrapConfig.InitHAKeeperMembers[0] + parts := strings.Split(member, ":") + assert.Equal(t, 2, len(parts)) + s.cfg.UUID = parts[1] + assert.NoError(t, s.BootstrapHAKeeper(ctx, s.cfg)) + } + runServiceTest(t, false, false, fn) + }) + + t.Run("context cancelled", func(t *testing.T) { + fn := func(t *testing.T, s *Service) { + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + cancel() + assert.Greater(t, len(s.cfg.BootstrapConfig.InitHAKeeperMembers), 0) + member := s.cfg.BootstrapConfig.InitHAKeeperMembers[0] + parts := strings.Split(member, ":") + assert.Equal(t, 2, len(parts)) + s.cfg.UUID = parts[1] + s.cfg.BootstrapConfig.Restore.FilePath = "" + assert.NoError(t, s.BootstrapHAKeeper(ctx, s.cfg)) + } + runServiceTest(t, false, false, fn) + }) +} From f02599dc784ddc68cc21956eae731e641332d95b Mon Sep 17 00:00:00 2001 From: aptend <49832303+aptend@users.noreply.github.com> Date: Thu, 12 Dec 2024 17:57:43 +0800 Subject: [PATCH 24/27] fix bad write format error - main (#20715) - Rollback a large statement will mess up `txn.writes` in dumpDeleteBatchLocked Approved by: @XuPeng-SH --- pkg/catalog/tuplesParse.go | 4 +-- pkg/vm/engine/disttae/txn.go | 4 --- pkg/vm/engine/test/testutil/types.go | 3 +- pkg/vm/engine/test/testutil/util.go | 11 +++--- pkg/vm/engine/test/workspace_test.go | 51 ++++++++++++++++++++++++++++ 5 files changed, 60 insertions(+), 13 deletions(-) diff --git a/pkg/catalog/tuplesParse.go b/pkg/catalog/tuplesParse.go index b932719959418..fdd1da4babbba 100644 --- a/pkg/catalog/tuplesParse.go +++ b/pkg/catalog/tuplesParse.go @@ -105,9 +105,9 @@ func ParseEntryList(es []*api.Entry) (any, []*api.Entry, error) { bat, _ := batch.ProtoBatchToBatch(e.Bat) batstr := "" if bat != nil { - batstr = common.MoBatchToString(bat, 20) + batstr = common.MoBatchToString(bat, 5) } - return nil, nil, moerr.NewInternalErrorNoCtxf("bad write format %q, %s", e.EntryType, batstr) + return nil, nil, moerr.NewInternalErrorNoCtxf("bad write format %q, %v, batch %s", e.EntryType, len(es), batstr) } if e.EntryType == api.Entry_Alter { bat, err := batch.ProtoBatchToBatch(e.Bat) diff --git a/pkg/vm/engine/disttae/txn.go b/pkg/vm/engine/disttae/txn.go index 998bc5279bf84..d196f86a46fe0 100644 --- a/pkg/vm/engine/disttae/txn.go +++ b/pkg/vm/engine/disttae/txn.go @@ -668,10 +668,6 @@ func (txn *Transaction) dumpDeleteBatchLocked(ctx context.Context, offset int, s } } - if deleteCnt < txn.engine.config.insertEntryMaxCount { - return nil - } - txn.writes = writes[:lastWritesIndex] for tbKey := range mp { diff --git a/pkg/vm/engine/test/testutil/types.go b/pkg/vm/engine/test/testutil/types.go index e2eeb37c0d2d0..ac430c57d8d55 100644 --- a/pkg/vm/engine/test/testutil/types.go +++ b/pkg/vm/engine/test/testutil/types.go @@ -17,6 +17,7 @@ package testutil import ( "context" "fmt" + "path/filepath" "time" "github.com/matrixorigin/matrixone/pkg/container/batch" @@ -25,7 +26,6 @@ import ( "github.com/matrixorigin/matrixone/pkg/perfcounter" "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae/logtailreplay" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/options" - "path/filepath" ) type PObjectStats struct { @@ -92,6 +92,7 @@ func (s *PartitionStateStats) String() string { type TestOptions struct { TaeEngineOptions *options.Options Timeout time.Duration + DisttaeOptions []TestDisttaeEngineOptions } func getS3SharedFileServiceOption(ctx context.Context, dir string) (*options.Options, error) { diff --git a/pkg/vm/engine/test/testutil/util.go b/pkg/vm/engine/test/testutil/util.go index fe05832472290..835c4038af131 100644 --- a/pkg/vm/engine/test/testutil/util.go +++ b/pkg/vm/engine/test/testutil/util.go @@ -17,7 +17,6 @@ package testutil import ( "context" "fmt" - "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/options" "os" "os/user" "path" @@ -25,14 +24,12 @@ import ( "testing" "time" - "github.com/matrixorigin/matrixone/pkg/common/moerr" - "github.com/matrixorigin/matrixone/pkg/vm/engine/engine_util" - "golang.org/x/exp/rand" - "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "golang.org/x/exp/rand" catalog2 "github.com/matrixorigin/matrixone/pkg/catalog" + "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/common/mpool" "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/vector" @@ -43,7 +40,9 @@ import ( "github.com/matrixorigin/matrixone/pkg/txn/client" "github.com/matrixorigin/matrixone/pkg/vm/engine" "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae" + "github.com/matrixorigin/matrixone/pkg/vm/engine/engine_util" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/catalog" + "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/options" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/tasks" ) @@ -263,7 +262,7 @@ func InitEnginePack(opts TestOptions, t *testing.T) *EnginePack { t: t, cancelF: cancel, } - pack.D, pack.T, pack.R, pack.Mp = CreateEngines(ctx, opts, t) + pack.D, pack.T, pack.R, pack.Mp = CreateEngines(ctx, opts, t, opts.DisttaeOptions...) timeout := opts.Timeout if timeout == 0 { timeout = 5 * time.Minute diff --git a/pkg/vm/engine/test/workspace_test.go b/pkg/vm/engine/test/workspace_test.go index 7d661afdb953c..284bc8a7b2d8a 100644 --- a/pkg/vm/engine/test/workspace_test.go +++ b/pkg/vm/engine/test/workspace_test.go @@ -1005,6 +1005,57 @@ func Test_BasicRollbackStatementS3(t *testing.T) { require.NoError(t, txn.Commit(ctx)) } +// https://github.com/matrixorigin/MO-Cloud/issues/4602 +func Test_RollbackDeleteAndDrop(t *testing.T) { + opts := config.WithLongScanAndCKPOpts(nil) + p := testutil.InitEnginePack(testutil.TestOptions{ + TaeEngineOptions: opts, + DisttaeOptions: []testutil.TestDisttaeEngineOptions{testutil.WithDisttaeEngineInsertEntryMaxCount(5)}}, + t) + defer p.Close() + + schema := catalog2.MockSchemaAll(10, 1) + schema.Name = "test" + schema2 := catalog2.MockSchemaAll(10, 1) + schema2.Name = "test2" + schema3 := catalog2.MockSchemaAll(10, 1) + schema3.Name = "test3" + txnop := p.StartCNTxn() + + bat := catalog2.MockBatch(schema, 10) + _, rels := p.CreateDBAndTables(txnop, "db", schema, schema2, schema3) + require.NoError(t, rels[2].Write(p.Ctx, containers.ToCNBatch(bat))) + require.NoError(t, txnop.Commit(p.Ctx)) + + v, ok := runtime.ServiceRuntime("").GetGlobalVariables(runtime.InternalSQLExecutor) + if !ok { + panic(fmt.Sprintf("missing sql executor in service %q", "")) + } + txnop = p.StartCNTxn() + exec := v.(executor.SQLExecutor) + execopts := executor.Options{}.WithTxn(txnop).WithDisableIncrStatement() + txnop.GetWorkspace().StartStatement() + txnop.GetWorkspace().IncrStatementID(p.Ctx, false) + dropTable := func() { + _, err := exec.Exec(p.Ctx, "delete from db.test3 where mock_1 = 0", execopts) + require.NoError(t, err) + p.DeleteTableInDB(txnop, "db", "test") + p.DeleteTableInDB(txnop, "db", "test2") + _, err = exec.Exec(p.Ctx, "delete from db.test3 where mock_1 = 2", execopts) + require.NoError(t, err) + } + dropTable() // approximateInMemDeleteCnt = 2 + txnop.GetWorkspace().RollbackLastStatement(p.Ctx) + txnop.GetWorkspace().IncrStatementID(p.Ctx, false) + dropTable() // approximateInMemDeleteCnt = 4 + txnop.GetWorkspace().RollbackLastStatement(p.Ctx) + txnop.GetWorkspace().IncrStatementID(p.Ctx, false) + dropTable() // approximateInMemDeleteCnt = 6 + t.Log(txnop.GetWorkspace().PPString()) + err := txnop.Commit(p.Ctx) // dumpDeleteBatchLocked messes up the writes list and get bad write format error + require.NoError(t, err) +} + // #endregion // #region multi-txn rollback test From a786a717d2cd04493fbce6cae9e452eb9d71c3a8 Mon Sep 17 00:00:00 2001 From: qingxinhome <70939751+qingxinhome@users.noreply.github.com> Date: Thu, 12 Dec 2024 23:09:53 +0800 Subject: [PATCH 25/27] Blob supports a maximum storage capacity of 64M data (#20713) Blob supports a maximum storage capacity of 64M data Approved by: @m-schen, @XuPeng-SH --- pkg/container/types/bytes.go | 8 ++++---- pkg/sql/plan/function/func_unary.go | 3 ++- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/pkg/container/types/bytes.go b/pkg/container/types/bytes.go index a49b9b394830d..da3766afeb686 100644 --- a/pkg/container/types/bytes.go +++ b/pkg/container/types/bytes.go @@ -30,10 +30,10 @@ const ( MaxBinaryLen = 255 MaxEnumLen = 65535 MaxBitLen = 64 - MaxBlobLen = 65535 - MaxVarcharLen = MaxBlobLen - MaxVarBinaryLen = MaxBlobLen - MaxStringSize = MaxBlobLen + MaxBlobLen = 67108864 // 64 MB + MaxStringSize = 65535 // 64 KB + MaxVarcharLen = MaxStringSize + MaxVarBinaryLen = MaxStringSize ) func (v *Varlena) UnsafePtr() unsafe.Pointer { diff --git a/pkg/sql/plan/function/func_unary.go b/pkg/sql/plan/function/func_unary.go index 192890a28938e..77ac15b9570d1 100644 --- a/pkg/sql/plan/function/func_unary.go +++ b/pkg/sql/plan/function/func_unary.go @@ -563,7 +563,8 @@ func LoadFile(ivecs []*vector.Vector, result vector.FunctionResultWrapper, proc if err != nil { return err } - if len(ctx) > 65536 /*blob size*/ { + + if len(ctx) > types.MaxBlobLen /*blob size*/ { return moerr.NewInternalError(proc.Ctx, "Data too long for blob") } if len(ctx) == 0 { From d8ea269ec2d4a8ed2c52626eda96931beb7c5f54 Mon Sep 17 00:00:00 2001 From: Wei Ziran Date: Fri, 13 Dec 2024 00:15:41 +0800 Subject: [PATCH 26/27] prevent cn oom (#20738) limit object count to prevent cn oom Approved by: @XuPeng-SH --- pkg/vm/engine/tae/db/merge/cnScheduler_test.go | 3 +++ pkg/vm/engine/tae/db/merge/executor.go | 5 +++++ pkg/vm/engine/tae/db/merge/policy_test.go | 2 +- 3 files changed, 9 insertions(+), 1 deletion(-) diff --git a/pkg/vm/engine/tae/db/merge/cnScheduler_test.go b/pkg/vm/engine/tae/db/merge/cnScheduler_test.go index 2b002352a7026..1a947dc4492ef 100644 --- a/pkg/vm/engine/tae/db/merge/cnScheduler_test.go +++ b/pkg/vm/engine/tae/db/merge/cnScheduler_test.go @@ -25,6 +25,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/db/dbutils" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/txn/txnbase" "github.com/stretchr/testify/require" + "slices" "testing" "time" ) @@ -108,6 +109,8 @@ func TestExecutorCNMerge(t *testing.T) { require.NotEmpty(t, cnScheduler.activeObjsString()) executor.executeFor(tbl, []*catalog.ObjectEntry{entry}, taskHostCN) + entry2 := newSortedDataEntryWithTableEntry(t, tbl, txn2, 0, 1, overlapSizeThreshold) + executor.executeFor(tbl, slices.Repeat([]*catalog.ObjectEntry{entry2}, 31), taskHostCN) executor.cnSched.prune(0, 0) executor.cnSched.prune(0, time.Hour) diff --git a/pkg/vm/engine/tae/db/merge/executor.go b/pkg/vm/engine/tae/db/merge/executor.go index 97ab940acb148..34ac1fe1a70d2 100644 --- a/pkg/vm/engine/tae/db/merge/executor.go +++ b/pkg/vm/engine/tae/db/merge/executor.go @@ -66,6 +66,11 @@ func (e *executor) executeFor(entry *catalog.TableEntry, objs []*catalog.ObjectE return } + // prevent CN OOM + if len(objs) > 30 { + objs = objs[:30] + } + stats := make([][]byte, 0, len(objs)) cids := make([]common.ID, 0, len(objs)) for _, obj := range objs { diff --git a/pkg/vm/engine/tae/db/merge/policy_test.go b/pkg/vm/engine/tae/db/merge/policy_test.go index dc1eb072de8c7..5b2eeeab29528 100644 --- a/pkg/vm/engine/tae/db/merge/policy_test.go +++ b/pkg/vm/engine/tae/db/merge/policy_test.go @@ -50,7 +50,7 @@ func newSortedDataEntryWithTableEntry(t *testing.T, tbl *catalog.TableEntry, txn zm := index.NewZM(types.T_int32, 0) index.UpdateZM(zm, types.EncodeInt32(&v1)) index.UpdateZM(zm, types.EncodeInt32(&v2)) - stats := objectio.NewObjectStats() + stats := objectio.NewObjectStatsWithObjectID(objectio.NewObjectid(), false, true, false) require.NoError(t, objectio.SetObjectStatsSortKeyZoneMap(stats, zm)) require.NoError(t, objectio.SetObjectStatsOriginSize(stats, size)) require.NoError(t, objectio.SetObjectStatsRowCnt(stats, 2)) From 265fc337c0a4ba5bd4dae28823af5c093f11e873 Mon Sep 17 00:00:00 2001 From: XuPeng-SH Date: Fri, 13 Dec 2024 01:14:06 +0800 Subject: [PATCH 27/27] Improve debug info (#20730) improve debug info Approved by: @LeftHandCold, @jiangxinmeng1 --- pkg/vm/engine/tae/db/checkpoint/replay.go | 49 ++++++++--- pkg/vm/engine/tae/db/db.go | 84 +++++++++++++++---- pkg/vm/engine/tae/db/open.go | 42 +++++----- pkg/vm/engine/tae/db/replay.go | 13 +-- .../logstore/driver/batchstoredriver/store.go | 12 +-- .../driver/logservicedriver/driver.go | 17 ++-- pkg/vm/engine/tae/rpc/handle.go | 33 ++------ pkg/vm/engine/tae/tables/aobj.go | 10 +-- pkg/vm/engine/tae/txn/txnimpl/store.go | 5 +- pkg/vm/engine/tae/txn/txnimpl/txndb.go | 12 ++- 10 files changed, 168 insertions(+), 109 deletions(-) diff --git a/pkg/vm/engine/tae/db/checkpoint/replay.go b/pkg/vm/engine/tae/db/checkpoint/replay.go index 2fff342c1883a..2a31257ff88a8 100644 --- a/pkg/vm/engine/tae/db/checkpoint/replay.go +++ b/pkg/vm/engine/tae/db/checkpoint/replay.go @@ -122,7 +122,11 @@ func (c *CkpReplayer) ReadCkpFiles() (err error) { var checkpointVersion int // in version 1, checkpoint metadata doesn't contain 'version'. vecLen := len(bats[0].Vecs) - logutil.Infof("checkpoint version: %d, list and load duration: %v", vecLen, time.Since(t0)) + logutil.Info( + "Replay-Checkpoint", + zap.Int("col-cnt", vecLen), + zap.Duration("load-cost", time.Since(t0)), + ) if vecLen < CheckpointSchemaColumnCountV1 { checkpointVersion = 1 } else if vecLen < CheckpointSchemaColumnCountV2 { @@ -258,7 +262,7 @@ func (c *CkpReplayer) ReadCkpFiles() (err error) { } // ReplayThreeTablesObjectlist replays the object list the three tables, and check the LSN and TS. -func (c *CkpReplayer) ReplayThreeTablesObjectlist() ( +func (c *CkpReplayer) ReplayThreeTablesObjectlist(phase string) ( maxTs types.TS, maxLSN uint64, isLSNValid bool, @@ -282,9 +286,19 @@ func (c *CkpReplayer) ReplayThreeTablesObjectlist() ( dataFactory := c.dataF maxGlobal := r.MaxGlobalCheckpoint() if maxGlobal != nil { - logutil.Infof("replay checkpoint %v", maxGlobal) err = datas[c.globalCkpIdx].ApplyReplayTo(r.catalog, dataFactory, true) c.applyCount++ + logger := logutil.Info + if err != nil { + logger = logutil.Error + } + logger( + "Replay-3-Table-From-Global", + zap.String("phase", phase), + zap.String("checkpoint", maxGlobal.String()), + zap.Duration("cost", time.Since(t0)), + zap.Error(err), + ) if err != nil { return } @@ -308,6 +322,7 @@ func (c *CkpReplayer) ReplayThreeTablesObjectlist() ( e.String()) } } + logger := logutil.Info for i := 0; i < len(entries); i++ { checkpointEntry := entries[i] if checkpointEntry == nil { @@ -316,8 +331,17 @@ func (c *CkpReplayer) ReplayThreeTablesObjectlist() ( if checkpointEntry.end.LE(&maxTs) { continue } - logutil.Infof("replay checkpoint %v", checkpointEntry) - err = datas[i].ApplyReplayTo(r.catalog, dataFactory, true) + start := time.Now() + if err = datas[i].ApplyReplayTo(r.catalog, dataFactory, true); err != nil { + logger = logutil.Error + } + logger( + "Replay-3-Table-From-Incremental", + zap.String("phase", phase), + zap.String("checkpoint", checkpointEntry.String()), + zap.Duration("cost", time.Since(start)), + zap.Error(err), + ) c.applyCount++ if err != nil { return @@ -342,7 +366,10 @@ func (c *CkpReplayer) ReplayThreeTablesObjectlist() ( return } -func (c *CkpReplayer) ReplayCatalog(readTxn txnif.AsyncTxn) (err error) { +func (c *CkpReplayer) ReplayCatalog( + readTxn txnif.AsyncTxn, + phase string, +) (err error) { start := time.Now() defer func() { @@ -351,8 +378,8 @@ func (c *CkpReplayer) ReplayCatalog(readTxn txnif.AsyncTxn) (err error) { logger = logutil.Error } logger( - "open-tae", - zap.String("replay", "checkpoint-catalog"), + "Replay-Catalog", + zap.String("phase", phase), zap.Duration("cost", time.Since(start)), zap.Error(err), ) @@ -379,7 +406,7 @@ func (c *CkpReplayer) ReplayCatalog(readTxn txnif.AsyncTxn) (err error) { } // ReplayObjectlist replays the data part of the checkpoint. -func (c *CkpReplayer) ReplayObjectlist() (err error) { +func (c *CkpReplayer) ReplayObjectlist(phase string) (err error) { if len(c.ckpEntries) == 0 { return } @@ -425,8 +452,8 @@ func (c *CkpReplayer) ReplayObjectlist() (err error) { r.catalog.GetUsageMemo().(*logtail.TNUsageMemo).PrepareReplay(ckpDatas, ckpVers) r.source.Init(maxTs) logutil.Info( - "open-tae", - zap.String("replay", "checkpoint-objectlist"), + "Replay-Checkpoints", + zap.String("phase", phase), zap.Duration("apply-cost", c.applyDuration), zap.Duration("read-cost", c.readDuration), zap.Int("apply-count", c.applyCount), diff --git a/pkg/vm/engine/tae/db/db.go b/pkg/vm/engine/tae/db/db.go index 26207ea09b355..8f4c1cbdd2c83 100644 --- a/pkg/vm/engine/tae/db/db.go +++ b/pkg/vm/engine/tae/db/db.go @@ -16,12 +16,12 @@ package db import ( "context" - "fmt" "io" "sync/atomic" "time" "github.com/matrixorigin/matrixone/pkg/util/fault" + "go.uber.org/zap" "github.com/matrixorigin/matrixone/pkg/logutil" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/db/dbutils" @@ -121,7 +121,8 @@ func (db *DB) FlushTable( func (db *DB) ForceCheckpoint( ctx context.Context, ts types.TS, - flushDuration time.Duration) (err error) { + flushDuration time.Duration, +) (err error) { // FIXME: cannot disable with a running job db.BGCheckpointRunner.DisableCheckpoint() defer db.BGCheckpointRunner.EnableCheckpoint() @@ -131,7 +132,23 @@ func (db *DB) ForceCheckpoint( } t0 := time.Now() err = db.BGCheckpointRunner.ForceFlush(ts, ctx, flushDuration) - logutil.Infof("[Force Checkpoint] flush takes %v: %v", time.Since(t0), err) + forceFlushCost := time.Since(t0) + + defer func() { + logger := logutil.Info + if err != nil { + logger = logutil.Error + } + logger( + "Control-Force-Checkpoint", + zap.Error(err), + zap.Duration("total-cost", time.Since(t0)), + zap.String("ts", ts.ToString()), + zap.Duration("flush-duration", flushDuration), + zap.Duration("force-flush-cost", forceFlushCost), + ) + }() + if err != nil { return err } @@ -145,7 +162,8 @@ func (db *DB) ForceCheckpoint( for { select { case <-timeout: - return moerr.NewInternalError(ctx, fmt.Sprintf("timeout for: %v", err)) + err = moerr.NewInternalError(ctx, "force checkpoint timeout") + return default: err = db.BGCheckpointRunner.ForceIncrementalCheckpoint(ts, true) if dbutils.IsRetrieableCheckpoint(err) { @@ -154,8 +172,7 @@ func (db *DB) ForceCheckpoint( time.Sleep(time.Duration(interval)) break } - logutil.Debugf("[Force Checkpoint] takes %v", time.Since(t0)) - return err + return } } } @@ -163,21 +180,37 @@ func (db *DB) ForceCheckpoint( func (db *DB) ForceGlobalCheckpoint( ctx context.Context, ts types.TS, - flushDuration, versionInterval time.Duration) (err error) { + flushDuration, versionInterval time.Duration, +) (err error) { // FIXME: cannot disable with a running job db.BGCheckpointRunner.DisableCheckpoint() defer db.BGCheckpointRunner.EnableCheckpoint() db.BGCheckpointRunner.CleanPenddingCheckpoint() t0 := time.Now() err = db.BGCheckpointRunner.ForceFlush(ts, ctx, flushDuration) - logutil.Infof("[Force Global Checkpoint] flush takes %v: %v", time.Since(t0), err) + forceFlushCost := time.Since(t0) + defer func() { + logger := logutil.Info + if err != nil { + logger = logutil.Error + } + logger( + "Control-ForceGlobalCheckpoint", + zap.Duration("total-cost", time.Since(t0)), + zap.Duration("force-flush-cost", forceFlushCost), + zap.Duration("flush-duration", flushDuration), + zap.Duration("version-interval", versionInterval), + zap.Error(err), + ) + }() + if err != nil { - return err - } - if err = db.BGCheckpointRunner.ForceGlobalCheckpointSynchronously(ctx, ts, versionInterval); err != nil { - return err + return } - logutil.Infof("[Force Global Checkpoint] takes %v", time.Since(t0)) + + err = db.BGCheckpointRunner.ForceGlobalCheckpointSynchronously( + ctx, ts, versionInterval, + ) return err } @@ -192,14 +225,29 @@ func (db *DB) ForceCheckpointForBackup( db.BGCheckpointRunner.CleanPenddingCheckpoint() t0 := time.Now() err = db.BGCheckpointRunner.ForceFlush(ts, ctx, flushDuration) - logutil.Infof("[Force Checkpoint] flush takes %v: %v", time.Since(t0), err) + forceFlushCost := time.Since(t0) + + defer func() { + logger := logutil.Info + if err != nil { + logger = logutil.Error + } + logger( + "Control-ForeCheckpointForBackup", + zap.Duration("total-cost", time.Since(t0)), + zap.Duration("force-flush-cost", forceFlushCost), + zap.Duration("flush-duration", flushDuration), + zap.String("location", location), + zap.Error(err), + ) + }() + if err != nil { return } - if location, err = db.BGCheckpointRunner.ForceCheckpointForBackup(ts); err != nil { - return - } - logutil.Debugf("[Force Checkpoint] takes %v", time.Since(t0)) + + location, err = db.BGCheckpointRunner.ForceCheckpointForBackup(ts) + return } diff --git a/pkg/vm/engine/tae/db/open.go b/pkg/vm/engine/tae/db/open.go index df0ace26c563b..802687c2940ce 100644 --- a/pkg/vm/engine/tae/db/open.go +++ b/pkg/vm/engine/tae/db/open.go @@ -51,6 +51,8 @@ import ( const ( WALDir = "wal" + + Phase_Open = "open-tae" ) func fillRuntimeOptions(opts *options.Options) { @@ -78,9 +80,9 @@ func Open( dbLocker, err := createDBLock(dirname) logutil.Info( - "open-tae", - common.OperationField("Start"), - common.OperandField("open"), + Phase_Open, + zap.String("db-dirname", dirname), + zap.Error(err), ) totalTime := time.Now() @@ -92,10 +94,9 @@ func Open( dbLocker.Close() } logutil.Info( - "open-tae", common.OperationField("End"), - common.OperandField("open"), - common.AnyField("cost", time.Since(totalTime)), - common.AnyField("err", err), + Phase_Open, + zap.Duration("open-tae-cost", time.Since(totalTime)), + zap.Error(err), ) }() @@ -105,10 +106,9 @@ func Open( wbuf := &bytes.Buffer{} werr := toml.NewEncoder(wbuf).Encode(opts) logutil.Info( - "open-tae", - common.OperationField("Config"), - common.AnyField("toml", wbuf.String()), - common.ErrorField(werr), + Phase_Open, + zap.String("config", wbuf.String()), + zap.Error(werr), ) serviceDir := path.Join(dirname, "data") if opts.Fs == nil { @@ -205,7 +205,7 @@ func Open( } // 1. replay three tables objectlist - checkpointed, ckpLSN, valid, err := ckpReplayer.ReplayThreeTablesObjectlist() + checkpointed, ckpLSN, valid, err := ckpReplayer.ReplayThreeTablesObjectlist(Phase_Open) if err != nil { panic(err) } @@ -219,20 +219,18 @@ func Open( store.BindTxn(txn) } // 2. replay all table Entries - if err = ckpReplayer.ReplayCatalog(txn); err != nil { + if err = ckpReplayer.ReplayCatalog(txn, Phase_Open); err != nil { panic(err) } // 3. replay other tables' objectlist - if err = ckpReplayer.ReplayObjectlist(); err != nil { + if err = ckpReplayer.ReplayObjectlist(Phase_Open); err != nil { panic(err) } logutil.Info( - "open-tae", - common.OperationField("replay"), - common.OperandField("checkpoints"), - common.AnyField("cost", time.Since(now)), - common.AnyField("checkpointed", checkpointed.ToString()), + Phase_Open, + zap.Duration("replay-checkpoints-cost", time.Since(now)), + zap.String("max-checkpoint", checkpointed.ToString()), ) now = time.Now() @@ -241,10 +239,8 @@ func Open( // checkObjectState(db) logutil.Info( - "open-tae", - common.OperationField("replay"), - common.OperandField("wal"), - common.AnyField("cost", time.Since(now)), + Phase_Open, + zap.Duration("replay-wal-cost", time.Since(now)), ) db.DBLocker, dbLocker = dbLocker, nil diff --git a/pkg/vm/engine/tae/db/replay.go b/pkg/vm/engine/tae/db/replay.go index 8a767b4f4fcc0..7180f953b7dcf 100644 --- a/pkg/vm/engine/tae/db/replay.go +++ b/pkg/vm/engine/tae/db/replay.go @@ -23,11 +23,11 @@ import ( "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/logutil" "github.com/matrixorigin/matrixone/pkg/objectio" + "go.uber.org/zap" "sync" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/catalog" - "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/iface/txnif" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/tables" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/txn/txnbase" @@ -120,11 +120,12 @@ func (replayer *Replayer) Replay() { close(replayer.txnCmdChan) replayer.wg.Wait() replayer.postReplayWal() - logutil.Info("open-tae", common.OperationField("replay"), - common.OperandField("wal"), - common.AnyField("apply logentries cost", replayer.applyDuration), - common.AnyField("read count", replayer.readCount), - common.AnyField("apply count", replayer.applyCount)) + logutil.Info( + "Replay-Wal", + zap.Duration("apply-cost", replayer.applyDuration), + zap.Int("read-count", replayer.readCount), + zap.Int("apply-count", replayer.applyCount), + ) } func (replayer *Replayer) OnReplayEntry(group uint32, lsn uint64, payload []byte, typ uint16, info any) { diff --git a/pkg/vm/engine/tae/logstore/driver/batchstoredriver/store.go b/pkg/vm/engine/tae/logstore/driver/batchstoredriver/store.go index 2703d0a265d40..0b7d9be896ed6 100644 --- a/pkg/vm/engine/tae/logstore/driver/batchstoredriver/store.go +++ b/pkg/vm/engine/tae/logstore/driver/batchstoredriver/store.go @@ -20,10 +20,10 @@ import ( "time" "github.com/matrixorigin/matrixone/pkg/logutil" - "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/driver" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/driver/entry" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/sm" + "go.uber.org/zap" ) var ( @@ -275,11 +275,11 @@ func (bs *baseStore) Replay(h driver.ApplyHandle) error { return err } bs.onReplay(r) - logutil.Info("open-tae", common.OperationField("replay"), - common.OperandField("wal"), - common.AnyField("backend", "batchstore"), - common.AnyField("apply cost", r.applyDuration), - common.AnyField("read cost", r.readDuration)) + logutil.Info( + "Replay-Wal-From-LogStore", + zap.Duration("apply-cost", r.applyDuration), + zap.Duration("read-cost", r.readDuration), + ) return nil } diff --git a/pkg/vm/engine/tae/logstore/driver/logservicedriver/driver.go b/pkg/vm/engine/tae/logstore/driver/logservicedriver/driver.go index f0830def9156e..437c021d0c00c 100644 --- a/pkg/vm/engine/tae/logstore/driver/logservicedriver/driver.go +++ b/pkg/vm/engine/tae/logstore/driver/logservicedriver/driver.go @@ -19,11 +19,11 @@ import ( "time" "github.com/panjf2000/ants/v2" + "go.uber.org/zap" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/common/mpool" "github.com/matrixorigin/matrixone/pkg/logutil" - "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/driver" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/sm" ) @@ -136,14 +136,13 @@ func (d *LogServiceDriver) Replay(h driver.ApplyHandle) error { d.onReplay(r) r.d.resetReadCache() d.PostReplay() - logutil.Info("open-tae", common.OperationField("replay"), - common.OperandField("wal"), - common.AnyField("backend", "logservice"), - common.AnyField("apply cost", r.applyDuration), - common.AnyField("read cost", d.readDuration), - common.AnyField("read count", r.readCount), - common.AnyField("internal count", r.internalCount), - common.AnyField("apply count", r.applyCount), + logutil.Info( + "Replay-Wal-From-LogService", + zap.Duration("apply-cost", r.applyDuration), + zap.Duration("read-cost", d.readDuration), + zap.Int("read-count", r.readCount), + zap.Int("internal-count", r.internalCount), + zap.Int("apply-count", r.applyCount), ) return nil diff --git a/pkg/vm/engine/tae/rpc/handle.go b/pkg/vm/engine/tae/rpc/handle.go index 439a94680a0f0..64a850c6349f4 100644 --- a/pkg/vm/engine/tae/rpc/handle.go +++ b/pkg/vm/engine/tae/rpc/handle.go @@ -578,13 +578,6 @@ func (h *Handle) HandleCreateDatabase( _, span := trace.Start(ctx, "HandleCreateDatabase") defer span.End() - defer func() { - common.DoIfDebugEnabled(func() { - logutil.Debugf("[precommit] create database end txn: %s", txn.String()) - }) - }() - - // modify memory structure for i, c := range req.Cmds { common.DoIfInfoEnabled(func() { logutil.Info( @@ -617,12 +610,8 @@ func (h *Handle) HandleCreateDatabase( func (h *Handle) HandleDropDatabase( ctx context.Context, txn txnif.AsyncTxn, - req *pkgcatalog.DropDatabaseReq) (err error) { - defer func() { - common.DoIfDebugEnabled(func() { - logutil.Debugf("[precommit] drop database end: %s", txn.String()) - }) - }() + req *pkgcatalog.DropDatabaseReq, +) (err error) { for i, c := range req.Cmds { common.DoIfInfoEnabled(func() { logutil.Info( @@ -652,13 +641,8 @@ func (h *Handle) HandleDropDatabase( func (h *Handle) HandleCreateRelation( ctx context.Context, txn txnif.AsyncTxn, - req *pkgcatalog.CreateTableReq) error { - defer func() { - common.DoIfDebugEnabled(func() { - logutil.Debugf("[precommit] create relation end txn: %s", txn.String()) - }) - }() - + req *pkgcatalog.CreateTableReq, +) error { for i, c := range req.Cmds { common.DoIfInfoEnabled(func() { logutil.Info( @@ -698,13 +682,8 @@ func (h *Handle) HandleCreateRelation( func (h *Handle) HandleDropRelation( ctx context.Context, txn txnif.AsyncTxn, - req *pkgcatalog.DropTableReq) error { - defer func() { - common.DoIfDebugEnabled(func() { - logutil.Debugf("[precommit] drop/truncate relation end txn: %s", txn.String()) - }) - }() - + req *pkgcatalog.DropTableReq, +) error { for i, c := range req.Cmds { common.DoIfInfoEnabled(func() { logutil.Info( diff --git a/pkg/vm/engine/tae/tables/aobj.go b/pkg/vm/engine/tae/tables/aobj.go index d9b4a2cb8cd10..e72f0c69fb1c1 100644 --- a/pkg/vm/engine/tae/tables/aobj.go +++ b/pkg/vm/engine/tae/tables/aobj.go @@ -210,17 +210,13 @@ func (obj *aobject) GetDuplicatedRows( node := obj.PinNode() defer node.Unref() if !node.IsPersisted() { - fn := func() (min, max int32, err error) { + fn := func() (minv, maxv int32, err error) { obj.RUnlock() defer obj.RLock() - max, err = obj.GetMaxRowByTS(to) - if err != nil { - return - } - min, err = obj.GetMaxRowByTS(from) - if err != nil { + if maxv, err = obj.GetMaxRowByTS(to); err != nil { return } + minv, err = obj.GetMaxRowByTS(from) return } return node.GetDuplicatedRows( diff --git a/pkg/vm/engine/tae/txn/txnimpl/store.go b/pkg/vm/engine/tae/txn/txnimpl/store.go index 609af6426f745..a29fb1c6d1bfb 100644 --- a/pkg/vm/engine/tae/txn/txnimpl/store.go +++ b/pkg/vm/engine/tae/txn/txnimpl/store.go @@ -746,7 +746,10 @@ func (store *txnStore) PrePrepare(ctx context.Context) (err error) { } approxSize := store.approxSize() if approxSize > MaxWalSize { - return moerr.NewInternalError(ctx, fmt.Sprintf("WAL entry approxSize %d is too large, max is %d", approxSize, MaxWalSize)) + return moerr.NewInternalError( + ctx, + fmt.Sprintf("txn %x WAL entry approxSize %d is too large, max is %d", store.txn.GetID(), approxSize, MaxWalSize), + ) } if approxSize > 50*mpool.MB { logutil.Warnf("[Large-WAL-Entry]txn %x, WAL entry approxSize %d", store.txn.GetID(), approxSize) diff --git a/pkg/vm/engine/tae/txn/txnimpl/txndb.go b/pkg/vm/engine/tae/txn/txnimpl/txndb.go index 2dce8c5e37c80..a97b588e733f7 100644 --- a/pkg/vm/engine/tae/txn/txnimpl/txndb.go +++ b/pkg/vm/engine/tae/txn/txnimpl/txndb.go @@ -29,6 +29,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/containers" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/iface/handle" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/iface/txnif" + "go.uber.org/zap" ) type txnDB struct { @@ -502,7 +503,16 @@ func (db *txnDB) PrePrepare(ctx context.Context) (err error) { return } } - v2.TxnTNPrePrepareDeduplicateDurationHistogram.Observe(time.Since(now).Seconds()) + duration := time.Since(now) + v2.TxnTNPrePrepareDeduplicateDurationHistogram.Observe(duration.Seconds()) + + if duration > time.Millisecond*500 { + logutil.Info( + "SLOW-LOG-PrePrepareDedup", + zap.String("txn", db.store.txn.String()), + zap.Duration("duration", duration), + ) + } for _, table := range db.tables { if err = table.PrePrepare(); err != nil {